yihua commented on code in PR #13711: URL: https://github.com/apache/hudi/pull/13711#discussion_r2302297589
########## hudi-hadoop-common/src/main/java/org/apache/parquet/avro/HoodieAvroSchemaConverter.java: ########## @@ -0,0 +1,593 @@ +/* + * 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.parquet.avro; + +import org.apache.avro.LogicalType; +import org.apache.avro.LogicalTypes; +import org.apache.avro.Schema; + +import org.apache.hadoop.conf.Configuration; +import org.apache.parquet.schema.ConversionPatterns; +import org.apache.parquet.schema.GroupType; +import org.apache.parquet.schema.LogicalTypeAnnotation; +import org.apache.parquet.schema.MessageType; +import org.apache.parquet.schema.PrimitiveType; +import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName; +import org.apache.parquet.schema.Type; +import org.apache.parquet.schema.Types; +import org.apache.parquet.schema.LogicalTypeAnnotation.UUIDLogicalTypeAnnotation; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import static org.apache.avro.JsonProperties.NULL_VALUE; +import static org.apache.parquet.avro.AvroReadSupport.READ_INT96_AS_FIXED; +import static org.apache.parquet.avro.AvroReadSupport.READ_INT96_AS_FIXED_DEFAULT; +import static org.apache.parquet.avro.AvroRecordConverter.getRuntimeAvroVersion; +import static org.apache.parquet.avro.AvroWriteSupport.WRITE_FIXED_AS_INT96; +import static org.apache.parquet.avro.AvroWriteSupport.WRITE_OLD_LIST_STRUCTURE; +import static org.apache.parquet.avro.AvroWriteSupport.WRITE_OLD_LIST_STRUCTURE_DEFAULT; +import static org.apache.parquet.avro.AvroWriteSupport.WRITE_PARQUET_UUID; +import static org.apache.parquet.avro.AvroWriteSupport.WRITE_PARQUET_UUID_DEFAULT; +import static org.apache.parquet.schema.LogicalTypeAnnotation.TimeUnit.MICROS; +import static org.apache.parquet.schema.LogicalTypeAnnotation.TimeUnit.MILLIS; +import static org.apache.parquet.schema.LogicalTypeAnnotation.dateType; +import static org.apache.parquet.schema.LogicalTypeAnnotation.decimalType; +import static org.apache.parquet.schema.LogicalTypeAnnotation.enumType; +import static org.apache.parquet.schema.LogicalTypeAnnotation.stringType; +import static org.apache.parquet.schema.LogicalTypeAnnotation.timeType; +import static org.apache.parquet.schema.LogicalTypeAnnotation.timestampType; +import static org.apache.parquet.schema.LogicalTypeAnnotation.uuidType; +import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.BINARY; +import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.BOOLEAN; +import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.DOUBLE; +import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY; +import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.FLOAT; +import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.INT32; +import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.INT64; +import static org.apache.parquet.schema.Type.Repetition.REPEATED; + +/** + * <p> + * Converts an Avro schema into a Parquet schema, or vice versa. See package + * documentation for details of the mapping. + * </p> + */ +@SuppressWarnings("all") +public class HoodieAvroSchemaConverter { Review Comment: Add more javadocs on why this is needed, where the code is copied from, additional changes in Hudi, and how to maintain this class? ########## hudi-common/src/test/resources/nullWrong.avsc: ########## @@ -197,7 +197,48 @@ }, "null" ] - } + }, + { + "name": "timeMicroField", Review Comment: Is this needed as it's for `nullWrong` schema which seems to be unrelated? ########## hudi-hadoop-common/src/main/java/org/apache/hudi/common/util/OrcUtils.java: ########## @@ -257,7 +258,7 @@ public Schema readAvroSchema(HoodieStorage storage, StoragePath filePath) { } @Override - public List<HoodieColumnRangeMetadata<Comparable>> readColumnStatsFromMetadata(HoodieStorage storage, StoragePath filePath, List<String> columnList) { + public List<HoodieColumnRangeMetadata<Comparable>> readColumnStatsFromMetadata(HoodieStorage storage, StoragePath filePath, List<String> columnList, HoodieIndexVersion indexVersion) { Review Comment: Similar here on avoiding `indexVersion` ########## hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/HoodieBloomIndex.java: ########## @@ -231,12 +231,13 @@ protected List<Pair<String, BloomIndexFileInfo>> loadColumnRangesFromMetaIndex( List<Pair<String, BloomIndexFileInfo>> result = new ArrayList<>(fileToColumnStatsMap.size()); for (Map.Entry<Pair<String, String>, HoodieMetadataColumnStats> entry : fileToColumnStatsMap.entrySet()) { + ValueMetadata valueMetadata = ValueMetadata.getValueMetadata(entry.getValue().getValueType()); result.add(Pair.of(entry.getKey().getLeft(), new BloomIndexFileInfo( partitionAndFileNameToFileId.get(entry.getKey()), // NOTE: Here we assume that the type of the primary key field is string - unwrapAvroValueWrapper(entry.getValue().getMinValue()).toString(), - unwrapAvroValueWrapper(entry.getValue().getMaxValue()).toString() + valueMetadata.unwrapValue(entry.getValue().getMinValue()).toString(), + valueMetadata.unwrapValue(entry.getValue().getMaxValue()).toString() Review Comment: For ensuring backwards compatibility through tests, could you generate test artifacts/tables by using 0.15.0 and 1.0.2 release (and from this PR) with column stats enabled and columns of different primitive and logical types (use MOR with log files), and used that for validating col stats reading and data skipping? That'll give us much more confidence on compatibility. ########## hudi-hadoop-common/src/main/java/org/apache/hudi/common/util/ParquetUtils.java: ########## @@ -473,7 +481,10 @@ private <T extends Comparable<T>> HoodieColumnRangeMetadata<T> getColumnRangeInF .reduce(HoodieColumnRangeMetadata::merge).get(); } - private static Comparable<?> convertToNativeJavaType(PrimitiveType primitiveType, Comparable<?> val) { + private static Comparable<?> convertToNativeJavaType(PrimitiveType primitiveType, Comparable<?> val, ValueMetadata valueMetadata) { + if (valueMetadata.getValueType() != ValueType.V1) { + return valueMetadata.standardizeJavaTypeAndPromote(val); + } Review Comment: Could `ParquetUtils#readColumnStatsFromMetadata` avoid checking the value type or index version and let the caller to do the conversion, i.e., reading parquet metadata and returning the raw stats should not care about the Hudi-related semantics? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
