[HUDI-2763] Metadata table records - support for key deduplication based on hardcoded key field (#4449)
* [HUDI-2763] Metadata table records - support for key deduplication and virtual keys - The backing log format for the metadata table is HFile, a KeyValue type. Since the key field in the metadata record payload is a duplicate of the Key in the Cell, the redundant key field in the record can be emptied to save on the cost. - HoodieHFileWriter and HoodieHFileDataBlock will now serialize records with the key field emptied by default. HFile writer tries to find if the record has metadata payload schema field 'key' and if so it does the key trimming from the record payload. - HoodieHFileReader when reading the serialized records back from disk, it materializes the missing keyFields if any. HFile reader tries to find if the record has metadata payload schema fiels 'key' and if so it does the key materialization in the record payload. - Tests have been added to verify the default virtual keys and key deduplication support for the metadata table records. Co-authored-by: Vinoth Chandar <vinoth@apache.org>
This commit is contained in:
committed by
GitHub
parent
dd4ce1bdfd
commit
f87c47352a
@@ -126,7 +126,7 @@ public final class HoodieMetadataConfig extends HoodieConfig {
|
||||
|
||||
public static final ConfigProperty<Boolean> POPULATE_META_FIELDS = ConfigProperty
|
||||
.key(METADATA_PREFIX + ".populate.meta.fields")
|
||||
.defaultValue(true)
|
||||
.defaultValue(false)
|
||||
.sinceVersion("0.10.0")
|
||||
.withDocumentation("When enabled, populates all meta fields. When disabled, no meta fields are populated.");
|
||||
|
||||
|
||||
@@ -22,8 +22,8 @@ import org.apache.hudi.avro.HoodieAvroUtils;
|
||||
import org.apache.hudi.common.fs.inline.InLineFSUtils;
|
||||
import org.apache.hudi.common.fs.inline.InLineFileSystem;
|
||||
import org.apache.hudi.common.model.HoodieLogFile;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.StringUtils;
|
||||
import org.apache.hudi.common.util.ValidationUtils;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.io.storage.HoodieHBaseKVComparator;
|
||||
@@ -83,10 +83,6 @@ public class HoodieHFileDataBlock extends HoodieDataBlock {
|
||||
super(records, header, new HashMap<>(), keyField);
|
||||
}
|
||||
|
||||
public HoodieHFileDataBlock(@Nonnull List<IndexedRecord> records, @Nonnull Map<HeaderMetadataType, String> header) {
|
||||
this(records, header, HoodieRecord.RECORD_KEY_METADATA_FIELD);
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieLogBlockType getBlockType() {
|
||||
return HoodieLogBlockType.HFILE_DATA_BLOCK;
|
||||
@@ -110,8 +106,8 @@ public class HoodieHFileDataBlock extends HoodieDataBlock {
|
||||
boolean useIntegerKey = false;
|
||||
int key = 0;
|
||||
int keySize = 0;
|
||||
Field keyField = records.get(0).getSchema().getField(this.keyField);
|
||||
if (keyField == null) {
|
||||
final Field keyFieldSchema = records.get(0).getSchema().getField(HoodieHFileReader.KEY_FIELD_NAME);
|
||||
if (keyFieldSchema == null) {
|
||||
// Missing key metadata field so we should use an integer sequence key
|
||||
useIntegerKey = true;
|
||||
keySize = (int) Math.ceil(Math.log(records.size())) + 1;
|
||||
@@ -122,9 +118,9 @@ public class HoodieHFileDataBlock extends HoodieDataBlock {
|
||||
if (useIntegerKey) {
|
||||
recordKey = String.format("%" + keySize + "s", key++);
|
||||
} else {
|
||||
recordKey = record.get(keyField.pos()).toString();
|
||||
recordKey = record.get(keyFieldSchema.pos()).toString();
|
||||
}
|
||||
byte[] recordBytes = HoodieAvroUtils.indexedRecordToBytes(record);
|
||||
final byte[] recordBytes = serializeRecord(record, Option.ofNullable(keyFieldSchema));
|
||||
ValidationUtils.checkState(!sortedRecordsMap.containsKey(recordKey),
|
||||
"Writing multiple records with same key not supported for " + this.getClass().getName());
|
||||
sortedRecordsMap.put(recordKey, recordBytes);
|
||||
@@ -162,6 +158,20 @@ public class HoodieHFileDataBlock extends HoodieDataBlock {
|
||||
return records;
|
||||
}
|
||||
|
||||
/**
|
||||
* Serialize the record to byte buffer.
|
||||
*
|
||||
* @param record - Record to serialize
|
||||
* @param keyField - Key field in the schema
|
||||
* @return Serialized byte buffer for the record
|
||||
*/
|
||||
private byte[] serializeRecord(final IndexedRecord record, final Option<Field> keyField) {
|
||||
if (keyField.isPresent()) {
|
||||
record.put(keyField.get().pos(), StringUtils.EMPTY_STRING);
|
||||
}
|
||||
return HoodieAvroUtils.indexedRecordToBytes(record);
|
||||
}
|
||||
|
||||
private void readWithInlineFS(List<String> keys) throws IOException {
|
||||
boolean enableFullScan = keys.isEmpty();
|
||||
// Get schema from the header
|
||||
|
||||
@@ -50,6 +50,7 @@ import org.apache.hudi.common.bloom.BloomFilter;
|
||||
import org.apache.hudi.common.bloom.BloomFilterFactory;
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.ValidationUtils;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
|
||||
@@ -63,6 +64,7 @@ public class HoodieHFileReader<R extends IndexedRecord> implements HoodieFileRea
|
||||
// key retrieval.
|
||||
private HFileScanner keyScanner;
|
||||
|
||||
public static final String KEY_FIELD_NAME = "key";
|
||||
public static final String KEY_SCHEMA = "schema";
|
||||
public static final String KEY_BLOOM_FILTER_META_BLOCK = "bloomFilter";
|
||||
public static final String KEY_BLOOM_FILTER_TYPE_CODE = "bloomFilterTypeCode";
|
||||
@@ -151,15 +153,15 @@ public class HoodieHFileReader<R extends IndexedRecord> implements HoodieFileRea
|
||||
}
|
||||
|
||||
public List<Pair<String, R>> readAllRecords(Schema writerSchema, Schema readerSchema) throws IOException {
|
||||
final Option<Schema.Field> keyFieldSchema = Option.ofNullable(readerSchema.getField(KEY_FIELD_NAME));
|
||||
List<Pair<String, R>> recordList = new LinkedList<>();
|
||||
try {
|
||||
final HFileScanner scanner = reader.getScanner(false, false);
|
||||
if (scanner.seekTo()) {
|
||||
do {
|
||||
Cell c = scanner.getKeyValue();
|
||||
byte[] keyBytes = Arrays.copyOfRange(c.getRowArray(), c.getRowOffset(), c.getRowOffset() + c.getRowLength());
|
||||
R record = getRecordFromCell(c, writerSchema, readerSchema);
|
||||
recordList.add(new Pair<>(new String(keyBytes), record));
|
||||
final Pair<String, R> keyAndRecordPair = getRecordFromCell(c, writerSchema, readerSchema, keyFieldSchema);
|
||||
recordList.add(keyAndRecordPair);
|
||||
} while (scanner.next());
|
||||
}
|
||||
|
||||
@@ -196,6 +198,9 @@ public class HoodieHFileReader<R extends IndexedRecord> implements HoodieFileRea
|
||||
@Override
|
||||
public Iterator getRecordIterator(Schema readerSchema) throws IOException {
|
||||
final HFileScanner scanner = reader.getScanner(false, false);
|
||||
final Option<Schema.Field> keyFieldSchema = Option.ofNullable(readerSchema.getField(KEY_FIELD_NAME));
|
||||
ValidationUtils.checkState(keyFieldSchema != null,
|
||||
"Missing key field '" + KEY_FIELD_NAME + "' in the schema!");
|
||||
return new Iterator<R>() {
|
||||
private R next = null;
|
||||
private boolean eof = false;
|
||||
@@ -206,7 +211,8 @@ public class HoodieHFileReader<R extends IndexedRecord> implements HoodieFileRea
|
||||
// To handle when hasNext() is called multiple times for idempotency and/or the first time
|
||||
if (this.next == null && !this.eof) {
|
||||
if (!scanner.isSeeked() && scanner.seekTo()) {
|
||||
this.next = getRecordFromCell(scanner.getKeyValue(), getSchema(), readerSchema);
|
||||
final Pair<String, R> keyAndRecordPair = getRecordFromCell(scanner.getKeyValue(), getSchema(), readerSchema, keyFieldSchema);
|
||||
this.next = keyAndRecordPair.getSecond();
|
||||
}
|
||||
}
|
||||
return this.next != null;
|
||||
@@ -226,7 +232,8 @@ public class HoodieHFileReader<R extends IndexedRecord> implements HoodieFileRea
|
||||
}
|
||||
R retVal = this.next;
|
||||
if (scanner.next()) {
|
||||
this.next = getRecordFromCell(scanner.getKeyValue(), getSchema(), readerSchema);
|
||||
final Pair<String, R> keyAndRecordPair = getRecordFromCell(scanner.getKeyValue(), getSchema(), readerSchema, keyFieldSchema);
|
||||
this.next = keyAndRecordPair.getSecond();
|
||||
} else {
|
||||
this.next = null;
|
||||
this.eof = true;
|
||||
@@ -242,6 +249,8 @@ public class HoodieHFileReader<R extends IndexedRecord> implements HoodieFileRea
|
||||
@Override
|
||||
public Option getRecordByKey(String key, Schema readerSchema) throws IOException {
|
||||
byte[] value = null;
|
||||
final Option<Schema.Field> keyFieldSchema = Option.ofNullable(readerSchema.getField(KEY_FIELD_NAME));
|
||||
ValidationUtils.checkState(keyFieldSchema != null);
|
||||
KeyValue kv = new KeyValue(key.getBytes(), null, null, null);
|
||||
|
||||
synchronized (this) {
|
||||
@@ -257,16 +266,51 @@ public class HoodieHFileReader<R extends IndexedRecord> implements HoodieFileRea
|
||||
}
|
||||
|
||||
if (value != null) {
|
||||
R record = (R)HoodieAvroUtils.bytesToAvro(value, getSchema(), readerSchema);
|
||||
R record = deserialize(key.getBytes(), value, getSchema(), readerSchema, keyFieldSchema);
|
||||
return Option.of(record);
|
||||
}
|
||||
|
||||
return Option.empty();
|
||||
}
|
||||
|
||||
private R getRecordFromCell(Cell c, Schema writerSchema, Schema readerSchema) throws IOException {
|
||||
byte[] value = Arrays.copyOfRange(c.getValueArray(), c.getValueOffset(), c.getValueOffset() + c.getValueLength());
|
||||
return (R)HoodieAvroUtils.bytesToAvro(value, writerSchema, readerSchema);
|
||||
private Pair<String, R> getRecordFromCell(Cell cell, Schema writerSchema, Schema readerSchema, Option<Schema.Field> keyFieldSchema) throws IOException {
|
||||
final byte[] keyBytes = Arrays.copyOfRange(cell.getRowArray(), cell.getRowOffset(), cell.getRowOffset() + cell.getRowLength());
|
||||
final byte[] valueBytes = Arrays.copyOfRange(cell.getValueArray(), cell.getValueOffset(), cell.getValueOffset() + cell.getValueLength());
|
||||
R record = deserialize(keyBytes, valueBytes, writerSchema, readerSchema, keyFieldSchema);
|
||||
return new Pair<>(new String(keyBytes), record);
|
||||
}
|
||||
|
||||
/**
|
||||
* Deserialize the record byte array contents to record object.
|
||||
*
|
||||
* @param keyBytes - Record key as byte array
|
||||
* @param valueBytes - Record content as byte array
|
||||
* @param writerSchema - Writer schema
|
||||
* @param readerSchema - Reader schema
|
||||
* @param keyFieldSchema - Key field id in the schema
|
||||
* @return Deserialized record object
|
||||
*/
|
||||
private R deserialize(final byte[] keyBytes, final byte[] valueBytes, Schema writerSchema, Schema readerSchema,
|
||||
Option<Schema.Field> keyFieldSchema) throws IOException {
|
||||
R record = (R) HoodieAvroUtils.bytesToAvro(valueBytes, writerSchema, readerSchema);
|
||||
materializeRecordIfNeeded(keyBytes, record, keyFieldSchema);
|
||||
return record;
|
||||
}
|
||||
|
||||
/**
|
||||
* Materialize the record for any missing fields, if needed.
|
||||
*
|
||||
* @param keyBytes - Key byte array
|
||||
* @param record - Record object to materialize
|
||||
* @param keyFieldSchema - Key field id in the schema
|
||||
*/
|
||||
private void materializeRecordIfNeeded(final byte[] keyBytes, R record, Option<Schema.Field> keyFieldSchema) {
|
||||
if (keyFieldSchema.isPresent()) {
|
||||
final Object keyObject = record.get(keyFieldSchema.get().pos());
|
||||
if (keyObject != null && keyObject.toString().isEmpty()) {
|
||||
record.put(keyFieldSchema.get().pos(), new String(keyBytes));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
|
||||
@@ -139,7 +139,7 @@ public class HoodieMetadataMergedLogRecordReader extends HoodieMergedLogRecordSc
|
||||
|
||||
@Override
|
||||
protected String getKeyField() {
|
||||
return HoodieMetadataPayload.SCHEMA_FIELD_ID_KEY;
|
||||
return HoodieMetadataPayload.KEY_FIELD_NAME;
|
||||
}
|
||||
|
||||
/**
|
||||
|
||||
@@ -33,6 +33,7 @@ import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.io.storage.HoodieHFileReader;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
@@ -63,9 +64,9 @@ import static org.apache.hudi.metadata.HoodieTableMetadata.RECORDKEY_PARTITION_L
|
||||
public class HoodieMetadataPayload implements HoodieRecordPayload<HoodieMetadataPayload> {
|
||||
|
||||
// HoodieMetadata schema field ids
|
||||
public static final String SCHEMA_FIELD_ID_KEY = "key";
|
||||
public static final String SCHEMA_FIELD_ID_TYPE = "type";
|
||||
public static final String SCHEMA_FIELD_ID_METADATA = "filesystemMetadata";
|
||||
public static final String KEY_FIELD_NAME = HoodieHFileReader.KEY_FIELD_NAME;
|
||||
public static final String SCHEMA_FIELD_NAME_TYPE = "type";
|
||||
public static final String SCHEMA_FIELD_NAME_METADATA = "filesystemMetadata";
|
||||
|
||||
// Type of the record
|
||||
// This can be an enum in the schema but Avro 1.8 has a bug - https://issues.apache.org/jira/browse/AVRO-1810
|
||||
@@ -84,9 +85,9 @@ public class HoodieMetadataPayload implements HoodieRecordPayload<HoodieMetadata
|
||||
if (record.isPresent()) {
|
||||
// This can be simplified using SpecificData.deepcopy once this bug is fixed
|
||||
// https://issues.apache.org/jira/browse/AVRO-1811
|
||||
key = record.get().get(SCHEMA_FIELD_ID_KEY).toString();
|
||||
type = (int) record.get().get(SCHEMA_FIELD_ID_TYPE);
|
||||
if (record.get().get(SCHEMA_FIELD_ID_METADATA) != null) {
|
||||
key = record.get().get(KEY_FIELD_NAME).toString();
|
||||
type = (int) record.get().get(SCHEMA_FIELD_NAME_TYPE);
|
||||
if (record.get().get(SCHEMA_FIELD_NAME_METADATA) != null) {
|
||||
filesystemMetadata = (Map<String, HoodieMetadataFileInfo>) record.get().get("filesystemMetadata");
|
||||
filesystemMetadata.keySet().forEach(k -> {
|
||||
GenericRecord v = filesystemMetadata.get(k);
|
||||
@@ -237,8 +238,8 @@ public class HoodieMetadataPayload implements HoodieRecordPayload<HoodieMetadata
|
||||
@Override
|
||||
public String toString() {
|
||||
final StringBuilder sb = new StringBuilder("HoodieMetadataPayload {");
|
||||
sb.append(SCHEMA_FIELD_ID_KEY + "=").append(key).append(", ");
|
||||
sb.append(SCHEMA_FIELD_ID_TYPE + "=").append(type).append(", ");
|
||||
sb.append(KEY_FIELD_NAME + "=").append(key).append(", ");
|
||||
sb.append(SCHEMA_FIELD_NAME_TYPE + "=").append(type).append(", ");
|
||||
sb.append("creations=").append(Arrays.toString(getFilenames().toArray())).append(", ");
|
||||
sb.append("deletions=").append(Arrays.toString(getDeletions().toArray())).append(", ");
|
||||
sb.append('}');
|
||||
|
||||
Reference in New Issue
Block a user