[HUDI-3664] Fixing Column Stats Index composition (#5181)
Co-authored-by: Sagar Sumit <sagarsumit09@gmail.com>
This commit is contained in:
@@ -122,17 +122,195 @@
|
||||
"doc": "Minimum value in the range. Based on user data table schema, we can convert this to appropriate type",
|
||||
"name": "minValue",
|
||||
"type": [
|
||||
// Those types should be aligned with Parquet `Statistics` impl
|
||||
// making sure that we implement semantic consistent across file formats
|
||||
//
|
||||
// NOTE: Other logical types (decimal, date, timestamp, etc) will be converted
|
||||
// into one of the following types, making sure that their corresponding
|
||||
// ordering is preserved
|
||||
"null",
|
||||
"string"
|
||||
]
|
||||
{
|
||||
"namespace": "org.apache.hudi.avro.model",
|
||||
"type": "record",
|
||||
"name": "BooleanWrapper",
|
||||
"doc": "A record wrapping boolean type to be able to be used it w/in Avro's Union",
|
||||
"fields": [
|
||||
{
|
||||
"type": "boolean",
|
||||
"name": "value"
|
||||
}
|
||||
]
|
||||
},
|
||||
{
|
||||
"namespace": "org.apache.hudi.avro.model",
|
||||
"type": "record",
|
||||
"name": "IntWrapper",
|
||||
"doc": "A record wrapping int type to be able to be used it w/in Avro's Union",
|
||||
"fields": [
|
||||
{
|
||||
"type": "int",
|
||||
"name": "value"
|
||||
}
|
||||
]
|
||||
},
|
||||
{
|
||||
"namespace": "org.apache.hudi.avro.model",
|
||||
"type": "record",
|
||||
"name": "LongWrapper",
|
||||
"doc": "A record wrapping long type to be able to be used it w/in Avro's Union",
|
||||
"fields": [
|
||||
{
|
||||
"type": "long",
|
||||
"name": "value"
|
||||
}
|
||||
]
|
||||
},
|
||||
{
|
||||
"namespace": "org.apache.hudi.avro.model",
|
||||
"type": "record",
|
||||
"name": "FloatWrapper",
|
||||
"doc": "A record wrapping float type to be able to be used it w/in Avro's Union",
|
||||
"fields": [
|
||||
{
|
||||
"type": "float",
|
||||
"name": "value"
|
||||
}
|
||||
]
|
||||
},
|
||||
{
|
||||
"namespace": "org.apache.hudi.avro.model",
|
||||
"type": "record",
|
||||
"name": "DoubleWrapper",
|
||||
"doc": "A record wrapping double type to be able to be used it w/in Avro's Union",
|
||||
"fields": [
|
||||
{
|
||||
"type": "double",
|
||||
"name": "value"
|
||||
}
|
||||
]
|
||||
},
|
||||
{
|
||||
"namespace": "org.apache.hudi.avro.model",
|
||||
"type": "record",
|
||||
"name": "BytesWrapper",
|
||||
"doc": "A record wrapping bytes type to be able to be used it w/in Avro's Union",
|
||||
"fields": [
|
||||
{
|
||||
"type": "bytes",
|
||||
"name": "value"
|
||||
}
|
||||
]
|
||||
},
|
||||
{
|
||||
"namespace": "org.apache.hudi.avro.model",
|
||||
"type": "record",
|
||||
"name": "StringWrapper",
|
||||
"doc": "A record wrapping string type to be able to be used it w/in Avro's Union",
|
||||
"fields": [
|
||||
{
|
||||
"type": "string",
|
||||
"name": "value"
|
||||
}
|
||||
]
|
||||
},
|
||||
{
|
||||
"namespace": "org.apache.hudi.avro.model",
|
||||
"type": "record",
|
||||
"name": "DateWrapper",
|
||||
"doc": "A record wrapping Date logical type to be able to be used it w/in Avro's Union",
|
||||
"fields": [
|
||||
{
|
||||
"type": {
|
||||
"type": "int"
|
||||
// NOTE: Due to breaking changes in code-gen b/w Avro 1.8.2 and 1.10, we can't
|
||||
// rely on logical types to do proper encoding of the native Java types,
|
||||
// and hereby have to encode statistic manually
|
||||
//"logicalType": "date"
|
||||
},
|
||||
"name": "value"
|
||||
}
|
||||
]
|
||||
},
|
||||
{
|
||||
"namespace": "org.apache.hudi.avro.model",
|
||||
"type": "record",
|
||||
"name": "DecimalWrapper",
|
||||
"doc": "A record wrapping Decimal logical type to be able to be used it w/in Avro's Union",
|
||||
"fields": [
|
||||
{
|
||||
"type": {
|
||||
"type": "bytes",
|
||||
"logicalType": "decimal",
|
||||
// NOTE: This is equivalent to Spark's [[DoubleDecimal]] and should
|
||||
// be enough for almost any possible use-cases
|
||||
"precision": 30,
|
||||
"scale": 15
|
||||
},
|
||||
"name": "value"
|
||||
}
|
||||
]
|
||||
},
|
||||
{
|
||||
"namespace": "org.apache.hudi.avro.model",
|
||||
"type": "record",
|
||||
"name": "TimeMicrosWrapper",
|
||||
"doc": "A record wrapping Time-micros logical type to be able to be used it w/in Avro's Union",
|
||||
"fields": [
|
||||
{
|
||||
"type": {
|
||||
"type": "long",
|
||||
"logicalType": "time-micros"
|
||||
},
|
||||
"name": "value"
|
||||
|
||||
}
|
||||
]
|
||||
},
|
||||
{
|
||||
"namespace": "org.apache.hudi.avro.model",
|
||||
"type": "record",
|
||||
"name": "TimestampMicrosWrapper",
|
||||
"doc": "A record wrapping Timestamp-micros logical type to be able to be used it w/in Avro's Union",
|
||||
"fields": [
|
||||
{
|
||||
"type": {
|
||||
"type": "long"
|
||||
// NOTE: Due to breaking changes in code-gen b/w Avro 1.8.2 and 1.10, we can't
|
||||
// rely on logical types to do proper encoding of the native Java types,
|
||||
// and hereby have to encode statistic manually
|
||||
//"logicalType": "timestamp-micros"
|
||||
},
|
||||
"name": "value"
|
||||
}
|
||||
]
|
||||
}
|
||||
],
|
||||
"default": null
|
||||
},
|
||||
{
|
||||
"doc": "Maximum value in the range. Based on user data table schema, we can convert it to appropriate type",
|
||||
"name": "maxValue",
|
||||
"type": [
|
||||
// Those types should be aligned with Parquet `Statistics` impl
|
||||
// making sure that we implement semantic consistent across file formats
|
||||
//
|
||||
// NOTE: Other logical types (decimal, date, timestamp, etc) will be converted
|
||||
// into one of the following types, making sure that their corresponding
|
||||
// ordering is preserved
|
||||
"null",
|
||||
"string"
|
||||
]
|
||||
"org.apache.hudi.avro.model.BooleanWrapper",
|
||||
"org.apache.hudi.avro.model.IntWrapper",
|
||||
"org.apache.hudi.avro.model.LongWrapper",
|
||||
"org.apache.hudi.avro.model.FloatWrapper",
|
||||
"org.apache.hudi.avro.model.DoubleWrapper",
|
||||
"org.apache.hudi.avro.model.BytesWrapper",
|
||||
"org.apache.hudi.avro.model.StringWrapper",
|
||||
"org.apache.hudi.avro.model.DateWrapper",
|
||||
"org.apache.hudi.avro.model.DecimalWrapper",
|
||||
"org.apache.hudi.avro.model.TimeMicrosWrapper",
|
||||
"org.apache.hudi.avro.model.TimestampMicrosWrapper"
|
||||
],
|
||||
"default": null
|
||||
},
|
||||
{
|
||||
"doc": "Total count of values",
|
||||
|
||||
@@ -0,0 +1,144 @@
|
||||
/*
|
||||
* 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.avro;
|
||||
|
||||
import org.apache.avro.Conversions;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.UnresolvedUnionException;
|
||||
import org.apache.avro.data.TimeConversions;
|
||||
import org.apache.avro.generic.GenericData;
|
||||
import org.apache.avro.generic.GenericFixed;
|
||||
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* Custom instance of the {@link GenericData} model incorporating conversions from the
|
||||
* common Avro logical types like "decimal", "uuid", "date", "time-micros", "timestamp-micros"
|
||||
*
|
||||
* NOTE: Given that this code has to be interoperable w/ Spark 2 (which relies on Avro 1.8.2)
|
||||
* this model can't support newer conversion introduced in Avro 1.10 at the moment
|
||||
*/
|
||||
public class ConvertingGenericData extends GenericData {
|
||||
|
||||
private static final Conversions.DecimalConversion DECIMAL_CONVERSION = new Conversions.DecimalConversion();
|
||||
private static final Conversions.UUIDConversion UUID_CONVERSION = new Conversions.UUIDConversion();
|
||||
private static final TimeConversions.DateConversion DATE_CONVERSION = new TimeConversions.DateConversion();
|
||||
private static final TimeConversions.TimeMicrosConversion TIME_MICROS_CONVERSION = new TimeConversions.TimeMicrosConversion();
|
||||
private static final TimeConversions.TimestampMicrosConversion TIMESTAMP_MICROS_CONVERSION = new TimeConversions.TimestampMicrosConversion();
|
||||
|
||||
// NOTE: Those are not supported in Avro 1.8.2
|
||||
// TODO re-enable upon upgrading to 1.10
|
||||
// private static final TimeConversions.TimestampMillisConversion TIMESTAMP_MILLIS_CONVERSION = new TimeConversions.TimestampMillisConversion();
|
||||
// private static final TimeConversions.TimeMillisConversion TIME_MILLIS_CONVERSION = new TimeConversions.TimeMillisConversion();
|
||||
// private static final TimeConversions.LocalTimestampMillisConversion LOCAL_TIMESTAMP_MILLIS_CONVERSION = new TimeConversions.LocalTimestampMillisConversion();
|
||||
// private static final TimeConversions.LocalTimestampMicrosConversion LOCAL_TIMESTAMP_MICROS_CONVERSION = new TimeConversions.LocalTimestampMicrosConversion();
|
||||
|
||||
public static final GenericData INSTANCE = new ConvertingGenericData();
|
||||
|
||||
private ConvertingGenericData() {
|
||||
addLogicalTypeConversion(DECIMAL_CONVERSION);
|
||||
addLogicalTypeConversion(UUID_CONVERSION);
|
||||
addLogicalTypeConversion(DATE_CONVERSION);
|
||||
addLogicalTypeConversion(TIME_MICROS_CONVERSION);
|
||||
addLogicalTypeConversion(TIMESTAMP_MICROS_CONVERSION);
|
||||
// NOTE: Those are not supported in Avro 1.8.2
|
||||
// TODO re-enable upon upgrading to 1.10
|
||||
// addLogicalTypeConversion(TIME_MILLIS_CONVERSION);
|
||||
// addLogicalTypeConversion(TIMESTAMP_MILLIS_CONVERSION);
|
||||
// addLogicalTypeConversion(LOCAL_TIMESTAMP_MILLIS_CONVERSION);
|
||||
// addLogicalTypeConversion(LOCAL_TIMESTAMP_MICROS_CONVERSION);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean validate(Schema schema, Object datum) {
|
||||
switch (schema.getType()) {
|
||||
case RECORD:
|
||||
if (!isRecord(datum)) {
|
||||
return false;
|
||||
}
|
||||
for (Schema.Field f : schema.getFields()) {
|
||||
if (!validate(f.schema(), getField(datum, f.name(), f.pos()))) {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
return true;
|
||||
case ENUM:
|
||||
if (!isEnum(datum)) {
|
||||
return false;
|
||||
}
|
||||
return schema.getEnumSymbols().contains(datum.toString());
|
||||
case ARRAY:
|
||||
if (!(isArray(datum))) {
|
||||
return false;
|
||||
}
|
||||
for (Object element : getArrayAsCollection(datum)) {
|
||||
if (!validate(schema.getElementType(), element)) {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
return true;
|
||||
case MAP:
|
||||
if (!(isMap(datum))) {
|
||||
return false;
|
||||
}
|
||||
@SuppressWarnings(value = "unchecked")
|
||||
Map<Object, Object> map = (Map<Object, Object>) datum;
|
||||
for (Map.Entry<Object, Object> entry : map.entrySet()) {
|
||||
if (!validate(schema.getValueType(), entry.getValue())) {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
return true;
|
||||
case UNION:
|
||||
try {
|
||||
int i = resolveUnion(schema, datum);
|
||||
return validate(schema.getTypes().get(i), datum);
|
||||
} catch (UnresolvedUnionException e) {
|
||||
return false;
|
||||
}
|
||||
case FIXED:
|
||||
return (datum instanceof GenericFixed && ((GenericFixed) datum).bytes().length == schema.getFixedSize())
|
||||
|| DECIMAL_CONVERSION.getConvertedType().isInstance(datum);
|
||||
case STRING:
|
||||
return isString(datum)
|
||||
|| UUID_CONVERSION.getConvertedType().isInstance(datum);
|
||||
case BYTES:
|
||||
return isBytes(datum)
|
||||
|| DECIMAL_CONVERSION.getConvertedType().isInstance(datum);
|
||||
case INT:
|
||||
return isInteger(datum)
|
||||
|| DATE_CONVERSION.getConvertedType().isInstance(datum);
|
||||
case LONG:
|
||||
return isLong(datum)
|
||||
|| TIME_MICROS_CONVERSION.getConvertedType().isInstance(datum)
|
||||
|| TIMESTAMP_MICROS_CONVERSION.getConvertedType().isInstance(datum);
|
||||
case FLOAT:
|
||||
return isFloat(datum);
|
||||
case DOUBLE:
|
||||
return isDouble(datum);
|
||||
case BOOLEAN:
|
||||
return isBoolean(datum);
|
||||
case NULL:
|
||||
return datum == null;
|
||||
default:
|
||||
return false;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -69,6 +69,7 @@ import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
import java.util.HashMap;
|
||||
import java.util.TimeZone;
|
||||
import java.util.Iterator;
|
||||
@@ -82,9 +83,8 @@ import static org.apache.avro.Schema.Type.UNION;
|
||||
*/
|
||||
public class HoodieAvroUtils {
|
||||
|
||||
private static ThreadLocal<BinaryEncoder> reuseEncoder = ThreadLocal.withInitial(() -> null);
|
||||
|
||||
private static ThreadLocal<BinaryDecoder> reuseDecoder = ThreadLocal.withInitial(() -> null);
|
||||
private static final ThreadLocal<BinaryEncoder> BINARY_ENCODER = ThreadLocal.withInitial(() -> null);
|
||||
private static final ThreadLocal<BinaryDecoder> BINARY_DECODER = ThreadLocal.withInitial(() -> null);
|
||||
|
||||
private static final long MILLIS_PER_DAY = 86400000L;
|
||||
|
||||
@@ -92,9 +92,9 @@ public class HoodieAvroUtils {
|
||||
public static final Conversions.DecimalConversion DECIMAL_CONVERSION = new Conversions.DecimalConversion();
|
||||
|
||||
// As per https://avro.apache.org/docs/current/spec.html#names
|
||||
private static String INVALID_AVRO_CHARS_IN_NAMES = "[^A-Za-z0-9_]";
|
||||
private static String INVALID_AVRO_FIRST_CHAR_IN_NAMES = "[^A-Za-z_]";
|
||||
private static String MASK_FOR_INVALID_CHARS_IN_NAMES = "__";
|
||||
private static final String INVALID_AVRO_CHARS_IN_NAMES = "[^A-Za-z0-9_]";
|
||||
private static final String INVALID_AVRO_FIRST_CHAR_IN_NAMES = "[^A-Za-z_]";
|
||||
private static final String MASK_FOR_INVALID_CHARS_IN_NAMES = "__";
|
||||
|
||||
// All metadata fields are optional strings.
|
||||
public static final Schema METADATA_FIELD_SCHEMA =
|
||||
@@ -110,10 +110,10 @@ public class HoodieAvroUtils {
|
||||
}
|
||||
|
||||
public static <T extends IndexedRecord> byte[] indexedRecordToBytes(T record) {
|
||||
GenericDatumWriter<T> writer = new GenericDatumWriter<>(record.getSchema());
|
||||
GenericDatumWriter<T> writer = new GenericDatumWriter<>(record.getSchema(), ConvertingGenericData.INSTANCE);
|
||||
try (ByteArrayOutputStream out = new ByteArrayOutputStream()) {
|
||||
BinaryEncoder encoder = EncoderFactory.get().binaryEncoder(out, reuseEncoder.get());
|
||||
reuseEncoder.set(encoder);
|
||||
BinaryEncoder encoder = EncoderFactory.get().binaryEncoder(out, BINARY_ENCODER.get());
|
||||
BINARY_ENCODER.set(encoder);
|
||||
writer.write(record, encoder);
|
||||
encoder.flush();
|
||||
return out.toByteArray();
|
||||
@@ -148,8 +148,8 @@ public class HoodieAvroUtils {
|
||||
* Convert serialized bytes back into avro record.
|
||||
*/
|
||||
public static GenericRecord bytesToAvro(byte[] bytes, Schema writerSchema, Schema readerSchema) throws IOException {
|
||||
BinaryDecoder decoder = DecoderFactory.get().binaryDecoder(bytes, reuseDecoder.get());
|
||||
reuseDecoder.set(decoder);
|
||||
BinaryDecoder decoder = DecoderFactory.get().binaryDecoder(bytes, BINARY_DECODER.get());
|
||||
BINARY_DECODER.set(decoder);
|
||||
GenericDatumReader<GenericRecord> reader = new GenericDatumReader<>(writerSchema, readerSchema);
|
||||
return reader.read(null, decoder);
|
||||
}
|
||||
@@ -391,7 +391,7 @@ public class HoodieAvroUtils {
|
||||
}
|
||||
}
|
||||
|
||||
if (!GenericData.get().validate(newSchema, newRecord)) {
|
||||
if (!ConvertingGenericData.INSTANCE.validate(newSchema, newRecord)) {
|
||||
throw new SchemaCompatibilityException(
|
||||
"Unable to validate the rewritten record " + oldRecord + " against schema " + newSchema);
|
||||
}
|
||||
@@ -429,9 +429,13 @@ public class HoodieAvroUtils {
|
||||
|
||||
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;
|
||||
Object newFieldValue;
|
||||
if (fieldValue instanceof GenericRecord) {
|
||||
GenericRecord record = (GenericRecord) fieldValue;
|
||||
newFieldValue = rewriteRecord(record, resolveUnionSchema(field.schema(), record.getSchema().getFullName()));
|
||||
} else {
|
||||
newFieldValue = fieldValue;
|
||||
}
|
||||
newRecord.put(field.name(), newFieldValue);
|
||||
} else if (field.defaultVal() instanceof JsonProperties.Null) {
|
||||
newRecord.put(field.name(), null);
|
||||
@@ -518,6 +522,56 @@ public class HoodieAvroUtils {
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Get schema for the given field and record. Field can be nested, denoted by dot notation. e.g: a.b.c
|
||||
*
|
||||
* @param record - record containing the value of the given field
|
||||
* @param fieldName - name of the field
|
||||
* @return
|
||||
*/
|
||||
public static Schema getNestedFieldSchemaFromRecord(GenericRecord record, String fieldName) {
|
||||
String[] parts = fieldName.split("\\.");
|
||||
GenericRecord valueNode = record;
|
||||
int i = 0;
|
||||
for (; i < parts.length; i++) {
|
||||
String part = parts[i];
|
||||
Object val = valueNode.get(part);
|
||||
|
||||
if (i == parts.length - 1) {
|
||||
return resolveNullableSchema(valueNode.getSchema().getField(part).schema());
|
||||
} else {
|
||||
if (!(val instanceof GenericRecord)) {
|
||||
throw new HoodieException("Cannot find a record at part value :" + part);
|
||||
}
|
||||
valueNode = (GenericRecord) val;
|
||||
}
|
||||
}
|
||||
throw new HoodieException("Failed to get schema. Not a valid field name: " + fieldName);
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Get schema for the given field and write schema. Field can be nested, denoted by dot notation. e.g: a.b.c
|
||||
* Use this method when record is not available. Otherwise, prefer to use {@link #getNestedFieldSchemaFromRecord(GenericRecord, String)}
|
||||
*
|
||||
* @param writeSchema - write schema of the record
|
||||
* @param fieldName - name of the field
|
||||
* @return
|
||||
*/
|
||||
public static Schema getNestedFieldSchemaFromWriteSchema(Schema writeSchema, String fieldName) {
|
||||
String[] parts = fieldName.split("\\.");
|
||||
int i = 0;
|
||||
for (; i < parts.length; i++) {
|
||||
String part = parts[i];
|
||||
Schema schema = writeSchema.getField(part).schema();
|
||||
|
||||
if (i == parts.length - 1) {
|
||||
return resolveNullableSchema(schema);
|
||||
}
|
||||
}
|
||||
throw new HoodieException("Failed to get schema. Not a valid field name: " + fieldName);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the string value of the given record {@code rec} and field {@code fieldName}.
|
||||
* The field and value both could be missing.
|
||||
@@ -653,7 +707,27 @@ public class HoodieAvroUtils {
|
||||
return getRecordColumnValues(record, columns, schema.get(), consistentLogicalTimestampEnabled);
|
||||
}
|
||||
|
||||
private static Schema resolveNullableSchema(Schema schema) {
|
||||
private static Schema resolveUnionSchema(Schema schema, String fieldSchemaFullName) {
|
||||
if (schema.getType() != Schema.Type.UNION) {
|
||||
return schema;
|
||||
}
|
||||
|
||||
List<Schema> innerTypes = schema.getTypes();
|
||||
Schema nonNullType =
|
||||
innerTypes.stream()
|
||||
.filter(it -> it.getType() != Schema.Type.NULL && Objects.equals(it.getFullName(), fieldSchemaFullName))
|
||||
.findFirst()
|
||||
.orElse(null);
|
||||
|
||||
if (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;
|
||||
}
|
||||
|
||||
public static Schema resolveNullableSchema(Schema schema) {
|
||||
if (schema.getType() != Schema.Type.UNION) {
|
||||
return schema;
|
||||
}
|
||||
|
||||
@@ -44,7 +44,7 @@ public class HoodieAvroWriteSupport extends AvroWriteSupport {
|
||||
public static final String HOODIE_BLOOM_FILTER_TYPE_CODE = "hoodie_bloom_filter_type_code";
|
||||
|
||||
public HoodieAvroWriteSupport(MessageType schema, Schema avroSchema, Option<BloomFilter> bloomFilterOpt) {
|
||||
super(schema, avroSchema);
|
||||
super(schema, avroSchema, ConvertingGenericData.INSTANCE);
|
||||
this.bloomFilterOpt = bloomFilterOpt;
|
||||
}
|
||||
|
||||
|
||||
@@ -28,6 +28,7 @@ import org.apache.hudi.common.table.HoodieTableConfig;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.view.FileSystemViewStorageConfig;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.StringUtils;
|
||||
import org.apache.hudi.common.util.collection.ImmutablePair;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
@@ -585,12 +586,21 @@ public class FSUtils {
|
||||
}
|
||||
|
||||
public static Path getPartitionPath(String basePath, String partitionPath) {
|
||||
return getPartitionPath(new Path(basePath), partitionPath);
|
||||
if (StringUtils.isNullOrEmpty(partitionPath)) {
|
||||
return new Path(basePath);
|
||||
}
|
||||
|
||||
// NOTE: We have to chop leading "/" to make sure Hadoop does not treat it like
|
||||
// absolute path
|
||||
String properPartitionPath = partitionPath.startsWith("/")
|
||||
? partitionPath.substring(1)
|
||||
: partitionPath;
|
||||
return getPartitionPath(new Path(basePath), properPartitionPath);
|
||||
}
|
||||
|
||||
public static Path getPartitionPath(Path basePath, String partitionPath) {
|
||||
// FOr non-partitioned table, return only base-path
|
||||
return ((partitionPath == null) || (partitionPath.isEmpty())) ? basePath : new Path(basePath, partitionPath);
|
||||
return StringUtils.isNullOrEmpty(partitionPath) ? basePath : new Path(basePath, partitionPath);
|
||||
}
|
||||
|
||||
/**
|
||||
|
||||
@@ -20,10 +20,7 @@ package org.apache.hudi.common.model;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.io.Serializable;
|
||||
import java.util.Comparator;
|
||||
import java.util.Objects;
|
||||
import java.util.function.BiFunction;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
/**
|
||||
* Hoodie metadata for the column range of data stored in columnar format (like Parquet)
|
||||
@@ -45,23 +42,6 @@ public class HoodieColumnRangeMetadata<T extends Comparable> implements Serializ
|
||||
private final long totalSize;
|
||||
private final long totalUncompressedSize;
|
||||
|
||||
public static final BiFunction<HoodieColumnRangeMetadata<Comparable>, HoodieColumnRangeMetadata<Comparable>, HoodieColumnRangeMetadata<Comparable>> COLUMN_RANGE_MERGE_FUNCTION =
|
||||
(oldColumnRange, newColumnRange) -> new HoodieColumnRangeMetadata<Comparable>(
|
||||
newColumnRange.getFilePath(),
|
||||
newColumnRange.getColumnName(),
|
||||
(Comparable) Stream.of(oldColumnRange.getMinValue(), newColumnRange.getMinValue())
|
||||
.filter(Objects::nonNull)
|
||||
.min(Comparator.naturalOrder())
|
||||
.orElse(null),
|
||||
(Comparable) Stream.of(oldColumnRange.getMinValue(), newColumnRange.getMinValue())
|
||||
.filter(Objects::nonNull)
|
||||
.max(Comparator.naturalOrder()).orElse(null),
|
||||
oldColumnRange.getNullCount() + newColumnRange.getNullCount(),
|
||||
oldColumnRange.getValueCount() + newColumnRange.getValueCount(),
|
||||
oldColumnRange.getTotalSize() + newColumnRange.getTotalSize(),
|
||||
oldColumnRange.getTotalUncompressedSize() + newColumnRange.getTotalUncompressedSize()
|
||||
);
|
||||
|
||||
private HoodieColumnRangeMetadata(String filePath,
|
||||
String columnName,
|
||||
@Nullable T minValue,
|
||||
@@ -168,18 +148,4 @@ public class HoodieColumnRangeMetadata<T extends Comparable> implements Serializ
|
||||
String columnName) {
|
||||
return new HoodieColumnRangeMetadata<>(filePath, columnName, null, null, -1, -1, -1, -1);
|
||||
}
|
||||
|
||||
/**
|
||||
* Statistics that is collected in {@link org.apache.hudi.metadata.MetadataPartitionType#COLUMN_STATS} index.
|
||||
*/
|
||||
public static final class Stats {
|
||||
public static final String VALUE_COUNT = "value_count";
|
||||
public static final String NULL_COUNT = "null_count";
|
||||
public static final String MIN = "min";
|
||||
public static final String MAX = "max";
|
||||
public static final String TOTAL_SIZE = "total_size";
|
||||
public static final String TOTAL_UNCOMPRESSED_SIZE = "total_uncompressed_size";
|
||||
|
||||
private Stats() {}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -130,8 +130,9 @@ public class HoodieCommitMetadata implements Serializable {
|
||||
public HashMap<String, String> getFileIdAndFullPaths(String basePath) {
|
||||
HashMap<String, String> fullPaths = new HashMap<>();
|
||||
for (Map.Entry<String, String> entry : getFileIdAndRelativePaths().entrySet()) {
|
||||
String fullPath =
|
||||
(entry.getValue() != null) ? (FSUtils.getPartitionPath(basePath, entry.getValue())).toString() : null;
|
||||
String fullPath = entry.getValue() != null
|
||||
? FSUtils.getPartitionPath(basePath, entry.getValue()).toString()
|
||||
: null;
|
||||
fullPaths.put(entry.getKey(), fullPath);
|
||||
}
|
||||
return fullPaths;
|
||||
|
||||
@@ -21,7 +21,6 @@ package org.apache.hudi.common.model;
|
||||
import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
|
||||
import java.io.Serializable;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
@@ -30,13 +29,14 @@ import java.util.Map;
|
||||
* Statistics about a single Hoodie delta log operation.
|
||||
*/
|
||||
@JsonIgnoreProperties(ignoreUnknown = true)
|
||||
@SuppressWarnings("rawtypes")
|
||||
public class HoodieDeltaWriteStat extends HoodieWriteStat {
|
||||
|
||||
private int logVersion;
|
||||
private long logOffset;
|
||||
private String baseFile;
|
||||
private List<String> logFiles = new ArrayList<>();
|
||||
private Option<RecordsStats<? extends Map>> recordsStats = Option.empty();
|
||||
private Option<Map<String, HoodieColumnRangeMetadata<Comparable>>> recordsStats = Option.empty();
|
||||
|
||||
public void setLogVersion(int logVersion) {
|
||||
this.logVersion = logVersion;
|
||||
@@ -74,23 +74,11 @@ public class HoodieDeltaWriteStat extends HoodieWriteStat {
|
||||
return logFiles;
|
||||
}
|
||||
|
||||
public void setRecordsStats(RecordsStats<? extends Map> stats) {
|
||||
public void setRecordsStats(Map<String, HoodieColumnRangeMetadata<Comparable>> stats) {
|
||||
recordsStats = Option.of(stats);
|
||||
}
|
||||
|
||||
public Option<RecordsStats<? extends Map>> getRecordsStats() {
|
||||
public Option<Map<String, HoodieColumnRangeMetadata<Comparable>>> getColumnStats() {
|
||||
return recordsStats;
|
||||
}
|
||||
|
||||
public static class RecordsStats<T> implements Serializable {
|
||||
private final T recordsStats;
|
||||
|
||||
public RecordsStats(T recordsStats) {
|
||||
this.recordsStats = recordsStats;
|
||||
}
|
||||
|
||||
public T getStats() {
|
||||
return recordsStats;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -39,6 +39,35 @@ public class DateTimeUtils {
|
||||
private static final Map<String, ChronoUnit> LABEL_TO_UNIT_MAP =
|
||||
Collections.unmodifiableMap(initMap());
|
||||
|
||||
/**
|
||||
* Converts provided microseconds (from epoch) to {@link Instant}
|
||||
*/
|
||||
public static Instant microsToInstant(long microsFromEpoch) {
|
||||
long epochSeconds = microsFromEpoch / (1_000_000L);
|
||||
long nanoAdjustment = (microsFromEpoch % (1_000_000L)) * 1_000L;
|
||||
|
||||
return Instant.ofEpochSecond(epochSeconds, nanoAdjustment);
|
||||
}
|
||||
|
||||
/**
|
||||
* Converts provided {@link Instant} to microseconds (from epoch)
|
||||
*/
|
||||
public static long instantToMicros(Instant instant) {
|
||||
long seconds = instant.getEpochSecond();
|
||||
int nanos = instant.getNano();
|
||||
|
||||
if (seconds < 0 && nanos > 0) {
|
||||
long micros = Math.multiplyExact(seconds + 1, 1_000_000L);
|
||||
long adjustment = (nanos / 1_000L) - 1_000_000;
|
||||
|
||||
return Math.addExact(micros, adjustment);
|
||||
} else {
|
||||
long micros = Math.multiplyExact(seconds, 1_000_000L);
|
||||
|
||||
return Math.addExact(micros, nanos / 1_000L);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Parse input String to a {@link java.time.Instant}.
|
||||
*
|
||||
|
||||
@@ -18,10 +18,30 @@
|
||||
|
||||
package org.apache.hudi.metadata;
|
||||
|
||||
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 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;
|
||||
@@ -35,31 +55,33 @@ 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.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.TypeUtils.unsafeCast;
|
||||
import static org.apache.hudi.common.util.DateTimeUtils.microsToInstant;
|
||||
import static org.apache.hudi.common.util.DateTimeUtils.instantToMicros;
|
||||
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;
|
||||
import static org.apache.hudi.metadata.HoodieTableMetadataUtil.getPartition;
|
||||
import static org.apache.hudi.metadata.HoodieTableMetadataUtil.tryUpcastDecimal;
|
||||
|
||||
/**
|
||||
* MetadataTable records are persisted with the schema defined in HoodieMetadata.avsc.
|
||||
@@ -119,6 +141,8 @@ public class HoodieMetadataPayload implements HoodieRecordPayload<HoodieMetadata
|
||||
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();
|
||||
|
||||
private String key = null;
|
||||
private int type = 0;
|
||||
private Map<String, HoodieMetadataFileInfo> filesystemMetadata = null;
|
||||
@@ -180,8 +204,8 @@ public class HoodieMetadataPayload implements HoodieRecordPayload<HoodieMetadata
|
||||
columnStatMetadata = HoodieMetadataColumnStats.newBuilder()
|
||||
.setFileName((String) columnStatsRecord.get(COLUMN_STATS_FIELD_FILE_NAME))
|
||||
.setColumnName((String) columnStatsRecord.get(COLUMN_STATS_FIELD_COLUMN_NAME))
|
||||
.setMinValue((String) columnStatsRecord.get(COLUMN_STATS_FIELD_MIN_VALUE))
|
||||
.setMaxValue((String) columnStatsRecord.get(COLUMN_STATS_FIELD_MAX_VALUE))
|
||||
.setMinValue(columnStatsRecord.get(COLUMN_STATS_FIELD_MIN_VALUE))
|
||||
.setMaxValue(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))
|
||||
@@ -351,7 +375,7 @@ public class HoodieMetadataPayload implements HoodieRecordPayload<HoodieMetadata
|
||||
HoodieMetadataColumnStats previousColStatsRecord = previousRecord.getColumnStatMetadata().get();
|
||||
HoodieMetadataColumnStats newColumnStatsRecord = getColumnStatMetadata().get();
|
||||
|
||||
return HoodieTableMetadataUtil.mergeColumnStats(previousColStatsRecord, newColumnStatsRecord);
|
||||
return mergeColumnStatsRecords(previousColStatsRecord, newColumnStatsRecord);
|
||||
}
|
||||
|
||||
@Override
|
||||
@@ -531,29 +555,69 @@ public class HoodieMetadataPayload implements HoodieRecordPayload<HoodieMetadata
|
||||
return getColumnStatsIndexKey(partitionIndexID, fileIndexID, columnIndexID);
|
||||
}
|
||||
|
||||
public static Stream<HoodieRecord> createColumnStatsRecords(
|
||||
String partitionName, Collection<HoodieColumnRangeMetadata<Comparable>> columnRangeMetadataList, boolean isDeleted) {
|
||||
public static Stream<HoodieRecord> createColumnStatsRecords(String partitionName,
|
||||
Collection<HoodieColumnRangeMetadata<Comparable>> columnRangeMetadataList,
|
||||
boolean isDeleted) {
|
||||
return columnRangeMetadataList.stream().map(columnRangeMetadata -> {
|
||||
HoodieKey key = new HoodieKey(getColumnStatsIndexKey(partitionName, columnRangeMetadata),
|
||||
MetadataPartitionType.COLUMN_STATS.getPartitionPath());
|
||||
|
||||
HoodieMetadataPayload payload = new HoodieMetadataPayload(key.getRecordKey(),
|
||||
HoodieMetadataColumnStats.newBuilder()
|
||||
.setFileName(new Path(columnRangeMetadata.getFilePath()).getName())
|
||||
.setColumnName(columnRangeMetadata.getColumnName())
|
||||
.setMinValue(columnRangeMetadata.getMinValue() == null ? null :
|
||||
columnRangeMetadata.getMinValue().toString())
|
||||
.setMaxValue(columnRangeMetadata.getMaxValue() == null ? null :
|
||||
columnRangeMetadata.getMaxValue().toString())
|
||||
.setMinValue(wrapStatisticValue(columnRangeMetadata.getMinValue()))
|
||||
.setMaxValue(wrapStatisticValue(columnRangeMetadata.getMaxValue()))
|
||||
.setNullCount(columnRangeMetadata.getNullCount())
|
||||
.setValueCount(columnRangeMetadata.getValueCount())
|
||||
.setTotalSize(columnRangeMetadata.getTotalSize())
|
||||
.setTotalUncompressedSize(columnRangeMetadata.getTotalUncompressedSize())
|
||||
.setIsDeleted(isDeleted)
|
||||
.build());
|
||||
|
||||
return new HoodieAvroRecord<>(key, payload);
|
||||
});
|
||||
}
|
||||
|
||||
@SuppressWarnings({"rawtypes", "unchecked"})
|
||||
private static HoodieMetadataColumnStats mergeColumnStatsRecords(HoodieMetadataColumnStats prevColumnStats,
|
||||
HoodieMetadataColumnStats newColumnStats) {
|
||||
checkArgument(Objects.equals(prevColumnStats.getFileName(), newColumnStats.getFileName()));
|
||||
checkArgument(Objects.equals(prevColumnStats.getColumnName(), newColumnStats.getColumnName()));
|
||||
|
||||
if (newColumnStats.getIsDeleted()) {
|
||||
return newColumnStats;
|
||||
}
|
||||
|
||||
Comparable minValue =
|
||||
(Comparable) Stream.of(
|
||||
(Comparable) unwrapStatisticValueWrapper(prevColumnStats.getMinValue()),
|
||||
(Comparable) unwrapStatisticValueWrapper(newColumnStats.getMinValue()))
|
||||
.filter(Objects::nonNull)
|
||||
.min(Comparator.naturalOrder())
|
||||
.orElse(null);
|
||||
|
||||
Comparable maxValue =
|
||||
(Comparable) Stream.of(
|
||||
(Comparable) unwrapStatisticValueWrapper(prevColumnStats.getMinValue()),
|
||||
(Comparable) unwrapStatisticValueWrapper(newColumnStats.getMinValue()))
|
||||
.filter(Objects::nonNull)
|
||||
.max(Comparator.naturalOrder())
|
||||
.orElse(null);
|
||||
|
||||
return HoodieMetadataColumnStats.newBuilder()
|
||||
.setFileName(newColumnStats.getFileName())
|
||||
.setColumnName(newColumnStats.getColumnName())
|
||||
.setMinValue(wrapStatisticValue(minValue))
|
||||
.setMaxValue(wrapStatisticValue(maxValue))
|
||||
.setValueCount(prevColumnStats.getValueCount() + newColumnStats.getValueCount())
|
||||
.setNullCount(prevColumnStats.getNullCount() + newColumnStats.getNullCount())
|
||||
.setTotalSize(prevColumnStats.getTotalSize() + newColumnStats.getTotalSize())
|
||||
.setTotalUncompressedSize(prevColumnStats.getTotalUncompressedSize() + newColumnStats.getTotalUncompressedSize())
|
||||
.setIsDeleted(newColumnStats.getIsDeleted())
|
||||
.build();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
final StringBuilder sb = new StringBuilder("HoodieMetadataPayload {");
|
||||
@@ -579,6 +643,85 @@ public class HoodieMetadataPayload implements HoodieRecordPayload<HoodieMetadata
|
||||
return sb.toString();
|
||||
}
|
||||
|
||||
private static Object wrapStatisticValue(Comparable<?> statValue) {
|
||||
if (statValue == null) {
|
||||
return null;
|
||||
} else if (statValue instanceof Date || statValue instanceof LocalDate) {
|
||||
// NOTE: Due to breaking changes in code-gen b/w Avro 1.8.2 and 1.10, we can't
|
||||
// rely on logical types to do proper encoding of the native Java types,
|
||||
// and hereby have to encode statistic manually
|
||||
LocalDate localDate = statValue instanceof LocalDate
|
||||
? (LocalDate) statValue
|
||||
: ((Date) statValue).toLocalDate();
|
||||
return DateWrapper.newBuilder().setValue((int) localDate.toEpochDay()).build();
|
||||
} else if (statValue instanceof BigDecimal) {
|
||||
Schema valueSchema = DecimalWrapper.SCHEMA$.getField("value").schema();
|
||||
BigDecimal upcastDecimal = tryUpcastDecimal((BigDecimal) statValue, (LogicalTypes.Decimal) valueSchema.getLogicalType());
|
||||
return DecimalWrapper.newBuilder()
|
||||
.setValue(AVRO_DECIMAL_CONVERSION.toBytes(upcastDecimal, valueSchema, valueSchema.getLogicalType()))
|
||||
.build();
|
||||
} else if (statValue instanceof Timestamp) {
|
||||
// NOTE: Due to breaking changes in code-gen b/w Avro 1.8.2 and 1.10, we can't
|
||||
// rely on logical types to do proper encoding of the native Java types,
|
||||
// and hereby have to encode statistic manually
|
||||
Instant instant = ((Timestamp) statValue).toInstant();
|
||||
return TimestampMicrosWrapper.newBuilder()
|
||||
.setValue(instantToMicros(instant))
|
||||
.build();
|
||||
} else if (statValue instanceof Boolean) {
|
||||
return BooleanWrapper.newBuilder().setValue((Boolean) statValue).build();
|
||||
} else if (statValue instanceof Integer) {
|
||||
return IntWrapper.newBuilder().setValue((Integer) statValue).build();
|
||||
} else if (statValue instanceof Long) {
|
||||
return LongWrapper.newBuilder().setValue((Long) statValue).build();
|
||||
} else if (statValue instanceof Float) {
|
||||
return FloatWrapper.newBuilder().setValue((Float) statValue).build();
|
||||
} else if (statValue instanceof Double) {
|
||||
return DoubleWrapper.newBuilder().setValue((Double) statValue).build();
|
||||
} else if (statValue instanceof ByteBuffer) {
|
||||
return BytesWrapper.newBuilder().setValue((ByteBuffer) statValue).build();
|
||||
} else if (statValue instanceof String || statValue instanceof Utf8) {
|
||||
return StringWrapper.newBuilder().setValue(statValue.toString()).build();
|
||||
} else {
|
||||
throw new UnsupportedOperationException(String.format("Unsupported type of the statistic (%s)", statValue.getClass()));
|
||||
}
|
||||
}
|
||||
|
||||
public static Comparable<?> unwrapStatisticValueWrapper(Object statValueWrapper) {
|
||||
if (statValueWrapper == null) {
|
||||
return null;
|
||||
} else if (statValueWrapper instanceof DateWrapper) {
|
||||
return LocalDate.ofEpochDay(((DateWrapper) statValueWrapper).getValue());
|
||||
} else if (statValueWrapper instanceof DecimalWrapper) {
|
||||
Schema valueSchema = DecimalWrapper.SCHEMA$.getField("value").schema();
|
||||
return AVRO_DECIMAL_CONVERSION.fromBytes(((DecimalWrapper) statValueWrapper).getValue(), valueSchema, valueSchema.getLogicalType());
|
||||
} else if (statValueWrapper instanceof TimestampMicrosWrapper) {
|
||||
return microsToInstant(((TimestampMicrosWrapper) statValueWrapper).getValue());
|
||||
} else if (statValueWrapper instanceof BooleanWrapper) {
|
||||
return ((BooleanWrapper) statValueWrapper).getValue();
|
||||
} else if (statValueWrapper instanceof IntWrapper) {
|
||||
return ((IntWrapper) statValueWrapper).getValue();
|
||||
} else if (statValueWrapper instanceof LongWrapper) {
|
||||
return ((LongWrapper) statValueWrapper).getValue();
|
||||
} else if (statValueWrapper instanceof FloatWrapper) {
|
||||
return ((FloatWrapper) statValueWrapper).getValue();
|
||||
} else if (statValueWrapper instanceof DoubleWrapper) {
|
||||
return ((DoubleWrapper) statValueWrapper).getValue();
|
||||
} else if (statValueWrapper instanceof BytesWrapper) {
|
||||
return ((BytesWrapper) statValueWrapper).getValue();
|
||||
} else if (statValueWrapper instanceof StringWrapper) {
|
||||
return ((StringWrapper) statValueWrapper).getValue();
|
||||
} else if (statValueWrapper instanceof GenericRecord) {
|
||||
// NOTE: This branch could be hit b/c Avro records could be reconstructed
|
||||
// as {@code GenericRecord)
|
||||
// TODO add logical type decoding
|
||||
GenericRecord record = (GenericRecord) statValueWrapper;
|
||||
return (Comparable<?>) record.get("value");
|
||||
} else {
|
||||
throw new UnsupportedOperationException(String.format("Unsupported type of the statistic (%s)", statValueWrapper.getClass()));
|
||||
}
|
||||
}
|
||||
|
||||
private static void validatePayload(int type, Map<String, HoodieMetadataFileInfo> filesystemMetadata) {
|
||||
if (type == METADATA_TYPE_FILE_LIST) {
|
||||
filesystemMetadata.forEach((fileName, fileInfo) -> {
|
||||
|
||||
@@ -18,6 +18,14 @@
|
||||
|
||||
package org.apache.hudi.metadata;
|
||||
|
||||
import org.apache.avro.AvroTypeException;
|
||||
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.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.avro.ConvertingGenericData;
|
||||
import org.apache.hudi.avro.model.HoodieCleanMetadata;
|
||||
import org.apache.hudi.avro.model.HoodieMetadataColumnStats;
|
||||
import org.apache.hudi.avro.model.HoodieRestoreMetadata;
|
||||
@@ -55,18 +63,13 @@ import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.exception.HoodieMetadataException;
|
||||
import org.apache.hudi.io.storage.HoodieFileReader;
|
||||
import org.apache.hudi.io.storage.HoodieFileReaderFactory;
|
||||
|
||||
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;
|
||||
|
||||
import javax.annotation.Nonnull;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.math.BigDecimal;
|
||||
import java.math.RoundingMode;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
@@ -77,23 +80,19 @@ import java.util.HashMap;
|
||||
import java.util.LinkedList;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
import java.util.Set;
|
||||
import java.util.function.BiFunction;
|
||||
import java.util.function.Function;
|
||||
import java.util.stream.Collector;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
import static org.apache.hudi.avro.HoodieAvroUtils.addMetadataFields;
|
||||
import static org.apache.hudi.avro.HoodieAvroUtils.getNestedFieldValAsString;
|
||||
import static org.apache.hudi.common.model.HoodieColumnRangeMetadata.COLUMN_RANGE_MERGE_FUNCTION;
|
||||
import static org.apache.hudi.common.model.HoodieColumnRangeMetadata.Stats.MAX;
|
||||
import static org.apache.hudi.common.model.HoodieColumnRangeMetadata.Stats.MIN;
|
||||
import static org.apache.hudi.common.model.HoodieColumnRangeMetadata.Stats.NULL_COUNT;
|
||||
import static org.apache.hudi.common.model.HoodieColumnRangeMetadata.Stats.TOTAL_SIZE;
|
||||
import static org.apache.hudi.common.model.HoodieColumnRangeMetadata.Stats.TOTAL_UNCOMPRESSED_SIZE;
|
||||
import static org.apache.hudi.common.model.HoodieColumnRangeMetadata.Stats.VALUE_COUNT;
|
||||
import static org.apache.hudi.avro.HoodieAvroUtils.convertValueForSpecificDataTypes;
|
||||
import static org.apache.hudi.avro.HoodieAvroUtils.getNestedFieldSchemaFromWriteSchema;
|
||||
import static org.apache.hudi.avro.HoodieAvroUtils.resolveNullableSchema;
|
||||
import static org.apache.hudi.common.util.StringUtils.isNullOrEmpty;
|
||||
import static org.apache.hudi.common.util.ValidationUtils.checkArgument;
|
||||
import static org.apache.hudi.metadata.HoodieMetadataPayload.unwrapStatisticValueWrapper;
|
||||
import static org.apache.hudi.metadata.HoodieTableMetadata.EMPTY_PARTITION_NAME;
|
||||
import static org.apache.hudi.metadata.HoodieTableMetadata.NON_PARTITIONED_NAME;
|
||||
|
||||
@@ -108,6 +107,100 @@ public class HoodieTableMetadataUtil {
|
||||
protected static final String PARTITION_NAME_COLUMN_STATS = "column_stats";
|
||||
protected static final String PARTITION_NAME_BLOOM_FILTERS = "bloom_filters";
|
||||
|
||||
/**
|
||||
* Collects {@link HoodieColumnRangeMetadata} for the provided collection of records, pretending
|
||||
* as if provided records have been persisted w/in given {@code filePath}
|
||||
*
|
||||
* @param records target records to compute column range metadata for
|
||||
* @param targetFields columns (fields) to be collected
|
||||
* @param filePath file path value required for {@link HoodieColumnRangeMetadata}
|
||||
*
|
||||
* @return map of {@link HoodieColumnRangeMetadata} for each of the provided target fields for
|
||||
* the collection of provided records
|
||||
*/
|
||||
public static Map<String, HoodieColumnRangeMetadata<Comparable>> collectColumnRangeMetadata(List<IndexedRecord> records,
|
||||
List<Schema.Field> targetFields,
|
||||
String filePath) {
|
||||
// Helper class to calculate column stats
|
||||
class ColumnStats {
|
||||
Object minValue;
|
||||
Object maxValue;
|
||||
long nullCount;
|
||||
long valueCount;
|
||||
}
|
||||
|
||||
HashMap<String, ColumnStats> allColumnStats = new HashMap<>();
|
||||
|
||||
// Collect stats for all columns by iterating through records while accounting
|
||||
// corresponding stats
|
||||
records.forEach((record) -> {
|
||||
// For each column (field) we have to index update corresponding column stats
|
||||
// with the values from this record
|
||||
targetFields.forEach(field -> {
|
||||
ColumnStats colStats = allColumnStats.computeIfAbsent(field.name(), (ignored) -> new ColumnStats());
|
||||
|
||||
GenericRecord genericRecord = (GenericRecord) record;
|
||||
|
||||
final Object fieldVal = convertValueForSpecificDataTypes(field.schema(), genericRecord.get(field.name()), true);
|
||||
final Schema fieldSchema = getNestedFieldSchemaFromWriteSchema(genericRecord.getSchema(), field.name());
|
||||
|
||||
if (fieldVal != null && canCompare(fieldSchema)) {
|
||||
// Set the min value of the field
|
||||
if (colStats.minValue == null
|
||||
|| ConvertingGenericData.INSTANCE.compare(fieldVal, colStats.minValue, fieldSchema) < 0) {
|
||||
colStats.minValue = fieldVal;
|
||||
}
|
||||
|
||||
// Set the max value of the field
|
||||
if (colStats.maxValue == null || ConvertingGenericData.INSTANCE.compare(fieldVal, colStats.maxValue, fieldSchema) > 0) {
|
||||
colStats.maxValue = fieldVal;
|
||||
}
|
||||
|
||||
colStats.valueCount++;
|
||||
} else {
|
||||
colStats.nullCount++;
|
||||
}
|
||||
});
|
||||
});
|
||||
|
||||
Collector<HoodieColumnRangeMetadata<Comparable>, ?, Map<String, HoodieColumnRangeMetadata<Comparable>>> collector =
|
||||
Collectors.toMap(colRangeMetadata -> colRangeMetadata.getColumnName(), Function.identity());
|
||||
|
||||
return (Map<String, HoodieColumnRangeMetadata<Comparable>>) targetFields.stream()
|
||||
.map(field -> {
|
||||
ColumnStats colStats = allColumnStats.get(field.name());
|
||||
return HoodieColumnRangeMetadata.<Comparable>create(
|
||||
filePath,
|
||||
field.name(),
|
||||
colStats == null ? null : coerceToComparable(field.schema(), colStats.minValue),
|
||||
colStats == null ? null : coerceToComparable(field.schema(), colStats.maxValue),
|
||||
colStats == null ? 0 : colStats.nullCount,
|
||||
colStats == null ? 0 : colStats.valueCount,
|
||||
// NOTE: Size and compressed size statistics are set to 0 to make sure we're not
|
||||
// mixing up those provided by Parquet with the ones from other encodings,
|
||||
// since those are not directly comparable
|
||||
0,
|
||||
0
|
||||
);
|
||||
})
|
||||
.collect(collector);
|
||||
}
|
||||
|
||||
/**
|
||||
* Converts instance of {@link HoodieMetadataColumnStats} to {@link HoodieColumnRangeMetadata}
|
||||
*/
|
||||
public static HoodieColumnRangeMetadata<Comparable> convertColumnStatsRecordToColumnRangeMetadata(HoodieMetadataColumnStats columnStats) {
|
||||
return HoodieColumnRangeMetadata.<Comparable>create(
|
||||
columnStats.getFileName(),
|
||||
columnStats.getColumnName(),
|
||||
unwrapStatisticValueWrapper(columnStats.getMinValue()),
|
||||
unwrapStatisticValueWrapper(columnStats.getMaxValue()),
|
||||
columnStats.getNullCount(),
|
||||
columnStats.getValueCount(),
|
||||
columnStats.getTotalSize(),
|
||||
columnStats.getTotalUncompressedSize());
|
||||
}
|
||||
|
||||
/**
|
||||
* Delete the metadata table for the dataset. This will be invoked during upgrade/downgrade operation during which
|
||||
* no other
|
||||
@@ -457,8 +550,11 @@ public class HoodieTableMetadataUtil {
|
||||
int parallelism = Math.max(Math.min(deleteFileList.size(), recordsGenerationParams.getColumnStatsIndexParallelism()), 1);
|
||||
return engineContext.parallelize(deleteFileList, parallelism)
|
||||
.flatMap(deleteFileInfoPair -> {
|
||||
if (deleteFileInfoPair.getRight().endsWith(HoodieFileFormat.PARQUET.getFileExtension())) {
|
||||
return getColumnStats(deleteFileInfoPair.getLeft(), deleteFileInfoPair.getRight(), dataTableMetaClient, columnsToIndex, true).iterator();
|
||||
String partitionPath = deleteFileInfoPair.getLeft();
|
||||
String filePath = deleteFileInfoPair.getRight();
|
||||
|
||||
if (filePath.endsWith(HoodieFileFormat.PARQUET.getFileExtension())) {
|
||||
return getColumnStatsRecords(partitionPath, filePath, dataTableMetaClient, columnsToIndex, true).iterator();
|
||||
}
|
||||
return Collections.emptyListIterator();
|
||||
});
|
||||
@@ -531,7 +627,8 @@ public class HoodieTableMetadataUtil {
|
||||
}
|
||||
|
||||
if (recordsGenerationParams.getEnabledPartitionTypes().contains(MetadataPartitionType.COLUMN_STATS)) {
|
||||
final HoodieData<HoodieRecord> metadataColumnStatsRDD = convertFilesToColumnStatsRecords(engineContext, partitionToDeletedFiles, partitionToAppendedFiles, recordsGenerationParams);
|
||||
final HoodieData<HoodieRecord> metadataColumnStatsRDD =
|
||||
convertFilesToColumnStatsRecords(engineContext, partitionToDeletedFiles, partitionToAppendedFiles, recordsGenerationParams);
|
||||
partitionToRecordsMap.put(MetadataPartitionType.COLUMN_STATS, metadataColumnStatsRDD);
|
||||
}
|
||||
|
||||
@@ -815,7 +912,7 @@ public class HoodieTableMetadataUtil {
|
||||
|
||||
return deletedFileList.stream().flatMap(deletedFile -> {
|
||||
final String filePathWithPartition = partitionName + "/" + deletedFile;
|
||||
return getColumnStats(partition, filePathWithPartition, dataTableMetaClient, columnsToIndex, true);
|
||||
return getColumnStatsRecords(partition, filePathWithPartition, dataTableMetaClient, columnsToIndex, true);
|
||||
}).iterator();
|
||||
});
|
||||
allRecordsRDD = allRecordsRDD.union(deletedFilesRecordsRDD);
|
||||
@@ -836,7 +933,7 @@ public class HoodieTableMetadataUtil {
|
||||
return Stream.empty();
|
||||
}
|
||||
final String filePathWithPartition = partitionName + "/" + appendedFileNameLengthEntry.getKey();
|
||||
return getColumnStats(partition, filePathWithPartition, dataTableMetaClient, columnsToIndex, false);
|
||||
return getColumnStatsRecords(partition, filePathWithPartition, dataTableMetaClient, columnsToIndex, false);
|
||||
}).iterator();
|
||||
|
||||
});
|
||||
@@ -1014,63 +1111,65 @@ public class HoodieTableMetadataUtil {
|
||||
return Arrays.asList(tableConfig.getRecordKeyFields().get());
|
||||
}
|
||||
|
||||
public static HoodieMetadataColumnStats mergeColumnStats(HoodieMetadataColumnStats prevColumnStatsRecord,
|
||||
HoodieMetadataColumnStats newColumnStatsRecord) {
|
||||
checkArgument(prevColumnStatsRecord.getFileName().equals(newColumnStatsRecord.getFileName()));
|
||||
checkArgument(prevColumnStatsRecord.getColumnName().equals(newColumnStatsRecord.getColumnName()));
|
||||
|
||||
if (newColumnStatsRecord.getIsDeleted()) {
|
||||
return newColumnStatsRecord;
|
||||
}
|
||||
|
||||
return HoodieMetadataColumnStats.newBuilder()
|
||||
.setFileName(newColumnStatsRecord.getFileName())
|
||||
.setColumnName(newColumnStatsRecord.getColumnName())
|
||||
.setMinValue(Stream.of(prevColumnStatsRecord.getMinValue(), newColumnStatsRecord.getMinValue()).filter(Objects::nonNull).min(Comparator.naturalOrder()).orElse(null))
|
||||
.setMaxValue(Stream.of(prevColumnStatsRecord.getMinValue(), newColumnStatsRecord.getMinValue()).filter(Objects::nonNull).max(Comparator.naturalOrder()).orElse(null))
|
||||
.setValueCount(prevColumnStatsRecord.getValueCount() + newColumnStatsRecord.getValueCount())
|
||||
.setNullCount(prevColumnStatsRecord.getNullCount() + newColumnStatsRecord.getNullCount())
|
||||
.setTotalSize(prevColumnStatsRecord.getTotalSize() + newColumnStatsRecord.getTotalSize())
|
||||
.setTotalUncompressedSize(prevColumnStatsRecord.getTotalUncompressedSize() + newColumnStatsRecord.getTotalUncompressedSize())
|
||||
.setIsDeleted(newColumnStatsRecord.getIsDeleted())
|
||||
.build();
|
||||
}
|
||||
|
||||
public static Stream<HoodieRecord> translateWriteStatToColumnStats(HoodieWriteStat writeStat,
|
||||
private static Stream<HoodieRecord> translateWriteStatToColumnStats(HoodieWriteStat writeStat,
|
||||
HoodieTableMetaClient datasetMetaClient,
|
||||
List<String> columnsToIndex) {
|
||||
if (writeStat instanceof HoodieDeltaWriteStat && ((HoodieDeltaWriteStat) writeStat).getRecordsStats().isPresent()) {
|
||||
Map<String, HoodieColumnRangeMetadata<Comparable>> columnRangeMap = ((HoodieDeltaWriteStat) writeStat).getRecordsStats().get().getStats();
|
||||
List<HoodieColumnRangeMetadata<Comparable>> columnRangeMetadataList = new ArrayList<>(columnRangeMap.values());
|
||||
if (writeStat instanceof HoodieDeltaWriteStat && ((HoodieDeltaWriteStat) writeStat).getColumnStats().isPresent()) {
|
||||
Map<String, HoodieColumnRangeMetadata<Comparable>> columnRangeMap = ((HoodieDeltaWriteStat) writeStat).getColumnStats().get();
|
||||
Collection<HoodieColumnRangeMetadata<Comparable>> columnRangeMetadataList = columnRangeMap.values();
|
||||
return HoodieMetadataPayload.createColumnStatsRecords(writeStat.getPartitionPath(), columnRangeMetadataList, false);
|
||||
}
|
||||
return getColumnStats(writeStat.getPartitionPath(), writeStat.getPath(), datasetMetaClient, columnsToIndex, false);
|
||||
|
||||
return getColumnStatsRecords(writeStat.getPartitionPath(), writeStat.getPath(), datasetMetaClient, columnsToIndex, false);
|
||||
}
|
||||
|
||||
private static Stream<HoodieRecord> getColumnStats(final String partitionPath, final String filePathWithPartition,
|
||||
HoodieTableMetaClient datasetMetaClient,
|
||||
List<String> columnsToIndex,
|
||||
boolean isDeleted) {
|
||||
final String partition = getPartition(partitionPath);
|
||||
final int offset = partition.equals(NON_PARTITIONED_NAME) ? (filePathWithPartition.startsWith("/") ? 1 : 0)
|
||||
: partition.length() + 1;
|
||||
final String fileName = filePathWithPartition.substring(offset);
|
||||
private static Stream<HoodieRecord> getColumnStatsRecords(String partitionPath,
|
||||
String filePath,
|
||||
HoodieTableMetaClient datasetMetaClient,
|
||||
List<String> columnsToIndex,
|
||||
boolean isDeleted) {
|
||||
String partitionName = getPartition(partitionPath);
|
||||
// NOTE: We have to chop leading "/" to make sure Hadoop does not treat it like
|
||||
// absolute path
|
||||
String filePartitionPath = filePath.startsWith("/") ? filePath.substring(1) : filePath;
|
||||
String fileName = partitionName.equals(NON_PARTITIONED_NAME)
|
||||
? filePartitionPath
|
||||
: filePartitionPath.substring(partitionName.length() + 1);
|
||||
|
||||
if (filePathWithPartition.endsWith(HoodieFileFormat.PARQUET.getFileExtension())) {
|
||||
final Path fullFilePath = new Path(datasetMetaClient.getBasePath(), filePathWithPartition);
|
||||
List<HoodieColumnRangeMetadata<Comparable>> columnRangeMetadataList;
|
||||
if (!isDeleted) {
|
||||
columnRangeMetadataList = new ParquetUtils().readRangeFromParquetMetadata(
|
||||
datasetMetaClient.getHadoopConf(), fullFilePath, columnsToIndex);
|
||||
} else {
|
||||
// TODO we should delete records instead of stubbing them
|
||||
columnRangeMetadataList =
|
||||
columnsToIndex.stream().map(entry -> HoodieColumnRangeMetadata.stub(fileName, entry))
|
||||
.collect(Collectors.toList());
|
||||
if (isDeleted) {
|
||||
// TODO we should delete records instead of stubbing them
|
||||
List<HoodieColumnRangeMetadata<Comparable>> columnRangeMetadataList = columnsToIndex.stream()
|
||||
.map(entry -> HoodieColumnRangeMetadata.stub(fileName, entry))
|
||||
.collect(Collectors.toList());
|
||||
|
||||
return HoodieMetadataPayload.createColumnStatsRecords(partitionPath, columnRangeMetadataList, true);
|
||||
}
|
||||
|
||||
List<HoodieColumnRangeMetadata<Comparable>> columnRangeMetadata =
|
||||
readColumnRangeMetadataFrom(filePartitionPath, datasetMetaClient, columnsToIndex);
|
||||
|
||||
return HoodieMetadataPayload.createColumnStatsRecords(partitionPath, columnRangeMetadata, false);
|
||||
}
|
||||
|
||||
private static List<HoodieColumnRangeMetadata<Comparable>> readColumnRangeMetadataFrom(String filePath,
|
||||
HoodieTableMetaClient datasetMetaClient,
|
||||
List<String> columnsToIndex) {
|
||||
try {
|
||||
if (filePath.endsWith(HoodieFileFormat.PARQUET.getFileExtension())) {
|
||||
Path fullFilePath = new Path(datasetMetaClient.getBasePath(), filePath);
|
||||
List<HoodieColumnRangeMetadata<Comparable>> columnRangeMetadataList =
|
||||
new ParquetUtils().readRangeFromParquetMetadata(datasetMetaClient.getHadoopConf(), fullFilePath, columnsToIndex);
|
||||
|
||||
return columnRangeMetadataList;
|
||||
}
|
||||
return HoodieMetadataPayload.createColumnStatsRecords(partitionPath, columnRangeMetadataList, isDeleted);
|
||||
} else {
|
||||
throw new HoodieException("Column range index not supported for filePathWithPartition " + fileName);
|
||||
|
||||
LOG.warn("Column range index not supported for: " + filePath);
|
||||
return Collections.emptyList();
|
||||
} catch (Exception e) {
|
||||
// NOTE: In case reading column range metadata from individual file failed,
|
||||
// we simply fall back, in lieu of failing the whole task
|
||||
LOG.error("Failed to fetch column range metadata for: " + filePath);
|
||||
return Collections.emptyList();
|
||||
}
|
||||
}
|
||||
|
||||
@@ -1105,72 +1204,37 @@ public class HoodieTableMetadataUtil {
|
||||
}
|
||||
|
||||
/**
|
||||
* Accumulates column range metadata for the given field and updates the column range map.
|
||||
*
|
||||
* @param field - column for which statistics will be computed
|
||||
* @param filePath - data file path
|
||||
* @param columnRangeMap - old column range statistics, which will be merged in this computation
|
||||
* @param columnToStats - map of column to map of each stat and its value
|
||||
* Does an upcast for {@link BigDecimal} instance to align it with scale/precision expected by
|
||||
* the {@link org.apache.avro.LogicalTypes.Decimal} Avro logical type
|
||||
*/
|
||||
public static void accumulateColumnRanges(Schema.Field field, String filePath,
|
||||
Map<String, HoodieColumnRangeMetadata<Comparable>> columnRangeMap,
|
||||
Map<String, Map<String, Object>> columnToStats) {
|
||||
Map<String, Object> columnStats = columnToStats.get(field.name());
|
||||
HoodieColumnRangeMetadata<Comparable> columnRangeMetadata = HoodieColumnRangeMetadata.create(
|
||||
filePath,
|
||||
field.name(),
|
||||
(Comparable) String.valueOf(columnStats.get(MIN)),
|
||||
(Comparable) String.valueOf(columnStats.get(MAX)),
|
||||
Long.parseLong(columnStats.getOrDefault(NULL_COUNT, 0).toString()),
|
||||
Long.parseLong(columnStats.getOrDefault(VALUE_COUNT, 0).toString()),
|
||||
Long.parseLong(columnStats.getOrDefault(TOTAL_SIZE, 0).toString()),
|
||||
Long.parseLong(columnStats.getOrDefault(TOTAL_UNCOMPRESSED_SIZE, 0).toString())
|
||||
);
|
||||
columnRangeMap.merge(field.name(), columnRangeMetadata, COLUMN_RANGE_MERGE_FUNCTION);
|
||||
}
|
||||
public static BigDecimal tryUpcastDecimal(BigDecimal value, final LogicalTypes.Decimal decimal) {
|
||||
final int scale = decimal.getScale();
|
||||
final int valueScale = value.scale();
|
||||
|
||||
/**
|
||||
* Aggregates column stats for each field.
|
||||
*
|
||||
* @param record - current record
|
||||
* @param fields - fields for which stats will be aggregated
|
||||
* @param columnToStats - map of column to map of each stat and its value which gets updates in this method
|
||||
* @param consistentLogicalTimestampEnabled - flag to deal with logical timestamp type when getting column value
|
||||
*/
|
||||
public static void aggregateColumnStats(IndexedRecord record, List<Schema.Field> fields,
|
||||
Map<String, Map<String, Object>> columnToStats,
|
||||
boolean consistentLogicalTimestampEnabled) {
|
||||
if (!(record instanceof GenericRecord)) {
|
||||
throw new HoodieIOException("Record is not a generic type to get column range metadata!");
|
||||
boolean scaleAdjusted = false;
|
||||
if (valueScale != scale) {
|
||||
try {
|
||||
value = value.setScale(scale, RoundingMode.UNNECESSARY);
|
||||
scaleAdjusted = true;
|
||||
} catch (ArithmeticException aex) {
|
||||
throw new AvroTypeException(
|
||||
"Cannot encode decimal with scale " + valueScale + " as scale " + scale + " without rounding");
|
||||
}
|
||||
}
|
||||
|
||||
fields.forEach(field -> {
|
||||
Map<String, Object> columnStats = columnToStats.getOrDefault(field.name(), new HashMap<>());
|
||||
final String fieldVal = getNestedFieldValAsString((GenericRecord) record, field.name(), true, consistentLogicalTimestampEnabled);
|
||||
// update stats
|
||||
final int fieldSize = fieldVal == null ? 0 : fieldVal.length();
|
||||
columnStats.put(TOTAL_SIZE, Long.parseLong(columnStats.getOrDefault(TOTAL_SIZE, 0).toString()) + fieldSize);
|
||||
columnStats.put(TOTAL_UNCOMPRESSED_SIZE, Long.parseLong(columnStats.getOrDefault(TOTAL_UNCOMPRESSED_SIZE, 0).toString()) + fieldSize);
|
||||
|
||||
if (!isNullOrEmpty(fieldVal)) {
|
||||
// set the min value of the field
|
||||
if (!columnStats.containsKey(MIN)) {
|
||||
columnStats.put(MIN, fieldVal);
|
||||
}
|
||||
if (fieldVal.compareTo(String.valueOf(columnStats.get(MIN))) < 0) {
|
||||
columnStats.put(MIN, fieldVal);
|
||||
}
|
||||
// set the max value of the field
|
||||
if (fieldVal.compareTo(String.valueOf(columnStats.getOrDefault(MAX, ""))) > 0) {
|
||||
columnStats.put(MAX, fieldVal);
|
||||
}
|
||||
// increment non-null value count
|
||||
columnStats.put(VALUE_COUNT, Long.parseLong(columnStats.getOrDefault(VALUE_COUNT, 0).toString()) + 1);
|
||||
int precision = decimal.getPrecision();
|
||||
int valuePrecision = value.precision();
|
||||
if (valuePrecision > precision) {
|
||||
if (scaleAdjusted) {
|
||||
throw new AvroTypeException("Cannot encode decimal with precision " + valuePrecision + " as max precision "
|
||||
+ precision + ". This is after safely adjusting scale from " + valueScale + " to required " + scale);
|
||||
} else {
|
||||
// increment null value count
|
||||
columnStats.put(NULL_COUNT, Long.parseLong(columnStats.getOrDefault(NULL_COUNT, 0).toString()) + 1);
|
||||
throw new AvroTypeException(
|
||||
"Cannot encode decimal with precision " + valuePrecision + " as max precision " + precision);
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
return value;
|
||||
}
|
||||
|
||||
private static Option<Schema> tryResolveSchemaForTable(HoodieTableMetaClient dataTableMetaClient) {
|
||||
@@ -1178,14 +1242,82 @@ public class HoodieTableMetadataUtil {
|
||||
return Option.empty();
|
||||
}
|
||||
|
||||
TableSchemaResolver schemaResolver = new TableSchemaResolver(dataTableMetaClient);
|
||||
try {
|
||||
TableSchemaResolver schemaResolver = new TableSchemaResolver(dataTableMetaClient);
|
||||
return Option.of(schemaResolver.getTableAvroSchema());
|
||||
} catch (Exception e) {
|
||||
throw new HoodieException("Failed to get latest columns for " + dataTableMetaClient.getBasePath(), e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Given a schema, coerces provided value to instance of {@link Comparable<?>} such that
|
||||
* it could subsequently used in column stats
|
||||
*
|
||||
* NOTE: This method has to stay compatible with the semantic of
|
||||
* {@link ParquetUtils#readRangeFromParquetMetadata} as they are used in tandem
|
||||
*/
|
||||
private static Comparable<?> coerceToComparable(Schema schema, Object val) {
|
||||
if (val == null) {
|
||||
return null;
|
||||
}
|
||||
|
||||
switch (schema.getType()) {
|
||||
case UNION:
|
||||
// TODO we need to handle unions in general case as well
|
||||
return coerceToComparable(resolveNullableSchema(schema), val);
|
||||
|
||||
case FIXED:
|
||||
case BYTES:
|
||||
if (schema.getLogicalType() instanceof LogicalTypes.Decimal) {
|
||||
return (Comparable<?>) val;
|
||||
}
|
||||
return (ByteBuffer) val;
|
||||
|
||||
|
||||
case INT:
|
||||
if (schema.getLogicalType() == LogicalTypes.date()
|
||||
|| schema.getLogicalType() == LogicalTypes.timeMillis()) {
|
||||
// NOTE: This type will be either {@code java.sql.Date} or {org.joda.LocalDate}
|
||||
// depending on the Avro version. Hence, we simply cast it to {@code Comparable<?>}
|
||||
return (Comparable<?>) val;
|
||||
}
|
||||
return (Integer) val;
|
||||
|
||||
case LONG:
|
||||
if (schema.getLogicalType() == LogicalTypes.timeMicros()
|
||||
|| schema.getLogicalType() == LogicalTypes.timestampMicros()
|
||||
|| schema.getLogicalType() == LogicalTypes.timestampMillis()) {
|
||||
// NOTE: This type will be either {@code java.sql.Date} or {org.joda.LocalDate}
|
||||
// depending on the Avro version. Hence, we simply cast it to {@code Comparable<?>}
|
||||
return (Comparable<?>) val;
|
||||
}
|
||||
return (Long) val;
|
||||
|
||||
case STRING:
|
||||
case FLOAT:
|
||||
case DOUBLE:
|
||||
case BOOLEAN:
|
||||
return (Comparable<?>) val;
|
||||
|
||||
|
||||
// TODO add support for those types
|
||||
case ENUM:
|
||||
case MAP:
|
||||
case NULL:
|
||||
case RECORD:
|
||||
case ARRAY:
|
||||
return null;
|
||||
|
||||
default:
|
||||
throw new IllegalStateException("Unexpected type: " + schema.getType());
|
||||
}
|
||||
}
|
||||
|
||||
private static boolean canCompare(Schema schema) {
|
||||
return schema.getType() != Schema.Type.MAP;
|
||||
}
|
||||
|
||||
public static Set<String> getInflightMetadataPartitions(HoodieTableConfig tableConfig) {
|
||||
return StringUtils.toSet(tableConfig.getMetadataPartitionsInflight());
|
||||
}
|
||||
|
||||
@@ -31,6 +31,8 @@ public enum MetadataPartitionType {
|
||||
// FileId prefix used for all file groups in this partition.
|
||||
private final String fileIdPrefix;
|
||||
// Total file groups
|
||||
// TODO fix: enum should not have any mutable aspect as this compromises whole idea
|
||||
// of the inum being static, immutable entity
|
||||
private int fileGroupCount = 1;
|
||||
|
||||
MetadataPartitionType(final String partitionPath, final String fileIdPrefix) {
|
||||
|
||||
@@ -19,6 +19,7 @@
|
||||
package org.apache.hudi.avro;
|
||||
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.testutils.SchemaTestUtil;
|
||||
import org.apache.hudi.exception.SchemaCompatibilityException;
|
||||
|
||||
import org.apache.avro.JsonProperties;
|
||||
@@ -27,12 +28,14 @@ import org.apache.avro.generic.GenericData;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.math.BigDecimal;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import static org.apache.hudi.avro.HoodieAvroUtils.getNestedFieldSchemaFromWriteSchema;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertNotNull;
|
||||
import static org.junit.jupiter.api.Assertions.assertNull;
|
||||
@@ -88,6 +91,12 @@ public class TestHoodieAvroUtils {
|
||||
+ "{\"name\":\"decimal_col\",\"type\":[\"null\","
|
||||
+ "{\"type\":\"bytes\",\"logicalType\":\"decimal\",\"precision\":8,\"scale\":4}],\"default\":null}]}";
|
||||
|
||||
private static String SCHEMA_WITH_NESTED_FIELD = "{\"name\":\"MyClass\",\"type\":\"record\",\"namespace\":\"com.acme.avro\",\"fields\":["
|
||||
+ "{\"name\":\"firstname\",\"type\":\"string\"},"
|
||||
+ "{\"name\":\"lastname\",\"type\":\"string\"},"
|
||||
+ "{\"name\":\"student\",\"type\":{\"name\":\"student\",\"type\":\"record\",\"fields\":["
|
||||
+ "{\"name\":\"firstname\",\"type\":[\"null\" ,\"string\"],\"default\": null},{\"name\":\"lastname\",\"type\":[\"null\" ,\"string\"],\"default\": null}]}}]}";
|
||||
|
||||
@Test
|
||||
public void testPropsPresent() {
|
||||
Schema schema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(EXAMPLE_SCHEMA));
|
||||
@@ -248,7 +257,7 @@ public class TestHoodieAvroUtils {
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetNestedFieldValWithDecimalFiled() {
|
||||
public void testGetNestedFieldValWithDecimalField() {
|
||||
GenericRecord rec = new GenericData.Record(new Schema.Parser().parse(SCHEMA_WITH_DECIMAL_FIELD));
|
||||
rec.put("key_col", "key");
|
||||
BigDecimal bigDecimal = new BigDecimal("1234.5678");
|
||||
@@ -264,4 +273,36 @@ public class TestHoodieAvroUtils {
|
||||
assertEquals(0, buffer.position());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetNestedFieldSchema() throws IOException {
|
||||
Schema schema = SchemaTestUtil.getEvolvedSchema();
|
||||
GenericRecord rec = new GenericData.Record(schema);
|
||||
rec.put("field1", "key1");
|
||||
rec.put("field2", "val1");
|
||||
rec.put("name", "val2");
|
||||
rec.put("favorite_number", 2);
|
||||
// test simple field schema
|
||||
assertEquals(Schema.create(Schema.Type.STRING), getNestedFieldSchemaFromWriteSchema(rec.getSchema(), "field1"));
|
||||
|
||||
GenericRecord rec2 = new GenericData.Record(schema);
|
||||
rec2.put("field1", "key1");
|
||||
rec2.put("field2", "val1");
|
||||
rec2.put("name", "val2");
|
||||
rec2.put("favorite_number", 12);
|
||||
// test comparison of non-string type
|
||||
assertEquals(-1, GenericData.get().compare(rec.get("favorite_number"), rec2.get("favorite_number"), getNestedFieldSchemaFromWriteSchema(rec.getSchema(), "favorite_number")));
|
||||
|
||||
// test nested field schema
|
||||
Schema nestedSchema = new Schema.Parser().parse(SCHEMA_WITH_NESTED_FIELD);
|
||||
GenericRecord rec3 = new GenericData.Record(nestedSchema);
|
||||
rec3.put("firstname", "person1");
|
||||
rec3.put("lastname", "person2");
|
||||
GenericRecord studentRecord = new GenericData.Record(rec3.getSchema().getField("student").schema());
|
||||
studentRecord.put("firstname", "person1");
|
||||
studentRecord.put("lastname", "person2");
|
||||
rec3.put("student", studentRecord);
|
||||
|
||||
assertEquals(Schema.create(Schema.Type.STRING), getNestedFieldSchemaFromWriteSchema(rec3.getSchema(), "student.firstname"));
|
||||
assertEquals(Schema.create(Schema.Type.STRING), getNestedFieldSchemaFromWriteSchema(nestedSchema, "student.firstname"));
|
||||
}
|
||||
}
|
||||
|
||||
@@ -1749,40 +1749,39 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
|
||||
|
||||
FileCreateUtils.createDeltaCommit(basePath, "100", fs);
|
||||
|
||||
HoodieLogFileReader reader = new HoodieLogFileReader(fs, new HoodieLogFile(writer.getLogFile().getPath(),
|
||||
fs.getFileStatus(writer.getLogFile().getPath()).getLen()), SchemaTestUtil.getSimpleSchema(),
|
||||
bufferSize, readBlocksLazily, true);
|
||||
HoodieLogFile logFile = new HoodieLogFile(writer.getLogFile().getPath(), fs.getFileStatus(writer.getLogFile().getPath()).getLen());
|
||||
try (HoodieLogFileReader reader = new HoodieLogFileReader(fs, logFile, SchemaTestUtil.getSimpleSchema(), bufferSize, readBlocksLazily, true)) {
|
||||
|
||||
assertTrue(reader.hasPrev(), "Last block should be available");
|
||||
HoodieLogBlock prevBlock = reader.prev();
|
||||
HoodieDataBlock dataBlockRead = (HoodieDataBlock) prevBlock;
|
||||
assertTrue(reader.hasPrev(), "Last block should be available");
|
||||
HoodieLogBlock prevBlock = reader.prev();
|
||||
HoodieDataBlock dataBlockRead = (HoodieDataBlock) prevBlock;
|
||||
|
||||
List<IndexedRecord> recordsRead1 = getRecords(dataBlockRead);
|
||||
assertEquals(copyOfRecords3.size(), recordsRead1.size(),
|
||||
"Third records size should be equal to the written records size");
|
||||
assertEquals(copyOfRecords3, recordsRead1,
|
||||
"Both records lists should be the same. (ordering guaranteed)");
|
||||
List<IndexedRecord> recordsRead1 = getRecords(dataBlockRead);
|
||||
assertEquals(copyOfRecords3.size(), recordsRead1.size(),
|
||||
"Third records size should be equal to the written records size");
|
||||
assertEquals(copyOfRecords3, recordsRead1,
|
||||
"Both records lists should be the same. (ordering guaranteed)");
|
||||
|
||||
assertTrue(reader.hasPrev(), "Second block should be available");
|
||||
prevBlock = reader.prev();
|
||||
dataBlockRead = (HoodieDataBlock) prevBlock;
|
||||
List<IndexedRecord> recordsRead2 = getRecords(dataBlockRead);
|
||||
assertEquals(copyOfRecords2.size(), recordsRead2.size(),
|
||||
"Read records size should be equal to the written records size");
|
||||
assertEquals(copyOfRecords2, recordsRead2,
|
||||
"Both records lists should be the same. (ordering guaranteed)");
|
||||
assertTrue(reader.hasPrev(), "Second block should be available");
|
||||
prevBlock = reader.prev();
|
||||
dataBlockRead = (HoodieDataBlock) prevBlock;
|
||||
List<IndexedRecord> recordsRead2 = getRecords(dataBlockRead);
|
||||
assertEquals(copyOfRecords2.size(), recordsRead2.size(),
|
||||
"Read records size should be equal to the written records size");
|
||||
assertEquals(copyOfRecords2, recordsRead2,
|
||||
"Both records lists should be the same. (ordering guaranteed)");
|
||||
|
||||
assertTrue(reader.hasPrev(), "First block should be available");
|
||||
prevBlock = reader.prev();
|
||||
dataBlockRead = (HoodieDataBlock) prevBlock;
|
||||
List<IndexedRecord> recordsRead3 = getRecords(dataBlockRead);
|
||||
assertEquals(copyOfRecords1.size(), recordsRead3.size(),
|
||||
"Read records size should be equal to the written records size");
|
||||
assertEquals(copyOfRecords1, recordsRead3,
|
||||
"Both records lists should be the same. (ordering guaranteed)");
|
||||
assertTrue(reader.hasPrev(), "First block should be available");
|
||||
prevBlock = reader.prev();
|
||||
dataBlockRead = (HoodieDataBlock) prevBlock;
|
||||
List<IndexedRecord> recordsRead3 = getRecords(dataBlockRead);
|
||||
assertEquals(copyOfRecords1.size(), recordsRead3.size(),
|
||||
"Read records size should be equal to the written records size");
|
||||
assertEquals(copyOfRecords1, recordsRead3,
|
||||
"Both records lists should be the same. (ordering guaranteed)");
|
||||
|
||||
assertFalse(reader.hasPrev());
|
||||
reader.close();
|
||||
assertFalse(reader.hasPrev());
|
||||
}
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@@ -1830,19 +1829,20 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
|
||||
writer.close();
|
||||
|
||||
// First round of reads - we should be able to read the first block and then EOF
|
||||
HoodieLogFileReader reader =
|
||||
new HoodieLogFileReader(fs, new HoodieLogFile(writer.getLogFile().getPath(),
|
||||
fs.getFileStatus(writer.getLogFile().getPath()).getLen()), schema, bufferSize, readBlocksLazily, true);
|
||||
HoodieLogFile logFile = new HoodieLogFile(writer.getLogFile().getPath(), fs.getFileStatus(writer.getLogFile().getPath()).getLen());
|
||||
|
||||
assertTrue(reader.hasPrev(), "Last block should be available");
|
||||
HoodieLogBlock block = reader.prev();
|
||||
assertTrue(block instanceof HoodieDataBlock, "Last block should be datablock");
|
||||
try (HoodieLogFileReader reader =
|
||||
new HoodieLogFileReader(fs, logFile, schema, bufferSize, readBlocksLazily, true)) {
|
||||
|
||||
assertTrue(reader.hasPrev(), "Last block should be available");
|
||||
assertThrows(CorruptedLogFileException.class, () -> {
|
||||
reader.prev();
|
||||
});
|
||||
reader.close();
|
||||
assertTrue(reader.hasPrev(), "Last block should be available");
|
||||
HoodieLogBlock block = reader.prev();
|
||||
assertTrue(block instanceof HoodieDataBlock, "Last block should be datablock");
|
||||
|
||||
assertTrue(reader.hasPrev(), "Last block should be available");
|
||||
assertThrows(CorruptedLogFileException.class, () -> {
|
||||
reader.prev();
|
||||
});
|
||||
}
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@@ -1882,28 +1882,28 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
|
||||
|
||||
FileCreateUtils.createDeltaCommit(basePath, "100", fs);
|
||||
|
||||
HoodieLogFileReader reader = new HoodieLogFileReader(fs, new HoodieLogFile(writer.getLogFile().getPath(),
|
||||
fs.getFileStatus(writer.getLogFile().getPath()).getLen()), SchemaTestUtil.getSimpleSchema(),
|
||||
bufferSize, readBlocksLazily, true);
|
||||
HoodieLogFile logFile = new HoodieLogFile(writer.getLogFile().getPath(), fs.getFileStatus(writer.getLogFile().getPath()).getLen());
|
||||
try (HoodieLogFileReader reader =
|
||||
new HoodieLogFileReader(fs, logFile, SchemaTestUtil.getSimpleSchema(), bufferSize, readBlocksLazily, true)) {
|
||||
|
||||
assertTrue(reader.hasPrev(), "Third block should be available");
|
||||
reader.moveToPrev();
|
||||
assertTrue(reader.hasPrev(), "Third block should be available");
|
||||
reader.moveToPrev();
|
||||
|
||||
assertTrue(reader.hasPrev(), "Second block should be available");
|
||||
reader.moveToPrev();
|
||||
assertTrue(reader.hasPrev(), "Second block should be available");
|
||||
reader.moveToPrev();
|
||||
|
||||
// After moving twice, this last reader.prev() should read the First block written
|
||||
assertTrue(reader.hasPrev(), "First block should be available");
|
||||
HoodieLogBlock prevBlock = reader.prev();
|
||||
HoodieDataBlock dataBlockRead = (HoodieDataBlock) prevBlock;
|
||||
List<IndexedRecord> recordsRead = getRecords(dataBlockRead);
|
||||
assertEquals(copyOfRecords1.size(), recordsRead.size(),
|
||||
"Read records size should be equal to the written records size");
|
||||
assertEquals(copyOfRecords1, recordsRead,
|
||||
"Both records lists should be the same. (ordering guaranteed)");
|
||||
// After moving twice, this last reader.prev() should read the First block written
|
||||
assertTrue(reader.hasPrev(), "First block should be available");
|
||||
HoodieLogBlock prevBlock = reader.prev();
|
||||
HoodieDataBlock dataBlockRead = (HoodieDataBlock) prevBlock;
|
||||
List<IndexedRecord> recordsRead = getRecords(dataBlockRead);
|
||||
assertEquals(copyOfRecords1.size(), recordsRead.size(),
|
||||
"Read records size should be equal to the written records size");
|
||||
assertEquals(copyOfRecords1, recordsRead,
|
||||
"Both records lists should be the same. (ordering guaranteed)");
|
||||
|
||||
assertFalse(reader.hasPrev());
|
||||
reader.close();
|
||||
assertFalse(reader.hasPrev());
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
|
||||
@@ -99,15 +99,6 @@ public class FileCreateUtils {
|
||||
return String.format("%s_%s_%s%s%s.%s", fileId, WRITE_TOKEN, instantTime, fileExtension, HoodieTableMetaClient.MARKER_EXTN, ioType);
|
||||
}
|
||||
|
||||
private static void createMetaFile(String basePath, String instantTime, String suffix) throws IOException {
|
||||
Path parentPath = Paths.get(basePath, HoodieTableMetaClient.METAFOLDER_NAME);
|
||||
Files.createDirectories(parentPath);
|
||||
Path metaFilePath = parentPath.resolve(instantTime + suffix);
|
||||
if (Files.notExists(metaFilePath)) {
|
||||
Files.createFile(metaFilePath);
|
||||
}
|
||||
}
|
||||
|
||||
private static void createMetaFile(String basePath, String instantTime, String suffix, FileSystem fs) throws IOException {
|
||||
org.apache.hadoop.fs.Path parentPath = new org.apache.hadoop.fs.Path(basePath, HoodieTableMetaClient.METAFOLDER_NAME);
|
||||
if (!fs.exists(parentPath)) {
|
||||
@@ -119,12 +110,20 @@ public class FileCreateUtils {
|
||||
}
|
||||
}
|
||||
|
||||
private static void createMetaFile(String basePath, String instantTime, String suffix) throws IOException {
|
||||
createMetaFile(basePath, instantTime, suffix, "".getBytes());
|
||||
}
|
||||
|
||||
private static void createMetaFile(String basePath, String instantTime, String suffix, byte[] content) throws IOException {
|
||||
Path parentPath = Paths.get(basePath, HoodieTableMetaClient.METAFOLDER_NAME);
|
||||
Files.createDirectories(parentPath);
|
||||
Path metaFilePath = parentPath.resolve(instantTime + suffix);
|
||||
if (Files.notExists(metaFilePath)) {
|
||||
Files.write(metaFilePath, content);
|
||||
if (content.length == 0) {
|
||||
Files.createFile(metaFilePath);
|
||||
} else {
|
||||
Files.write(metaFilePath, content);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -117,8 +117,12 @@ import static org.apache.hudi.common.util.StringUtils.EMPTY_STRING;
|
||||
|
||||
public class HoodieTestTable {
|
||||
|
||||
public static final String PHONY_TABLE_SCHEMA =
|
||||
"{\"namespace\": \"org.apache.hudi.avro.model\", \"type\": \"record\", \"name\": \"PhonyRecord\", \"fields\": []}";
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(HoodieTestTable.class);
|
||||
private static final Random RANDOM = new Random();
|
||||
|
||||
protected static HoodieTestTableState testTableState;
|
||||
private final List<String> inflightCommits = new ArrayList<>();
|
||||
|
||||
@@ -215,7 +219,7 @@ public class HoodieTestTable {
|
||||
writeStats.addAll(generateHoodieWriteStatForPartitionLogFiles(testTableState.getPartitionToLogFileInfoMap(commitTime), commitTime, bootstrap));
|
||||
}
|
||||
Map<String, String> extraMetadata = createImmutableMap("test", "test");
|
||||
return buildMetadata(writeStats, partitionToReplaceFileIds, Option.of(extraMetadata), operationType, EMPTY_STRING, action);
|
||||
return buildMetadata(writeStats, partitionToReplaceFileIds, Option.of(extraMetadata), operationType, PHONY_TABLE_SCHEMA, action);
|
||||
}
|
||||
|
||||
public HoodieTestTable moveInflightCommitToComplete(String instantTime, HoodieCommitMetadata metadata) throws IOException {
|
||||
@@ -779,7 +783,7 @@ public class HoodieTestTable {
|
||||
this.withBaseFilesInPartition(partition, testTableState.getPartitionToBaseFileInfoMap(commitTime).get(partition));
|
||||
}
|
||||
HoodieReplaceCommitMetadata replaceMetadata =
|
||||
(HoodieReplaceCommitMetadata) buildMetadata(writeStats, partitionToReplaceFileIds, Option.empty(), CLUSTER, EMPTY_STRING,
|
||||
(HoodieReplaceCommitMetadata) buildMetadata(writeStats, partitionToReplaceFileIds, Option.empty(), CLUSTER, PHONY_TABLE_SCHEMA,
|
||||
REPLACE_COMMIT_ACTION);
|
||||
addReplaceCommit(commitTime, Option.empty(), Option.empty(), replaceMetadata);
|
||||
return replaceMetadata;
|
||||
|
||||
Reference in New Issue
Block a user