[HUDI-1296] Support Metadata Table in Spark Datasource (#4789)
* Bootstrapping initial support for Metadata Table in Spark Datasource - Consolidated Avro/Row conversion utilities to center around Spark's AvroDeserializer ; removed duplication - Bootstrapped HoodieBaseRelation - Updated HoodieMergeOnReadRDD to be able to handle Metadata Table - Modified MOR relations to be able to read different Base File formats (Parquet, HFile)
This commit is contained in:
@@ -40,7 +40,6 @@ import org.apache.avro.io.EncoderFactory;
|
||||
import org.apache.avro.io.JsonDecoder;
|
||||
import org.apache.avro.io.JsonEncoder;
|
||||
import org.apache.avro.specific.SpecificRecordBase;
|
||||
|
||||
import org.apache.hudi.common.config.SerializableSchema;
|
||||
import org.apache.hudi.common.model.HoodieOperation;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
@@ -341,14 +340,24 @@ public class HoodieAvroUtils {
|
||||
/**
|
||||
* Given an Avro record with a given schema, rewrites it into the new schema while setting fields only from the new
|
||||
* schema.
|
||||
*
|
||||
* NOTE: This method is rewriting every record's field that is record itself recursively. It's
|
||||
* caller's responsibility to make sure that no unnecessary re-writing occurs (by preemptively
|
||||
* checking whether the record does require re-writing to adhere to the new schema)
|
||||
*
|
||||
* NOTE: Here, the assumption is that you cannot go from an evolved schema (schema with (N) fields)
|
||||
* to an older schema (schema with (N-1) fields). All fields present in the older record schema MUST be present in the
|
||||
* new schema and the default/existing values are carried over.
|
||||
* This particular method does the following things :
|
||||
* a) Create a new empty GenericRecord with the new schema.
|
||||
* b) For GenericRecord, copy over the data from the old schema to the new schema or set default values for all fields of this
|
||||
* transformed schema
|
||||
* c) For SpecificRecord, hoodie_metadata_fields have a special treatment. This is done because for code generated
|
||||
* to an older schema (schema with (N-1) fields). All fields present in the older record schema MUST be present in the
|
||||
* new schema and the default/existing values are carried over.
|
||||
*
|
||||
* This particular method does the following:
|
||||
* <ol>
|
||||
* <li>Create a new empty GenericRecord with the new schema.</li>
|
||||
* <li>For GenericRecord, copy over the data from the old schema to the new schema or set default values for all
|
||||
* fields of this transformed schema</li>
|
||||
* <li>For SpecificRecord, hoodie_metadata_fields have a special treatment (see below)</li>
|
||||
* </ol>
|
||||
*
|
||||
* For SpecificRecord we ignore Hudi Metadata fields, because for code generated
|
||||
* avro classes (HoodieMetadataRecord), the avro record is a SpecificBaseRecord type instead of a GenericRecord.
|
||||
* SpecificBaseRecord throws null pointer exception for record.get(name) if name is not present in the schema of the
|
||||
* record (which happens when converting a SpecificBaseRecord without hoodie_metadata_fields to a new record with it).
|
||||
@@ -360,16 +369,16 @@ public class HoodieAvroUtils {
|
||||
GenericRecord newRecord = new GenericData.Record(newSchema);
|
||||
boolean isSpecificRecord = oldRecord instanceof SpecificRecordBase;
|
||||
for (Schema.Field f : newSchema.getFields()) {
|
||||
if (!isSpecificRecord) {
|
||||
copyOldValueOrSetDefault(oldRecord, newRecord, f);
|
||||
} else if (!isMetadataField(f.name())) {
|
||||
if (!(isSpecificRecord && isMetadataField(f.name()))) {
|
||||
copyOldValueOrSetDefault(oldRecord, newRecord, f);
|
||||
}
|
||||
}
|
||||
|
||||
if (!GenericData.get().validate(newSchema, newRecord)) {
|
||||
throw new SchemaCompatibilityException(
|
||||
"Unable to validate the rewritten record " + oldRecord + " against schema " + newSchema);
|
||||
}
|
||||
|
||||
return newRecord;
|
||||
}
|
||||
|
||||
@@ -383,18 +392,20 @@ public class HoodieAvroUtils {
|
||||
return records.stream().map(r -> rewriteRecord(r, newSchema)).collect(Collectors.toList());
|
||||
}
|
||||
|
||||
private static void copyOldValueOrSetDefault(GenericRecord oldRecord, GenericRecord newRecord, Schema.Field f) {
|
||||
// cache the result of oldRecord.get() to save CPU expensive hash lookup
|
||||
private static void copyOldValueOrSetDefault(GenericRecord oldRecord, GenericRecord newRecord, Schema.Field field) {
|
||||
Schema oldSchema = oldRecord.getSchema();
|
||||
Object fieldValue = oldSchema.getField(f.name()) == null ? null : oldRecord.get(f.name());
|
||||
if (fieldValue == null) {
|
||||
if (f.defaultVal() instanceof JsonProperties.Null) {
|
||||
newRecord.put(f.name(), null);
|
||||
} else {
|
||||
newRecord.put(f.name(), f.defaultVal());
|
||||
}
|
||||
Object fieldValue = oldSchema.getField(field.name()) == null ? null : oldRecord.get(field.name());
|
||||
|
||||
if (fieldValue != null) {
|
||||
// In case field's value is a nested record, we have to rewrite it as well
|
||||
Object newFieldValue = fieldValue instanceof GenericRecord
|
||||
? rewriteRecord((GenericRecord) fieldValue, resolveNullableSchema(field.schema()))
|
||||
: fieldValue;
|
||||
newRecord.put(field.name(), newFieldValue);
|
||||
} else if (field.defaultVal() instanceof JsonProperties.Null) {
|
||||
newRecord.put(field.name(), null);
|
||||
} else {
|
||||
newRecord.put(f.name(), fieldValue);
|
||||
newRecord.put(field.name(), field.defaultVal());
|
||||
}
|
||||
}
|
||||
|
||||
@@ -610,4 +621,24 @@ public class HoodieAvroUtils {
|
||||
SerializableSchema schema, boolean consistentLogicalTimestampEnabled) {
|
||||
return getRecordColumnValues(record, columns, schema.get(), consistentLogicalTimestampEnabled);
|
||||
}
|
||||
|
||||
private static Schema resolveNullableSchema(Schema schema) {
|
||||
if (schema.getType() != Schema.Type.UNION) {
|
||||
return schema;
|
||||
}
|
||||
|
||||
List<Schema> innerTypes = schema.getTypes();
|
||||
Schema nonNullType =
|
||||
innerTypes.stream()
|
||||
.filter(it -> it.getType() != Schema.Type.NULL)
|
||||
.findFirst()
|
||||
.orElse(null);
|
||||
|
||||
if (innerTypes.size() != 2 || nonNullType == null) {
|
||||
throw new AvroRuntimeException(
|
||||
String.format("Unsupported Avro UNION type %s: Only UNION of a null type and a non-null type is supported", schema));
|
||||
}
|
||||
|
||||
return nonNullType;
|
||||
}
|
||||
}
|
||||
|
||||
@@ -18,15 +18,14 @@
|
||||
|
||||
package org.apache.hudi.avro;
|
||||
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.Schema.Type;
|
||||
import org.apache.avro.generic.GenericData;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
@@ -293,7 +292,7 @@ public class MercifulJsonConverter {
|
||||
for (Object v : (List) value) {
|
||||
listRes.add(convertJsonToAvroField(v, name, elementSchema));
|
||||
}
|
||||
return Pair.of(true, listRes);
|
||||
return Pair.of(true, new GenericData.Array<>(schema, listRes));
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@@ -143,7 +143,7 @@ public class TableSchemaResolver {
|
||||
* @throws Exception
|
||||
*/
|
||||
public Schema getTableAvroSchema() throws Exception {
|
||||
return getTableAvroSchema(true);
|
||||
return getTableAvroSchema(metaClient.getTableConfig().populateMetaFields());
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -526,6 +526,7 @@ public class TableSchemaResolver {
|
||||
Schema tableAvroSchema = getTableAvroSchemaFromDataFile();
|
||||
return tableAvroSchema.getField(HoodieRecord.OPERATION_METADATA_FIELD) != null;
|
||||
} catch (Exception e) {
|
||||
LOG.info(String.format("Failed to read operation field from avro schema (%s)", e.getMessage()));
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
@@ -19,8 +19,10 @@
|
||||
|
||||
package org.apache.hudi.metadata;
|
||||
|
||||
import org.apache.hudi.avro.model.HoodieMetadataColumnStats;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.avro.model.HoodieMetadataBloomFilter;
|
||||
import org.apache.hudi.avro.model.HoodieMetadataColumnStats;
|
||||
import org.apache.hudi.common.config.HoodieMetadataConfig;
|
||||
import org.apache.hudi.common.config.SerializableConfiguration;
|
||||
import org.apache.hudi.common.engine.HoodieEngineContext;
|
||||
@@ -38,9 +40,6 @@ 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.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
@@ -60,8 +59,8 @@ public abstract class BaseTableMetadata implements HoodieTableMetadata {
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(BaseTableMetadata.class);
|
||||
|
||||
static final long MAX_MEMORY_SIZE_IN_BYTES = 1024 * 1024 * 1024;
|
||||
static final int BUFFER_SIZE = 10 * 1024 * 1024;
|
||||
public static final long MAX_MEMORY_SIZE_IN_BYTES = 1024 * 1024 * 1024;
|
||||
public static final int BUFFER_SIZE = 10 * 1024 * 1024;
|
||||
|
||||
protected final transient HoodieEngineContext engineContext;
|
||||
protected final SerializableConfiguration hadoopConf;
|
||||
|
||||
@@ -290,7 +290,9 @@ public class HoodieBackedTableMetadata extends BaseTableMetadata {
|
||||
final long baseFileOpenMs = baseFileReaderOpenTimePair.getValue();
|
||||
|
||||
// Open the log record scanner using the log files from the latest file slice
|
||||
Pair<HoodieMetadataMergedLogRecordReader, Long> logRecordScannerOpenTimePair = getLogRecordScanner(slice, partitionName);
|
||||
List<HoodieLogFile> logFiles = slice.getLogFiles().collect(Collectors.toList());
|
||||
Pair<HoodieMetadataMergedLogRecordReader, Long> logRecordScannerOpenTimePair =
|
||||
getLogRecordScanner(logFiles, partitionName);
|
||||
HoodieMetadataMergedLogRecordReader logRecordScanner = logRecordScannerOpenTimePair.getKey();
|
||||
final long logScannerOpenMs = logRecordScannerOpenTimePair.getValue();
|
||||
|
||||
@@ -343,9 +345,9 @@ public class HoodieBackedTableMetadata extends BaseTableMetadata {
|
||||
return validInstantTimestamps;
|
||||
}
|
||||
|
||||
private Pair<HoodieMetadataMergedLogRecordReader, Long> getLogRecordScanner(FileSlice slice, String partitionName) {
|
||||
public Pair<HoodieMetadataMergedLogRecordReader, Long> getLogRecordScanner(List<HoodieLogFile> logFiles, String partitionName) {
|
||||
HoodieTimer timer = new HoodieTimer().startTimer();
|
||||
List<String> logFilePaths = slice.getLogFiles()
|
||||
List<String> sortedLogFilePaths = logFiles.stream()
|
||||
.sorted(HoodieLogFile.getLogFileComparator())
|
||||
.map(o -> o.getPath().toString())
|
||||
.collect(Collectors.toList());
|
||||
@@ -363,7 +365,7 @@ public class HoodieBackedTableMetadata extends BaseTableMetadata {
|
||||
HoodieMetadataMergedLogRecordReader logRecordScanner = HoodieMetadataMergedLogRecordReader.newBuilder()
|
||||
.withFileSystem(metadataMetaClient.getFs())
|
||||
.withBasePath(metadataBasePath)
|
||||
.withLogFilePaths(logFilePaths)
|
||||
.withLogFilePaths(sortedLogFilePaths)
|
||||
.withReaderSchema(schema)
|
||||
.withLatestInstantTime(latestMetadataInstantTime)
|
||||
.withMaxMemorySizeInBytes(MAX_MEMORY_SIZE_IN_BYTES)
|
||||
@@ -378,7 +380,7 @@ public class HoodieBackedTableMetadata extends BaseTableMetadata {
|
||||
|
||||
Long logScannerOpenMs = timer.endTimer();
|
||||
LOG.info(String.format("Opened %d metadata log files (dataset instant=%s, metadata instant=%s) in %d ms",
|
||||
logFilePaths.size(), getLatestDataInstantTime(), latestMetadataInstantTime, logScannerOpenMs));
|
||||
sortedLogFilePaths.size(), getLatestDataInstantTime(), latestMetadataInstantTime, logScannerOpenMs));
|
||||
return Pair.of(logRecordScanner, logScannerOpenMs);
|
||||
}
|
||||
|
||||
|
||||
@@ -18,6 +18,13 @@
|
||||
|
||||
package org.apache.hudi.metadata;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
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.avro.model.HoodieMetadataBloomFilter;
|
||||
import org.apache.hudi.avro.model.HoodieMetadataColumnStats;
|
||||
import org.apache.hudi.avro.model.HoodieMetadataFileInfo;
|
||||
@@ -37,14 +44,6 @@ 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;
|
||||
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 java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.Arrays;
|
||||
@@ -56,6 +55,9 @@ import java.util.Properties;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
import static org.apache.hudi.TypeUtils.unsafeCast;
|
||||
import static org.apache.hudi.common.util.ValidationUtils.checkArgument;
|
||||
import static org.apache.hudi.common.util.ValidationUtils.checkState;
|
||||
import static org.apache.hudi.metadata.HoodieTableMetadata.RECORDKEY_PARTITION_LIST;
|
||||
|
||||
/**
|
||||
@@ -95,8 +97,8 @@ public class HoodieMetadataPayload implements HoodieRecordPayload<HoodieMetadata
|
||||
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";
|
||||
private static final String SCHEMA_FIELD_ID_COLUMN_STATS = "ColumnStatsMetadata";
|
||||
private static final String SCHEMA_FIELD_ID_BLOOM_FILTER = "BloomFilterMetadata";
|
||||
public static final String SCHEMA_FIELD_ID_COLUMN_STATS = "ColumnStatsMetadata";
|
||||
public static final String SCHEMA_FIELD_ID_BLOOM_FILTER = "BloomFilterMetadata";
|
||||
|
||||
// HoodieMetadata bloom filter payload field ids
|
||||
private static final String FIELD_IS_DELETED = "isDeleted";
|
||||
@@ -125,48 +127,65 @@ public class HoodieMetadataPayload implements HoodieRecordPayload<HoodieMetadata
|
||||
this(Option.of(record));
|
||||
}
|
||||
|
||||
public HoodieMetadataPayload(Option<GenericRecord> record) {
|
||||
if (record.isPresent()) {
|
||||
public HoodieMetadataPayload(Option<GenericRecord> recordOpt) {
|
||||
if (recordOpt.isPresent()) {
|
||||
GenericRecord record = recordOpt.get();
|
||||
// This can be simplified using SpecificData.deepcopy once this bug is fixed
|
||||
// https://issues.apache.org/jira/browse/AVRO-1811
|
||||
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");
|
||||
//
|
||||
// NOTE: {@code HoodieMetadataRecord} has to always carry both "key" nad "type" fields
|
||||
// for it to be handled appropriately, therefore these fields have to be reflected
|
||||
// in any (read-)projected schema
|
||||
key = record.get(KEY_FIELD_NAME).toString();
|
||||
type = (int) record.get(SCHEMA_FIELD_NAME_TYPE);
|
||||
|
||||
Map<String, HoodieMetadataFileInfo> metadata = getNestedFieldValue(record, SCHEMA_FIELD_NAME_METADATA);
|
||||
if (metadata != null) {
|
||||
filesystemMetadata = metadata;
|
||||
filesystemMetadata.keySet().forEach(k -> {
|
||||
GenericRecord v = filesystemMetadata.get(k);
|
||||
filesystemMetadata.put(k.toString(), new HoodieMetadataFileInfo((Long) v.get("size"), (Boolean) v.get("isDeleted")));
|
||||
filesystemMetadata.put(k, new HoodieMetadataFileInfo((Long) v.get("size"), (Boolean) v.get("isDeleted")));
|
||||
});
|
||||
}
|
||||
|
||||
if (type == METADATA_TYPE_BLOOM_FILTER) {
|
||||
final GenericRecord metadataRecord = (GenericRecord) record.get().get(SCHEMA_FIELD_ID_BLOOM_FILTER);
|
||||
if (metadataRecord == null) {
|
||||
throw new HoodieMetadataException("Valid " + SCHEMA_FIELD_ID_BLOOM_FILTER + " record expected for type: " + METADATA_TYPE_BLOOM_FILTER);
|
||||
GenericRecord bloomFilterRecord = getNestedFieldValue(record, SCHEMA_FIELD_ID_BLOOM_FILTER);
|
||||
// NOTE: Only legitimate reason for {@code BloomFilterMetadata} to not be present is when
|
||||
// it's not been read from the storage (ie it's not been a part of projected schema).
|
||||
// Otherwise, it has to be present or the record would be considered invalid
|
||||
if (bloomFilterRecord == null) {
|
||||
checkArgument(record.getSchema().getField(SCHEMA_FIELD_ID_BLOOM_FILTER) == null,
|
||||
String.format("Valid %s record expected for type: %s", SCHEMA_FIELD_ID_BLOOM_FILTER, METADATA_TYPE_COLUMN_STATS));
|
||||
} else {
|
||||
bloomFilterMetadata = new HoodieMetadataBloomFilter(
|
||||
(String) bloomFilterRecord.get(BLOOM_FILTER_FIELD_TYPE),
|
||||
(String) bloomFilterRecord.get(BLOOM_FILTER_FIELD_TIMESTAMP),
|
||||
(ByteBuffer) bloomFilterRecord.get(BLOOM_FILTER_FIELD_BLOOM_FILTER),
|
||||
(Boolean) bloomFilterRecord.get(BLOOM_FILTER_FIELD_IS_DELETED)
|
||||
);
|
||||
}
|
||||
bloomFilterMetadata = new HoodieMetadataBloomFilter(
|
||||
(String) metadataRecord.get(BLOOM_FILTER_FIELD_TYPE),
|
||||
(String) metadataRecord.get(BLOOM_FILTER_FIELD_TIMESTAMP),
|
||||
(ByteBuffer) metadataRecord.get(BLOOM_FILTER_FIELD_BLOOM_FILTER),
|
||||
(Boolean) metadataRecord.get(BLOOM_FILTER_FIELD_IS_DELETED)
|
||||
);
|
||||
}
|
||||
|
||||
if (type == METADATA_TYPE_COLUMN_STATS) {
|
||||
GenericRecord v = (GenericRecord) record.get().get(SCHEMA_FIELD_ID_COLUMN_STATS);
|
||||
if (v == null) {
|
||||
throw new HoodieMetadataException("Valid " + SCHEMA_FIELD_ID_COLUMN_STATS + " record expected for type: " + METADATA_TYPE_COLUMN_STATS);
|
||||
GenericRecord columnStatsRecord = getNestedFieldValue(record, SCHEMA_FIELD_ID_COLUMN_STATS);
|
||||
// NOTE: Only legitimate reason for {@code ColumnStatsMetadata} to not be present is when
|
||||
// it's not been read from the storage (ie it's not been a part of projected schema).
|
||||
// Otherwise, it has to be present or the record would be considered invalid
|
||||
if (columnStatsRecord == null) {
|
||||
checkArgument(record.getSchema().getField(SCHEMA_FIELD_ID_COLUMN_STATS) == null,
|
||||
String.format("Valid %s record expected for type: %s", SCHEMA_FIELD_ID_COLUMN_STATS, METADATA_TYPE_COLUMN_STATS));
|
||||
} else {
|
||||
columnStatMetadata = HoodieMetadataColumnStats.newBuilder()
|
||||
.setFileName((String) columnStatsRecord.get(COLUMN_STATS_FIELD_RESOURCE_NAME))
|
||||
.setMinValue((String) columnStatsRecord.get(COLUMN_STATS_FIELD_MIN_VALUE))
|
||||
.setMaxValue((String) columnStatsRecord.get(COLUMN_STATS_FIELD_MAX_VALUE))
|
||||
.setValueCount((Long) columnStatsRecord.get(COLUMN_STATS_FIELD_VALUE_COUNT))
|
||||
.setNullCount((Long) columnStatsRecord.get(COLUMN_STATS_FIELD_NULL_COUNT))
|
||||
.setTotalSize((Long) columnStatsRecord.get(COLUMN_STATS_FIELD_TOTAL_SIZE))
|
||||
.setTotalUncompressedSize((Long) columnStatsRecord.get(COLUMN_STATS_FIELD_TOTAL_UNCOMPRESSED_SIZE))
|
||||
.setIsDeleted((Boolean) columnStatsRecord.get(COLUMN_STATS_FIELD_IS_DELETED))
|
||||
.build();
|
||||
}
|
||||
columnStatMetadata = HoodieMetadataColumnStats.newBuilder()
|
||||
.setFileName((String) v.get(COLUMN_STATS_FIELD_RESOURCE_NAME))
|
||||
.setMinValue((String) v.get(COLUMN_STATS_FIELD_MIN_VALUE))
|
||||
.setMaxValue((String) v.get(COLUMN_STATS_FIELD_MAX_VALUE))
|
||||
.setValueCount((Long) v.get(COLUMN_STATS_FIELD_VALUE_COUNT))
|
||||
.setNullCount((Long) v.get(COLUMN_STATS_FIELD_NULL_COUNT))
|
||||
.setTotalSize((Long) v.get(COLUMN_STATS_FIELD_TOTAL_SIZE))
|
||||
.setTotalUncompressedSize((Long) v.get(COLUMN_STATS_FIELD_TOTAL_UNCOMPRESSED_SIZE))
|
||||
.setIsDeleted((Boolean) v.get(COLUMN_STATS_FIELD_IS_DELETED))
|
||||
.build();
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -175,12 +194,12 @@ public class HoodieMetadataPayload implements HoodieRecordPayload<HoodieMetadata
|
||||
this(key, type, filesystemMetadata, null, null);
|
||||
}
|
||||
|
||||
private HoodieMetadataPayload(String key, int type, HoodieMetadataBloomFilter metadataBloomFilter) {
|
||||
this(key, type, null, metadataBloomFilter, null);
|
||||
private HoodieMetadataPayload(String key, HoodieMetadataBloomFilter metadataBloomFilter) {
|
||||
this(key, METADATA_TYPE_BLOOM_FILTER, null, metadataBloomFilter, null);
|
||||
}
|
||||
|
||||
private HoodieMetadataPayload(String key, int type, HoodieMetadataColumnStats columnStats) {
|
||||
this(key, type, null, null, columnStats);
|
||||
private HoodieMetadataPayload(String key, HoodieMetadataColumnStats columnStats) {
|
||||
this(key, METADATA_TYPE_COLUMN_STATS, null, null, columnStats);
|
||||
}
|
||||
|
||||
protected HoodieMetadataPayload(String key, int type,
|
||||
@@ -257,7 +276,7 @@ public class HoodieMetadataPayload implements HoodieRecordPayload<HoodieMetadata
|
||||
new HoodieMetadataBloomFilter(BloomFilterTypeCode.DYNAMIC_V0.name(),
|
||||
timestamp, bloomFilter, isDeleted);
|
||||
HoodieMetadataPayload metadataPayload = new HoodieMetadataPayload(key.getRecordKey(),
|
||||
HoodieMetadataPayload.METADATA_TYPE_BLOOM_FILTER, metadataBloomFilter);
|
||||
metadataBloomFilter);
|
||||
return new HoodieAvroRecord<>(key, metadataPayload);
|
||||
}
|
||||
|
||||
@@ -273,9 +292,9 @@ public class HoodieMetadataPayload implements HoodieRecordPayload<HoodieMetadata
|
||||
return new HoodieMetadataPayload(key, type, combinedFileInfo);
|
||||
case METADATA_TYPE_BLOOM_FILTER:
|
||||
HoodieMetadataBloomFilter combineBloomFilterMetadata = combineBloomFilterMetadata(previousRecord);
|
||||
return new HoodieMetadataPayload(key, type, combineBloomFilterMetadata);
|
||||
return new HoodieMetadataPayload(key, combineBloomFilterMetadata);
|
||||
case METADATA_TYPE_COLUMN_STATS:
|
||||
return new HoodieMetadataPayload(key, type, combineColumnStatsMetadata(previousRecord));
|
||||
return new HoodieMetadataPayload(key, combineColumnStatsMetadata(previousRecord));
|
||||
default:
|
||||
throw new HoodieMetadataException("Unknown type of HoodieMetadataPayload: " + type);
|
||||
}
|
||||
@@ -446,7 +465,7 @@ public class HoodieMetadataPayload implements HoodieRecordPayload<HoodieMetadata
|
||||
return columnRangeMetadataList.stream().map(columnRangeMetadata -> {
|
||||
HoodieKey key = new HoodieKey(getColumnStatsIndexKey(partitionName, columnRangeMetadata),
|
||||
MetadataPartitionType.COLUMN_STATS.getPartitionPath());
|
||||
HoodieMetadataPayload payload = new HoodieMetadataPayload(key.getRecordKey(), METADATA_TYPE_COLUMN_STATS,
|
||||
HoodieMetadataPayload payload = new HoodieMetadataPayload(key.getRecordKey(),
|
||||
HoodieMetadataColumnStats.newBuilder()
|
||||
.setFileName(new Path(columnRangeMetadata.getFilePath()).getName())
|
||||
.setMinValue(columnRangeMetadata.getMinValue() == null ? null :
|
||||
@@ -473,7 +492,7 @@ public class HoodieMetadataPayload implements HoodieRecordPayload<HoodieMetadata
|
||||
sb.append("creations=").append(Arrays.toString(getFilenames().toArray())).append(", ");
|
||||
sb.append("deletions=").append(Arrays.toString(getDeletions().toArray())).append(", ");
|
||||
if (type == METADATA_TYPE_BLOOM_FILTER) {
|
||||
ValidationUtils.checkState(getBloomFilterMetadata().isPresent());
|
||||
checkState(getBloomFilterMetadata().isPresent());
|
||||
sb.append("BloomFilter: {");
|
||||
sb.append("bloom size: " + getBloomFilterMetadata().get().getBloomFilter().array().length).append(", ");
|
||||
sb.append("timestamp: " + getBloomFilterMetadata().get().getTimestamp()).append(", ");
|
||||
@@ -481,7 +500,7 @@ public class HoodieMetadataPayload implements HoodieRecordPayload<HoodieMetadata
|
||||
sb.append("}");
|
||||
}
|
||||
if (type == METADATA_TYPE_COLUMN_STATS) {
|
||||
ValidationUtils.checkState(getColumnStatMetadata().isPresent());
|
||||
checkState(getColumnStatMetadata().isPresent());
|
||||
sb.append("ColStats: {");
|
||||
sb.append(getColumnStatMetadata().get());
|
||||
sb.append("}");
|
||||
@@ -489,4 +508,13 @@ public class HoodieMetadataPayload implements HoodieRecordPayload<HoodieMetadata
|
||||
sb.append('}');
|
||||
return sb.toString();
|
||||
}
|
||||
|
||||
private static <T> T getNestedFieldValue(GenericRecord record, String fieldName) {
|
||||
// NOTE: This routine is more lightweight than {@code HoodieAvroUtils.getNestedFieldVal}
|
||||
if (record.getSchema().getField(fieldName) == null) {
|
||||
return null;
|
||||
}
|
||||
|
||||
return unsafeCast(record.get(fieldName));
|
||||
}
|
||||
}
|
||||
|
||||
@@ -18,14 +18,14 @@
|
||||
|
||||
package org.apache.hudi.metadata;
|
||||
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.avro.model.HoodieMetadataColumnStats;
|
||||
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.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.exception.HoodieMetadataException;
|
||||
|
||||
@@ -35,6 +35,8 @@ import java.nio.ByteBuffer;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import static org.apache.hudi.common.util.ValidationUtils.checkArgument;
|
||||
|
||||
/**
|
||||
* Interface that supports querying various pieces of metadata about a hudi table.
|
||||
*/
|
||||
@@ -58,12 +60,19 @@ public interface HoodieTableMetadata extends Serializable, AutoCloseable {
|
||||
static final String METADATA_TABLE_REL_PATH = HoodieTableMetaClient.METAFOLDER_NAME + Path.SEPARATOR + "metadata";
|
||||
|
||||
/**
|
||||
* Return the base path of the Metadata Table.
|
||||
*
|
||||
* @param tableBasePath The base path of the dataset
|
||||
* Return the base-path of the Metadata Table for the given Dataset identified by base-path
|
||||
*/
|
||||
static String getMetadataTableBasePath(String tableBasePath) {
|
||||
return tableBasePath + Path.SEPARATOR + METADATA_TABLE_REL_PATH;
|
||||
static String getMetadataTableBasePath(String dataTableBasePath) {
|
||||
return dataTableBasePath + Path.SEPARATOR + METADATA_TABLE_REL_PATH;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the base path of the Dataset provided the base-path of the Metadata Table of this
|
||||
* Dataset
|
||||
*/
|
||||
static String getDataTableBasePathFromMetadataTable(String metadataTableBasePath) {
|
||||
checkArgument(isMetadataTable(metadataTableBasePath));
|
||||
return metadataTableBasePath.substring(0, metadataTableBasePath.lastIndexOf(METADATA_TABLE_REL_PATH) - 1);
|
||||
}
|
||||
|
||||
/**
|
||||
|
||||
Reference in New Issue
Block a user