1
0

[HUDI-2177][HUDI-2200] Adding virtual keys support for MOR table (#3315)

This commit is contained in:
Sivabalan Narayanan
2021-08-02 09:45:09 -04:00
committed by GitHub
parent dde57b293c
commit fe508376fa
37 changed files with 633 additions and 261 deletions

View File

@@ -157,6 +157,11 @@ public class HoodieTableConfig extends HoodieConfig implements Serializable {
.withDocumentation("When enabled, populates all meta fields. When disabled, no meta fields are populated "
+ "and incremental queries will not be functional. This is only meant to be used for append only/immutable data for batch processing");
public static final ConfigProperty<String> HOODIE_TABLE_KEY_GENERATOR_CLASS = ConfigProperty
.key("hoodie.table.keygenerator.class")
.noDefaultValue()
.withDocumentation("Key Generator class property for the hoodie table");
public static final String NO_OP_BOOTSTRAP_INDEX_CLASS = NoOpBootstrapIndex.class.getName();
public HoodieTableConfig(FileSystem fs, String metaPath, String payloadClassName) {
@@ -276,6 +281,13 @@ public class HoodieTableConfig extends HoodieConfig implements Serializable {
return Option.empty();
}
/**
* @returns the partition field prop.
*/
public String getPartitionFieldProp() {
return getString(HOODIE_TABLE_PARTITION_FIELDS_PROP);
}
/**
* Read the payload class for HoodieRecords from the table properties.
*/
@@ -344,6 +356,13 @@ public class HoodieTableConfig extends HoodieConfig implements Serializable {
return Boolean.parseBoolean(getStringOrDefault(HOODIE_POPULATE_META_FIELDS));
}
/**
* @returns the record key field prop.
*/
public String getRecordKeyFieldProp() {
return getString(HOODIE_TABLE_RECORDKEY_FIELDS);
}
public Map<String, String> propsMap() {
return props.entrySet().stream()
.collect(Collectors.toMap(e -> String.valueOf(e.getKey()), e -> String.valueOf(e.getValue())));

View File

@@ -324,6 +324,14 @@ public class HoodieTableMetaClient implements Serializable {
&& Boolean.parseBoolean((String) properties.getOrDefault(HoodieTableConfig.HOODIE_POPULATE_META_FIELDS.key(), HoodieTableConfig.HOODIE_POPULATE_META_FIELDS.defaultValue()))) {
throw new HoodieException(HoodieTableConfig.HOODIE_POPULATE_META_FIELDS.key() + " already disabled for the table. Can't be re-enabled back");
}
// meta fields can be disabled only with SimpleKeyGenerator
if (!getTableConfig().populateMetaFields()
&& !properties.getProperty(HoodieTableConfig.HOODIE_TABLE_KEY_GENERATOR_CLASS.key(), "org.apache.hudi.keygen.SimpleKeyGenerator")
.equals("org.apache.hudi.keygen.SimpleKeyGenerator")) {
throw new HoodieException("Only simple key generator is supported when meta fields are disabled. KeyGenerator used : "
+ properties.getProperty(HoodieTableConfig.HOODIE_TABLE_KEY_GENERATOR_CLASS.key()));
}
}
/**
@@ -617,6 +625,7 @@ public class HoodieTableMetaClient implements Serializable {
private String bootstrapIndexClass;
private String bootstrapBasePath;
private Boolean populateMetaFields;
private String keyGeneratorClassProp;
private PropertyBuilder() {
@@ -695,6 +704,11 @@ public class HoodieTableMetaClient implements Serializable {
return this;
}
public PropertyBuilder setKeyGeneratorClassProp(String keyGeneratorClassProp) {
this.keyGeneratorClassProp = keyGeneratorClassProp;
return this;
}
public PropertyBuilder fromMetaClient(HoodieTableMetaClient metaClient) {
return setTableType(metaClient.getTableType())
.setTableName(metaClient.getTableConfig().getTableName())
@@ -748,6 +762,9 @@ public class HoodieTableMetaClient implements Serializable {
if (hoodieConfig.contains(HoodieTableConfig.HOODIE_POPULATE_META_FIELDS)) {
setPopulateMetaFields(hoodieConfig.getBoolean(HoodieTableConfig.HOODIE_POPULATE_META_FIELDS));
}
if (hoodieConfig.contains(HoodieTableConfig.HOODIE_TABLE_KEY_GENERATOR_CLASS)) {
setKeyGeneratorClassProp(hoodieConfig.getString(HoodieTableConfig.HOODIE_TABLE_KEY_GENERATOR_CLASS));
}
return this;
}
@@ -804,6 +821,9 @@ public class HoodieTableMetaClient implements Serializable {
if (null != populateMetaFields) {
tableConfig.setValue(HoodieTableConfig.HOODIE_POPULATE_META_FIELDS, Boolean.toString(populateMetaFields));
}
if (null != keyGeneratorClassProp) {
tableConfig.setValue(HoodieTableConfig.HOODIE_TABLE_KEY_GENERATOR_CLASS, keyGeneratorClassProp);
}
return tableConfig.getProps();
}

View File

@@ -18,15 +18,11 @@
package org.apache.hudi.common.table.log;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.common.model.HoodieLogFile;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.common.table.HoodieTableConfig;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.log.block.HoodieAvroDataBlock;
import org.apache.hudi.common.table.log.block.HoodieCommandBlock;
@@ -37,8 +33,15 @@ import org.apache.hudi.common.table.log.block.HoodieLogBlock;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.SpillableMapUtils;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
@@ -80,6 +83,8 @@ public abstract class AbstractHoodieLogRecordScanner {
private final HoodieTableMetaClient hoodieTableMetaClient;
// Merge strategy to use when combining records from log
private final String payloadClassFQN;
// simple key gen fields
private Option<Pair<String, String>> simpleKeyGenFields = Option.empty();
// Log File Paths
protected final List<String> logFilePaths;
// Read Lazily flag
@@ -115,6 +120,10 @@ public abstract class AbstractHoodieLogRecordScanner {
this.hoodieTableMetaClient = HoodieTableMetaClient.builder().setConf(fs.getConf()).setBasePath(basePath).build();
// load class from the payload fully qualified class name
this.payloadClassFQN = this.hoodieTableMetaClient.getTableConfig().getPayloadClass();
HoodieTableConfig tableConfig = this.hoodieTableMetaClient.getTableConfig();
if (!tableConfig.populateMetaFields()) {
this.simpleKeyGenFields = Option.of(Pair.of(tableConfig.getRecordKeyFieldProp(), tableConfig.getPartitionFieldProp()));
}
this.totalLogFiles.addAndGet(logFilePaths.size());
this.logFilePaths = logFilePaths;
this.readBlocksLazily = readBlocksLazily;
@@ -302,7 +311,11 @@ public abstract class AbstractHoodieLogRecordScanner {
}
protected HoodieRecord<?> createHoodieRecord(IndexedRecord rec) {
return SpillableMapUtils.convertToHoodieRecordPayload((GenericRecord) rec, this.payloadClassFQN);
if (!simpleKeyGenFields.isPresent()) {
return SpillableMapUtils.convertToHoodieRecordPayload((GenericRecord) rec, this.payloadClassFQN);
} else {
return SpillableMapUtils.convertToHoodieRecordPayload((GenericRecord) rec, this.payloadClassFQN, this.simpleKeyGenFields.get());
}
}
/**

View File

@@ -18,13 +18,16 @@
package org.apache.hudi.common.table.log;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.SpillableMapUtils;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.io.storage.HoodieFileReader;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.common.util.SpillableMapUtils;
import org.apache.hudi.io.storage.HoodieFileReader;
import java.io.IOException;
import java.util.Iterator;
@@ -36,11 +39,14 @@ public class HoodieFileSliceReader implements Iterator<HoodieRecord<? extends Ho
private Iterator<HoodieRecord<? extends HoodieRecordPayload>> recordsIterator;
public static <R extends IndexedRecord, T extends HoodieRecordPayload> HoodieFileSliceReader getFileSliceReader(
HoodieFileReader<R> baseFileReader, HoodieMergedLogRecordScanner scanner, Schema schema, String payloadClass) throws IOException {
HoodieFileReader<R> baseFileReader, HoodieMergedLogRecordScanner scanner, Schema schema, String payloadClass,
Option<Pair<String,String>> simpleKeyGenFieldsOpt) throws IOException {
Iterator<R> baseIterator = baseFileReader.getRecordIterator(schema);
while (baseIterator.hasNext()) {
GenericRecord record = (GenericRecord) baseIterator.next();
HoodieRecord<T> hoodieRecord = SpillableMapUtils.convertToHoodieRecordPayload(record, payloadClass);
GenericRecord record = (GenericRecord) baseIterator.next();
HoodieRecord<T> hoodieRecord = simpleKeyGenFieldsOpt.isPresent()
? SpillableMapUtils.convertToHoodieRecordPayload(record, payloadClass, simpleKeyGenFieldsOpt.get())
: SpillableMapUtils.convertToHoodieRecordPayload(record, payloadClass);
scanner.processNextRecord(hoodieRecord);
}
return new HoodieFileSliceReader(scanner.iterator());

View File

@@ -23,6 +23,7 @@ import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.common.util.collection.BitCaskDiskMap.FileEntry;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.exception.HoodieCorruptedDataException;
import org.apache.avro.generic.GenericRecord;
@@ -110,8 +111,15 @@ public class SpillableMapUtils {
* Utility method to convert bytes to HoodieRecord using schema and payload class.
*/
public static <R> R convertToHoodieRecordPayload(GenericRecord rec, String payloadClazz) {
String recKey = rec.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString();
String partitionPath = rec.get(HoodieRecord.PARTITION_PATH_METADATA_FIELD).toString();
return convertToHoodieRecordPayload(rec, payloadClazz, Pair.of(HoodieRecord.RECORD_KEY_METADATA_FIELD, HoodieRecord.PARTITION_PATH_METADATA_FIELD));
}
/**
* Utility method to convert bytes to HoodieRecord using schema and payload class.
*/
public static <R> R convertToHoodieRecordPayload(GenericRecord rec, String payloadClazz, Pair<String, String> recordKeyPartitionPathPair) {
String recKey = rec.get(recordKeyPartitionPathPair.getLeft()).toString();
String partitionPath = rec.get(recordKeyPartitionPathPair.getRight()).toString();
HoodieRecord<? extends HoodieRecordPayload> hoodieRecord = new HoodieRecord<>(new HoodieKey(recKey, partitionPath),
ReflectionUtils.loadPayload(payloadClazz, new Object[] {Option.of(rec)}, Option.class));
return (R) hoodieRecord;

View File

@@ -29,6 +29,7 @@ import org.apache.hudi.common.model.HoodieBaseFile;
import org.apache.hudi.common.model.HoodieLogFile;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.common.table.HoodieTableConfig;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.timeline.HoodieDefaultTimeline;
import org.apache.hudi.common.table.timeline.HoodieInstant;
@@ -37,6 +38,7 @@ import org.apache.hudi.common.util.HoodieTimer;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.SpillableMapUtils;
import org.apache.hudi.common.util.ValidationUtils;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.hudi.exception.TableNotFoundException;
@@ -70,6 +72,7 @@ public class HoodieBackedTableMetadata extends BaseTableMetadata {
private String metadataBasePath;
// Metadata table's timeline and metaclient
private HoodieTableMetaClient metaClient;
private HoodieTableConfig tableConfig;
private List<FileSlice> latestFileSystemMetadataSlices;
// should we reuse the open file handles, across calls
private final boolean reuse;
@@ -98,16 +101,19 @@ public class HoodieBackedTableMetadata extends BaseTableMetadata {
this.metadataBasePath = HoodieTableMetadata.getMetadataTableBasePath(datasetBasePath);
try {
this.metaClient = HoodieTableMetaClient.builder().setConf(hadoopConf.get()).setBasePath(metadataBasePath).build();
this.tableConfig = metaClient.getTableConfig();
HoodieTableFileSystemView fsView = new HoodieTableFileSystemView(metaClient, metaClient.getActiveTimeline());
latestFileSystemMetadataSlices = fsView.getLatestFileSlices(MetadataPartitionType.FILES.partitionPath()).collect(Collectors.toList());
} catch (TableNotFoundException e) {
LOG.warn("Metadata table was not found at path " + metadataBasePath);
this.enabled = false;
this.metaClient = null;
this.tableConfig = null;
} catch (Exception e) {
LOG.error("Failed to initialize metadata table at path " + metadataBasePath, e);
this.enabled = false;
this.metaClient = null;
this.tableConfig = null;
}
}
}
@@ -126,8 +132,10 @@ public class HoodieBackedTableMetadata extends BaseTableMetadata {
HoodieTimer readTimer = new HoodieTimer().startTimer();
Option<GenericRecord> baseRecord = baseFileReader.getRecordByKey(key);
if (baseRecord.isPresent()) {
hoodieRecord = SpillableMapUtils.convertToHoodieRecordPayload(baseRecord.get(),
metaClient.getTableConfig().getPayloadClass());
hoodieRecord = tableConfig.populateMetaFields() ? SpillableMapUtils.convertToHoodieRecordPayload(baseRecord.get(),
tableConfig.getPayloadClass()) : SpillableMapUtils.convertToHoodieRecordPayload(baseRecord.get(),
tableConfig.getPayloadClass(), Pair.of(tableConfig.getRecordKeyFieldProp(),
tableConfig.getPartitionFieldProp()));
metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.BASEFILE_READ_STR, readTimer.endTimer()));
}
}