/*
* 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.metadata;
import org.apache.hudi.avro.model.BooleanWrapper;
import org.apache.hudi.avro.model.BytesWrapper;
import org.apache.hudi.avro.model.DateWrapper;
import org.apache.hudi.avro.model.DecimalWrapper;
import org.apache.hudi.avro.model.DoubleWrapper;
import org.apache.hudi.avro.model.FloatWrapper;
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.avro.model.IntWrapper;
import org.apache.hudi.avro.model.LongWrapper;
import org.apache.hudi.avro.model.StringWrapper;
import org.apache.hudi.avro.model.TimestampMicrosWrapper;
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;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.common.util.Option;
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.hadoop.CachingPath;
import org.apache.hudi.io.storage.HoodieAvroHFileReader;
import org.apache.hudi.util.Lazy;
import org.apache.avro.Conversions;
import org.apache.avro.LogicalTypes;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord;
import org.apache.avro.util.Utf8;
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.math.BigDecimal;
import java.nio.ByteBuffer;
import java.sql.Date;
import java.sql.Timestamp;
import java.time.Instant;
import java.time.LocalDate;
import java.util.Arrays;
import java.util.Collection;
import java.util.Comparator;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Properties;
import java.util.function.Function;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import static org.apache.hudi.common.util.DateTimeUtils.instantToMicros;
import static org.apache.hudi.common.util.DateTimeUtils.microsToInstant;
import static org.apache.hudi.common.util.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.hadoop.CachingPath.createRelativePathUnsafe;
import static org.apache.hudi.metadata.HoodieTableMetadata.RECORDKEY_PARTITION_LIST;
import static org.apache.hudi.metadata.HoodieTableMetadataUtil.getPartitionIdentifier;
import static org.apache.hudi.metadata.HoodieTableMetadataUtil.tryUpcastDecimal;
/**
* MetadataTable records are persisted with the schema defined in HoodieMetadata.avsc.
* This class represents the payload for the MetadataTable.
*
* This single metadata payload is shared by all the partitions under the metadata table.
* The partition specific records are determined by the field "type" saved within the record.
* The following types are supported:
*
* METADATA_TYPE_PARTITION_LIST (1):
* -- List of all partitions. There is a single such record
* -- key = @{@link HoodieTableMetadata#RECORDKEY_PARTITION_LIST}
*
* METADATA_TYPE_FILE_LIST (2):
* -- List of all files in a partition. There is one such record for each partition
* -- key = partition name
*
* METADATA_TYPE_COLUMN_STATS (3):
* -- This is an index for column stats in the table
*
* METADATA_TYPE_BLOOM_FILTER (4):
* -- This is an index for base file bloom filters. This is a map of FileID to its BloomFilter byte[].
*
* During compaction on the table, the deletions are merged with additions and hence records are pruned.
*/
public class HoodieMetadataPayload implements HoodieRecordPayload {
// Type of the record. This can be an enum in the schema but Avro1.8
// has a bug - https://issues.apache.org/jira/browse/AVRO-1810
protected static final int METADATA_TYPE_PARTITION_LIST = 1;
protected static final int METADATA_TYPE_FILE_LIST = 2;
protected static final int METADATA_TYPE_COLUMN_STATS = 3;
protected static final int METADATA_TYPE_BLOOM_FILTER = 4;
// HoodieMetadata schema field ids
public static final String KEY_FIELD_NAME = HoodieAvroHFileReader.KEY_FIELD_NAME;
public static final String SCHEMA_FIELD_NAME_TYPE = "type";
public static final String SCHEMA_FIELD_NAME_METADATA = "filesystemMetadata";
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";
private static final String BLOOM_FILTER_FIELD_TYPE = "type";
private static final String BLOOM_FILTER_FIELD_TIMESTAMP = "timestamp";
private static final String BLOOM_FILTER_FIELD_BLOOM_FILTER = "bloomFilter";
private static final String BLOOM_FILTER_FIELD_IS_DELETED = FIELD_IS_DELETED;
// HoodieMetadata column stats payload field ids
public static final String COLUMN_STATS_FIELD_MIN_VALUE = "minValue";
public static final String COLUMN_STATS_FIELD_MAX_VALUE = "maxValue";
public static final String COLUMN_STATS_FIELD_NULL_COUNT = "nullCount";
public static final String COLUMN_STATS_FIELD_VALUE_COUNT = "valueCount";
public static final String COLUMN_STATS_FIELD_TOTAL_SIZE = "totalSize";
public static final String COLUMN_STATS_FIELD_FILE_NAME = "fileName";
public static final String COLUMN_STATS_FIELD_COLUMN_NAME = "columnName";
public static final String COLUMN_STATS_FIELD_TOTAL_UNCOMPRESSED_SIZE = "totalUncompressedSize";
public static final String COLUMN_STATS_FIELD_IS_DELETED = FIELD_IS_DELETED;
private static final Conversions.DecimalConversion AVRO_DECIMAL_CONVERSION = new Conversions.DecimalConversion();
// NOTE: PLEASE READ CAREFULLY
//
// In Avro 1.10 generated builders rely on {@code SpecificData.getForSchema} invocation that in turn
// does use reflection to load the code-gen'd class corresponding to the Avro record model. This has
// serious adverse effects in terms of performance when gets executed on the hot-path (both, in terms
// of runtime and efficiency).
//
// To work this around instead of using default code-gen'd builder invoking {@code SpecificData.getForSchema},
// we instead rely on overloaded ctor accepting another instance of the builder: {@code Builder(Builder)},
// which bypasses such invocation. Following corresponding builder's stubs are statically initialized
// to be used exactly for that purpose.
//
// You can find more details in HUDI-3834
private static final Lazy METADATA_COLUMN_STATS_BUILDER_STUB = Lazy.lazily(HoodieMetadataColumnStats::newBuilder);
private static final Lazy STRING_WRAPPER_BUILDER_STUB = Lazy.lazily(StringWrapper::newBuilder);
private static final Lazy BYTES_WRAPPER_BUILDER_STUB = Lazy.lazily(BytesWrapper::newBuilder);
private static final Lazy DOUBLE_WRAPPER_BUILDER_STUB = Lazy.lazily(DoubleWrapper::newBuilder);
private static final Lazy FLOAT_WRAPPER_BUILDER_STUB = Lazy.lazily(FloatWrapper::newBuilder);
private static final Lazy LONG_WRAPPER_BUILDER_STUB = Lazy.lazily(LongWrapper::newBuilder);
private static final Lazy INT_WRAPPER_BUILDER_STUB = Lazy.lazily(IntWrapper::newBuilder);
private static final Lazy BOOLEAN_WRAPPER_BUILDER_STUB = Lazy.lazily(BooleanWrapper::newBuilder);
private static final Lazy TIMESTAMP_MICROS_WRAPPER_BUILDER_STUB = Lazy.lazily(TimestampMicrosWrapper::newBuilder);
private static final Lazy DECIMAL_WRAPPER_BUILDER_STUB = Lazy.lazily(DecimalWrapper::newBuilder);
private static final Lazy DATE_WRAPPER_BUILDER_STUB = Lazy.lazily(DateWrapper::newBuilder);
private String key = null;
private int type = 0;
private Map filesystemMetadata = null;
private HoodieMetadataBloomFilter bloomFilterMetadata = null;
private HoodieMetadataColumnStats columnStatMetadata = null;
public HoodieMetadataPayload(GenericRecord record, Comparable orderingVal) {
this(Option.of(record));
}
public HoodieMetadataPayload(Option 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
//
// NOTE: {@code HoodieMetadataRecord} has to always carry both "key" and "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 metadata = getNestedFieldValue(record, SCHEMA_FIELD_NAME_METADATA);
if (metadata != null) {
filesystemMetadata = metadata;
filesystemMetadata.keySet().forEach(k -> {
GenericRecord v = filesystemMetadata.get(k);
filesystemMetadata.put(k, new HoodieMetadataFileInfo((Long) v.get("size"), (Boolean) v.get("isDeleted")));
});
}
if (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)
);
}
}
if (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(METADATA_COLUMN_STATS_BUILDER_STUB.get())
.setFileName((String) columnStatsRecord.get(COLUMN_STATS_FIELD_FILE_NAME))
.setColumnName((String) columnStatsRecord.get(COLUMN_STATS_FIELD_COLUMN_NAME))
// AVRO-2377 1.9.2 Modified the type of org.apache.avro.Schema#FIELD_RESERVED to Collections.unmodifiableSet.
// This causes Kryo to fail when deserializing a GenericRecord, See HUDI-5484.
// We should avoid using GenericRecord and convert GenericRecord into a serializable type.
.setMinValue(wrapStatisticValue(unwrapStatisticValueWrapper(columnStatsRecord.get(COLUMN_STATS_FIELD_MIN_VALUE))))
.setMaxValue(wrapStatisticValue(unwrapStatisticValueWrapper(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();
}
}
}
}
private HoodieMetadataPayload(String key, int type, Map filesystemMetadata) {
this(key, type, filesystemMetadata, null, null);
}
private HoodieMetadataPayload(String key, HoodieMetadataBloomFilter metadataBloomFilter) {
this(key, METADATA_TYPE_BLOOM_FILTER, null, metadataBloomFilter, null);
}
private HoodieMetadataPayload(String key, HoodieMetadataColumnStats columnStats) {
this(key, METADATA_TYPE_COLUMN_STATS, null, null, columnStats);
}
protected HoodieMetadataPayload(String key, int type,
Map filesystemMetadata,
HoodieMetadataBloomFilter metadataBloomFilter,
HoodieMetadataColumnStats columnStats) {
this.key = key;
this.type = type;
this.filesystemMetadata = filesystemMetadata;
this.bloomFilterMetadata = metadataBloomFilter;
this.columnStatMetadata = columnStats;
}
/**
* Create and return a {@code HoodieMetadataPayload} to save list of partitions.
*
* @param partitions The list of partitions
*/
public static HoodieRecord createPartitionListRecord(List partitions) {
return createPartitionListRecord(partitions, false);
}
/**
* Create and return a {@code HoodieMetadataPayload} to save list of partitions.
*
* @param partitions The list of partitions
*/
public static HoodieRecord createPartitionListRecord(List partitions, boolean isDeleted) {
Map fileInfo = new HashMap<>();
partitions.forEach(partition -> fileInfo.put(getPartitionIdentifier(partition), new HoodieMetadataFileInfo(0L, isDeleted)));
HoodieKey key = new HoodieKey(RECORDKEY_PARTITION_LIST, MetadataPartitionType.FILES.getPartitionPath());
HoodieMetadataPayload payload = new HoodieMetadataPayload(key.getRecordKey(), METADATA_TYPE_PARTITION_LIST,
fileInfo);
return new HoodieAvroRecord<>(key, payload);
}
/**
* Create and return a {@code HoodieMetadataPayload} to save list of partitions.
*
* @param partitionsAdded The list of added partitions
* @param partitionsDeleted The list of deleted partitions
*/
public static HoodieRecord createPartitionListRecord(List partitionsAdded, List partitionsDeleted) {
Map fileInfo = new HashMap<>();
partitionsAdded.forEach(partition -> fileInfo.put(partition, new HoodieMetadataFileInfo(0L, false)));
partitionsDeleted.forEach(partition -> fileInfo.put(partition, new HoodieMetadataFileInfo(0L, true)));
HoodieKey key = new HoodieKey(RECORDKEY_PARTITION_LIST, MetadataPartitionType.FILES.getPartitionPath());
HoodieMetadataPayload payload = new HoodieMetadataPayload(key.getRecordKey(), METADATA_TYPE_PARTITION_LIST,
fileInfo);
return new HoodieAvroRecord<>(key, payload);
}
/**
* Create and return a {@code HoodieMetadataPayload} to save list of files within a partition.
*
* @param partition The name of the partition
* @param filesAdded Mapping of files to their sizes for files which have been added to this partition
* @param filesDeleted List of files which have been deleted from this partition
*/
public static HoodieRecord createPartitionFilesRecord(String partition,
Option