[HUDI-2656] Generalize HoodieIndex for flexible record data type (#3893)
Co-authored-by: Raymond Xu <2701446+xushiyan@users.noreply.github.com>
This commit is contained in:
@@ -0,0 +1,50 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you 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 org.apache.hudi.common.model;
|
||||
|
||||
public class HoodieAvroRecord<T extends HoodieRecordPayload> extends HoodieRecord<T> {
|
||||
public HoodieAvroRecord(HoodieKey key, T data) {
|
||||
super(key, data);
|
||||
}
|
||||
|
||||
public HoodieAvroRecord(HoodieKey key, T data, HoodieOperation operation) {
|
||||
super(key, data, operation);
|
||||
}
|
||||
|
||||
public HoodieAvroRecord(HoodieRecord<T> record) {
|
||||
super(record);
|
||||
}
|
||||
|
||||
public HoodieAvroRecord() {
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieRecord<T> newInstance() {
|
||||
return new HoodieAvroRecord<>(this);
|
||||
}
|
||||
|
||||
@Override
|
||||
public T getData() {
|
||||
if (data == null) {
|
||||
throw new IllegalStateException("Payload already deflated for record.");
|
||||
}
|
||||
return data;
|
||||
}
|
||||
}
|
||||
@@ -18,21 +18,21 @@
|
||||
|
||||
package org.apache.hudi.common.model;
|
||||
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.IntStream;
|
||||
import org.apache.hudi.common.util.CollectionUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
|
||||
import java.io.Serializable;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.IntStream;
|
||||
|
||||
/**
|
||||
* A Single Record managed by Hoodie.
|
||||
*/
|
||||
public class HoodieRecord<T extends HoodieRecordPayload> implements Serializable {
|
||||
public abstract class HoodieRecord<T> implements Serializable {
|
||||
|
||||
public static final String COMMIT_TIME_METADATA_FIELD = "_hoodie_commit_time";
|
||||
public static final String COMMIT_SEQNO_METADATA_FIELD = "_hoodie_commit_seqno";
|
||||
@@ -64,7 +64,7 @@ public class HoodieRecord<T extends HoodieRecordPayload> implements Serializable
|
||||
/**
|
||||
* Actual payload of the record.
|
||||
*/
|
||||
private T data;
|
||||
protected T data;
|
||||
|
||||
/**
|
||||
* Current location of record on storage. Filled in by looking up index
|
||||
@@ -110,6 +110,8 @@ public class HoodieRecord<T extends HoodieRecordPayload> implements Serializable
|
||||
public HoodieRecord() {
|
||||
}
|
||||
|
||||
public abstract HoodieRecord<T> newInstance();
|
||||
|
||||
public HoodieKey getKey() {
|
||||
return key;
|
||||
}
|
||||
|
||||
@@ -18,6 +18,7 @@
|
||||
|
||||
package org.apache.hudi.common.table.log;
|
||||
|
||||
import org.apache.hudi.common.model.HoodieAvroRecord;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieLogFile;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
@@ -384,7 +385,7 @@ public abstract class AbstractHoodieLogRecordReader {
|
||||
* @param partitionName - Partition name
|
||||
* @return HoodieRecord created from the IndexedRecord
|
||||
*/
|
||||
protected HoodieRecord<?> createHoodieRecord(final IndexedRecord rec, final HoodieTableConfig hoodieTableConfig,
|
||||
protected HoodieAvroRecord<?> createHoodieRecord(final IndexedRecord rec, final HoodieTableConfig hoodieTableConfig,
|
||||
final String payloadClassFQN, final String preCombineField,
|
||||
final boolean withOperationField,
|
||||
final Option<Pair<String, String>> simpleKeyGenFields,
|
||||
|
||||
@@ -19,6 +19,7 @@
|
||||
package org.apache.hudi.common.table.log;
|
||||
|
||||
import org.apache.hudi.common.config.HoodieCommonConfig;
|
||||
import org.apache.hudi.common.model.HoodieAvroRecord;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieOperation;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
@@ -144,7 +145,7 @@ public class HoodieMergedLogRecordScanner extends AbstractHoodieLogRecordReader
|
||||
HoodieRecordPayload combinedValue = hoodieRecord.getData().preCombine(oldValue);
|
||||
boolean choosePrev = combinedValue.equals(oldValue);
|
||||
HoodieOperation operation = choosePrev ? oldRecord.getOperation() : hoodieRecord.getOperation();
|
||||
records.put(key, new HoodieRecord<>(new HoodieKey(key, hoodieRecord.getPartitionPath()), combinedValue, operation));
|
||||
records.put(key, new HoodieAvroRecord<>(new HoodieKey(key, hoodieRecord.getPartitionPath()), combinedValue, operation));
|
||||
} else {
|
||||
// Put the record as is
|
||||
records.put(key, hoodieRecord);
|
||||
|
||||
@@ -19,6 +19,7 @@
|
||||
package org.apache.hudi.common.util;
|
||||
|
||||
import org.apache.hudi.common.fs.SizeAwareDataOutputStream;
|
||||
import org.apache.hudi.common.model.HoodieAvroRecord;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieOperation;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
@@ -144,7 +145,7 @@ public class SpillableMapUtils {
|
||||
Object preCombineVal = getPreCombineVal(record, preCombineField);
|
||||
HoodieOperation operation = withOperationField
|
||||
? HoodieOperation.fromName(getNullableValAsString(record, HoodieRecord.OPERATION_METADATA_FIELD)) : null;
|
||||
HoodieRecord<? extends HoodieRecordPayload> hoodieRecord = new HoodieRecord<>(new HoodieKey(recKey, partitionPath),
|
||||
HoodieRecord<? extends HoodieRecordPayload> hoodieRecord = new HoodieAvroRecord<>(new HoodieKey(recKey, partitionPath),
|
||||
ReflectionUtils.loadPayload(payloadClazz, new Object[]{record, preCombineVal}, GenericRecord.class,
|
||||
Comparable.class), operation);
|
||||
|
||||
@@ -170,7 +171,7 @@ public class SpillableMapUtils {
|
||||
* Utility method to convert bytes to HoodieRecord using schema and payload class.
|
||||
*/
|
||||
public static <R> R generateEmptyPayload(String recKey, String partitionPath, String payloadClazz) {
|
||||
HoodieRecord<? extends HoodieRecordPayload> hoodieRecord = new HoodieRecord<>(new HoodieKey(recKey, partitionPath),
|
||||
HoodieRecord<? extends HoodieRecordPayload> hoodieRecord = new HoodieAvroRecord<>(new HoodieKey(recKey, partitionPath),
|
||||
ReflectionUtils.loadPayload(payloadClazz, new Object[] {Option.empty()}, Option.class));
|
||||
return (R) hoodieRecord;
|
||||
}
|
||||
|
||||
@@ -30,6 +30,7 @@ import org.apache.hudi.common.config.HoodieMetadataConfig;
|
||||
import org.apache.hudi.common.config.SerializableConfiguration;
|
||||
import org.apache.hudi.common.engine.HoodieEngineContext;
|
||||
import org.apache.hudi.common.model.FileSlice;
|
||||
import org.apache.hudi.common.model.HoodieAvroRecord;
|
||||
import org.apache.hudi.common.model.HoodieBaseFile;
|
||||
import org.apache.hudi.common.model.HoodieLogFile;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
@@ -215,7 +216,7 @@ public class HoodieBackedTableMetadata extends BaseTableMetadata {
|
||||
// merge base file record w/ log record if present
|
||||
if (logRecords.containsKey(key) && logRecords.get(key).isPresent()) {
|
||||
HoodieRecordPayload mergedPayload = logRecords.get(key).get().getData().preCombine(hoodieRecord.getData());
|
||||
result.add(Pair.of(key, Option.of(new HoodieRecord(hoodieRecord.getKey(), mergedPayload))));
|
||||
result.add(Pair.of(key, Option.of(new HoodieAvroRecord(hoodieRecord.getKey(), mergedPayload))));
|
||||
} else {
|
||||
// only base record
|
||||
result.add(Pair.of(key, Option.of(hoodieRecord)));
|
||||
|
||||
@@ -18,31 +18,31 @@
|
||||
|
||||
package org.apache.hudi.metadata;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
import org.apache.hudi.common.config.HoodieMetadataConfig;
|
||||
import org.apache.hudi.common.model.HoodieAvroRecord;
|
||||
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.table.HoodieTableConfig;
|
||||
import org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner;
|
||||
import org.apache.hudi.common.table.log.InstantRange;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.SpillableMapUtils;
|
||||
import org.apache.hudi.common.util.collection.ExternalSpillableMap;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
|
||||
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.hudi.common.config.HoodieMetadataConfig;
|
||||
import org.apache.hudi.common.table.HoodieTableConfig;
|
||||
import org.apache.hudi.common.util.SpillableMapUtils;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
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.table.log.HoodieMergedLogRecordScanner;
|
||||
import org.apache.hudi.common.table.log.InstantRange;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.collection.ExternalSpillableMap;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
* A {@code HoodieMergedLogRecordScanner} implementation which only merged records matching providing keys. This is
|
||||
@@ -87,7 +87,7 @@ public class HoodieMetadataMergedLogRecordReader extends HoodieMergedLogRecordSc
|
||||
}
|
||||
|
||||
@Override
|
||||
protected HoodieRecord<?> createHoodieRecord(final IndexedRecord rec, final HoodieTableConfig hoodieTableConfig,
|
||||
protected HoodieAvroRecord<?> createHoodieRecord(final IndexedRecord rec, final HoodieTableConfig hoodieTableConfig,
|
||||
final String payloadClassFQN, final String preCombineField,
|
||||
final boolean withOperationField,
|
||||
final Option<Pair<String, String>> simpleKeyGenFields,
|
||||
|
||||
@@ -18,12 +18,13 @@
|
||||
|
||||
package org.apache.hudi.metadata;
|
||||
|
||||
import org.apache.hudi.avro.model.HoodieMetadataColumnStats;
|
||||
import org.apache.hudi.avro.model.HoodieMetadataBloomFilter;
|
||||
import org.apache.hudi.avro.model.HoodieMetadataColumnStats;
|
||||
import org.apache.hudi.avro.model.HoodieMetadataFileInfo;
|
||||
import org.apache.hudi.avro.model.HoodieMetadataRecord;
|
||||
import org.apache.hudi.common.bloom.BloomFilterTypeCode;
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.common.model.HoodieAvroRecord;
|
||||
import org.apache.hudi.common.model.HoodieColumnRangeMetadata;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
@@ -32,9 +33,10 @@ import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.ValidationUtils;
|
||||
import org.apache.hudi.common.util.hash.ColumnIndexID;
|
||||
import org.apache.hudi.common.util.hash.FileIndexID;
|
||||
|
||||
import org.apache.hudi.common.util.hash.PartitionIndexID;
|
||||
import org.apache.hudi.exception.HoodieMetadataException;
|
||||
import org.apache.hudi.io.storage.HoodieHFileReader;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
@@ -42,7 +44,6 @@ 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.nio.ByteBuffer;
|
||||
@@ -204,7 +205,7 @@ public class HoodieMetadataPayload implements HoodieRecordPayload<HoodieMetadata
|
||||
HoodieKey key = new HoodieKey(RECORDKEY_PARTITION_LIST, MetadataPartitionType.FILES.getPartitionPath());
|
||||
HoodieMetadataPayload payload = new HoodieMetadataPayload(key.getRecordKey(), METADATA_TYPE_PARTITION_LIST,
|
||||
fileInfo);
|
||||
return new HoodieRecord<>(key, payload);
|
||||
return new HoodieAvroRecord<>(key, payload);
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -225,7 +226,7 @@ public class HoodieMetadataPayload implements HoodieRecordPayload<HoodieMetadata
|
||||
|
||||
HoodieKey key = new HoodieKey(partition, MetadataPartitionType.FILES.getPartitionPath());
|
||||
HoodieMetadataPayload payload = new HoodieMetadataPayload(key.getRecordKey(), METADATA_TYPE_FILE_LIST, fileInfo);
|
||||
return new HoodieRecord<>(key, payload);
|
||||
return new HoodieAvroRecord<>(key, payload);
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -256,7 +257,7 @@ public class HoodieMetadataPayload implements HoodieRecordPayload<HoodieMetadata
|
||||
timestamp, bloomFilter, isDeleted);
|
||||
HoodieMetadataPayload metadataPayload = new HoodieMetadataPayload(key.getRecordKey(),
|
||||
HoodieMetadataPayload.METADATA_TYPE_BLOOM_FILTER, metadataBloomFilter);
|
||||
return new HoodieRecord<>(key, metadataPayload);
|
||||
return new HoodieAvroRecord<>(key, metadataPayload);
|
||||
}
|
||||
|
||||
@Override
|
||||
@@ -447,7 +448,7 @@ public class HoodieMetadataPayload implements HoodieRecordPayload<HoodieMetadata
|
||||
.setTotalUncompressedSize(columnRangeMetadata.getTotalUncompressedSize())
|
||||
.setIsDeleted(isDeleted)
|
||||
.build());
|
||||
return new HoodieRecord<>(key, payload);
|
||||
return new HoodieAvroRecord<>(key, payload);
|
||||
});
|
||||
|
||||
|
||||
|
||||
Reference in New Issue
Block a user