talatuyarer commented on code in PR #15475: URL: https://github.com/apache/iceberg/pull/15475#discussion_r2876730376
########## flink/v2.1/flink/src/main/java/org/apache/flink/formats/avro/typeutils/AvroSchemaConverter.java: ########## @@ -0,0 +1,615 @@ +/* + * 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.flink.formats.avro.typeutils; + +import java.util.List; +import org.apache.avro.LogicalTypes; +import org.apache.avro.Schema; +import org.apache.avro.SchemaBuilder; +import org.apache.avro.SchemaParseException; +import org.apache.avro.specific.SpecificData; +import org.apache.avro.specific.SpecificRecord; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.formats.avro.AvroRowDataDeserializationSchema; +import org.apache.flink.formats.avro.AvroRowDataSerializationSchema; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.legacy.types.logical.TypeInformationRawType; +import org.apache.flink.table.types.AtomicDataType; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.ArrayType; +import org.apache.flink.table.types.logical.DecimalType; +import org.apache.flink.table.types.logical.IntType; +import org.apache.flink.table.types.logical.LocalZonedTimestampType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.LogicalTypeFamily; +import org.apache.flink.table.types.logical.MapType; +import org.apache.flink.table.types.logical.MultisetType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.logical.TimeType; +import org.apache.flink.table.types.logical.TimestampType; +import org.apache.flink.types.Row; +import org.apache.flink.util.Preconditions; + +/** + * Converts an Avro schema into Flink's type information. It uses {@link RowTypeInfo} for + * representing objects and converts Avro types into types that are compatible with Flink's Table & + * SQL API. + * + * <p>Note: Changes in this class need to be kept in sync with the corresponding runtime classes + * {@link AvroRowDataDeserializationSchema} and {@link AvroRowDataSerializationSchema}. Review Comment: Everything under `org/apache/flink/formats/` is a direct copy from the Flink codebase. I investigated `RowDataToAvroGenericRecordConverter` and confirmed that it truncates nanoseconds to milliseconds due to Flink's internal converter. Flink's internal `RowDataToAvroConverters` has no concept of nanoseconds for Avro conversions (it defaults to `TimestampData.getMillisecond()`). Therefore, when the `RowDataToAvroGenericRecordConverter` class reads Flink's internal `TimestampData` object, it forcefully strips the nanoseconds off before packing it into the Avro `GenericRecord`. Because the `RowDataToAvroConverters` class is used to emit Avro `GenericRecord`s directly into Flink's DataStream API (e.g., `AvroGenericRecordReaderFunction`), anyone using this function to automatically map Iceberg rows into Avro objects over the stream will unfortunately lose strict nanosecond precision. To natively fix this, we had two options: rewrite custom logic to avoid Flink's `RowDataToAvroConverters` entirely, or patch Flink. I chose to patch the Flink source code. However, for already published versions, I cannot modify their source code. To address this, I applied Classloader Manipulation for the existing Flink version by copying the actual source code and adding nanosecond support to these classes, rather than reimplementing things in our codebase. -- 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] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
