mxm commented on code in PR #15475:
URL: https://github.com/apache/iceberg/pull/15475#discussion_r2878517084


##########
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:
   +1 for highlighting the changes here. Could we add comments in the relevant 
sections and add a class-level JavaDoc?
   
   @talatuyarer Do you think we could open a PR against the Flink repository 
with the timestamp changes for RowDataAvroConverter and related classes?



##########
flink/v2.1/flink/src/main/java/org/apache/flink/formats/avro/RowDataToAvroConverters.java:
##########
@@ -0,0 +1,382 @@
+/*
+ * 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;
+
+import static 
org.apache.flink.formats.avro.typeutils.AvroSchemaConverter.extractValueTypeToAvroMap;
+
+import java.io.Serializable;
+import java.nio.ByteBuffer;
+import java.time.ZoneOffset;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.avro.util.Utf8;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.MapData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.TimestampData;
+import org.apache.flink.table.types.logical.ArrayType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.util.CollectionUtil;
+
+/** Tool class used to convert from {@link RowData} to Avro {@link 
GenericRecord}. */
+@Internal
+public class RowDataToAvroConverters {
+
+  // 
--------------------------------------------------------------------------------
+  // Runtime Converters
+  // 
--------------------------------------------------------------------------------
+
+  /**
+   * Runtime converter that converts objects of Flink Table & SQL internal 
data structures to
+   * corresponding Avro data structures.
+   */
+  @FunctionalInterface
+  public interface RowDataToAvroConverter extends Serializable {
+    Object convert(Schema schema, Object object);
+  }
+
+  // 
--------------------------------------------------------------------------------
+  // IMPORTANT! We use anonymous classes instead of lambdas for a reason here. 
It is
+  // necessary because the maven shade plugin cannot relocate classes in
+  // SerializedLambdas (MSHADE-260). On the other hand we want to relocate 
Avro for
+  // sql-client uber jars.
+  // 
--------------------------------------------------------------------------------
+
+  /**
+   * Creates a runtime converter according to the given logical type that 
converts objects of Flink
+   * Table & SQL internal data structures to corresponding Avro data 
structures.
+   */
+  public static RowDataToAvroConverter createConverter(LogicalType type) {
+    return createConverter(type, true);
+  }
+
+  public static RowDataToAvroConverter createConverter(
+      LogicalType type, boolean legacyTimestampMapping) {
+    final RowDataToAvroConverter converter;
+    switch (type.getTypeRoot()) {
+      case NULL:
+        converter =
+            new RowDataToAvroConverter() {
+              private static final long serialVersionUID = 1L;
+
+              @Override
+              public Object convert(Schema schema, Object object) {
+                return null;
+              }
+            };
+        break;
+      case TINYINT:
+        converter =
+            new RowDataToAvroConverter() {
+              private static final long serialVersionUID = 1L;
+
+              @Override
+              public Object convert(Schema schema, Object object) {
+                return ((Byte) object).intValue();
+              }
+            };
+        break;
+      case SMALLINT:
+        converter =
+            new RowDataToAvroConverter() {
+              private static final long serialVersionUID = 1L;
+
+              @Override
+              public Object convert(Schema schema, Object object) {
+                return ((Short) object).intValue();
+              }
+            };
+        break;
+      case BOOLEAN: // boolean
+      case INTEGER: // int
+      case INTERVAL_YEAR_MONTH: // long
+      case BIGINT: // long
+      case INTERVAL_DAY_TIME: // long
+      case FLOAT: // float
+      case DOUBLE: // double
+      case TIME_WITHOUT_TIME_ZONE: // int
+      case DATE: // int
+        converter =
+            new RowDataToAvroConverter() {
+              private static final long serialVersionUID = 1L;
+
+              @Override
+              public Object convert(Schema schema, Object object) {
+                return object;
+              }
+            };
+        break;
+      case CHAR:
+      case VARCHAR:
+        converter =
+            new RowDataToAvroConverter() {
+              private static final long serialVersionUID = 1L;
+
+              @Override
+              public Object convert(Schema schema, Object object) {
+                return new Utf8(object.toString());
+              }
+            };
+        break;
+      case BINARY:
+      case VARBINARY:
+        converter =
+            new RowDataToAvroConverter() {
+              private static final long serialVersionUID = 1L;
+
+              @Override
+              public Object convert(Schema schema, Object object) {
+                return ByteBuffer.wrap((byte[]) object);
+              }
+            };
+        break;
+      case TIMESTAMP_WITHOUT_TIME_ZONE:
+        final int tzPrecision;
+        if (type instanceof 
org.apache.flink.table.types.logical.TimestampType) {
+          tzPrecision = ((org.apache.flink.table.types.logical.TimestampType) 
type).getPrecision();
+        } else {
+          tzPrecision = 3;
+        }
+        if (legacyTimestampMapping) {
+          converter =
+              new RowDataToAvroConverter() {
+                private static final long serialVersionUID = 1L;
+
+                @Override
+                public Object convert(Schema schema, Object object) {
+                  TimestampData timestampData = (TimestampData) object;
+                  if (tzPrecision <= 3) {
+                    return timestampData.getMillisecond();
+                  } else if (tzPrecision <= 6) {
+                    return timestampData.getMillisecond() * 1000L
+                        + timestampData.getNanoOfMillisecond() / 1000;
+                  } else {
+                    return timestampData.getMillisecond() * 1_000_000L
+                        + timestampData.getNanoOfMillisecond();
+                  }
+                }
+              };
+        } else {
+          converter =
+              new RowDataToAvroConverter() {
+                private static final long serialVersionUID = 1L;
+
+                @Override
+                public Object convert(Schema schema, Object object) {
+                  TimestampData timestampData = (TimestampData) object;
+                  java.time.Instant instant =
+                      
timestampData.toLocalDateTime().toInstant(ZoneOffset.UTC);
+                  if (tzPrecision <= 3) {
+                    return instant.toEpochMilli();
+                  } else if (tzPrecision <= 6) {
+                    return instant.getEpochSecond() * 1_000_000L + 
instant.getNano() / 1000;
+                  } else {
+                    return instant.getEpochSecond() * 1_000_000_000L + 
instant.getNano();
+                  }
+                }
+              };

Review Comment:
   Same here, can be extracted, duplicate in lines 215-231.



##########
flink/v2.1/flink/src/main/java/org/apache/flink/formats/avro/RowDataToAvroConverters.java:
##########
@@ -0,0 +1,382 @@
+/*
+ * 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;
+
+import static 
org.apache.flink.formats.avro.typeutils.AvroSchemaConverter.extractValueTypeToAvroMap;
+
+import java.io.Serializable;
+import java.nio.ByteBuffer;
+import java.time.ZoneOffset;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.avro.util.Utf8;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.MapData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.TimestampData;
+import org.apache.flink.table.types.logical.ArrayType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.util.CollectionUtil;
+
+/** Tool class used to convert from {@link RowData} to Avro {@link 
GenericRecord}. */
+@Internal
+public class RowDataToAvroConverters {
+
+  // 
--------------------------------------------------------------------------------
+  // Runtime Converters
+  // 
--------------------------------------------------------------------------------
+
+  /**
+   * Runtime converter that converts objects of Flink Table & SQL internal 
data structures to
+   * corresponding Avro data structures.
+   */
+  @FunctionalInterface
+  public interface RowDataToAvroConverter extends Serializable {
+    Object convert(Schema schema, Object object);
+  }
+
+  // 
--------------------------------------------------------------------------------
+  // IMPORTANT! We use anonymous classes instead of lambdas for a reason here. 
It is
+  // necessary because the maven shade plugin cannot relocate classes in
+  // SerializedLambdas (MSHADE-260). On the other hand we want to relocate 
Avro for
+  // sql-client uber jars.
+  // 
--------------------------------------------------------------------------------
+
+  /**
+   * Creates a runtime converter according to the given logical type that 
converts objects of Flink
+   * Table & SQL internal data structures to corresponding Avro data 
structures.
+   */
+  public static RowDataToAvroConverter createConverter(LogicalType type) {
+    return createConverter(type, true);
+  }
+
+  public static RowDataToAvroConverter createConverter(
+      LogicalType type, boolean legacyTimestampMapping) {
+    final RowDataToAvroConverter converter;
+    switch (type.getTypeRoot()) {
+      case NULL:
+        converter =
+            new RowDataToAvroConverter() {
+              private static final long serialVersionUID = 1L;
+
+              @Override
+              public Object convert(Schema schema, Object object) {
+                return null;
+              }
+            };
+        break;
+      case TINYINT:
+        converter =
+            new RowDataToAvroConverter() {
+              private static final long serialVersionUID = 1L;
+
+              @Override
+              public Object convert(Schema schema, Object object) {
+                return ((Byte) object).intValue();
+              }
+            };
+        break;
+      case SMALLINT:
+        converter =
+            new RowDataToAvroConverter() {
+              private static final long serialVersionUID = 1L;
+
+              @Override
+              public Object convert(Schema schema, Object object) {
+                return ((Short) object).intValue();
+              }
+            };
+        break;
+      case BOOLEAN: // boolean
+      case INTEGER: // int
+      case INTERVAL_YEAR_MONTH: // long
+      case BIGINT: // long
+      case INTERVAL_DAY_TIME: // long
+      case FLOAT: // float
+      case DOUBLE: // double
+      case TIME_WITHOUT_TIME_ZONE: // int
+      case DATE: // int
+        converter =
+            new RowDataToAvroConverter() {
+              private static final long serialVersionUID = 1L;
+
+              @Override
+              public Object convert(Schema schema, Object object) {
+                return object;
+              }
+            };
+        break;
+      case CHAR:
+      case VARCHAR:
+        converter =
+            new RowDataToAvroConverter() {
+              private static final long serialVersionUID = 1L;
+
+              @Override
+              public Object convert(Schema schema, Object object) {
+                return new Utf8(object.toString());
+              }
+            };
+        break;
+      case BINARY:
+      case VARBINARY:
+        converter =
+            new RowDataToAvroConverter() {
+              private static final long serialVersionUID = 1L;
+
+              @Override
+              public Object convert(Schema schema, Object object) {
+                return ByteBuffer.wrap((byte[]) object);
+              }
+            };
+        break;
+      case TIMESTAMP_WITHOUT_TIME_ZONE:
+        final int tzPrecision;
+        if (type instanceof 
org.apache.flink.table.types.logical.TimestampType) {
+          tzPrecision = ((org.apache.flink.table.types.logical.TimestampType) 
type).getPrecision();
+        } else {
+          tzPrecision = 3;
+        }
+        if (legacyTimestampMapping) {
+          converter =
+              new RowDataToAvroConverter() {
+                private static final long serialVersionUID = 1L;
+
+                @Override
+                public Object convert(Schema schema, Object object) {
+                  TimestampData timestampData = (TimestampData) object;
+                  if (tzPrecision <= 3) {
+                    return timestampData.getMillisecond();
+                  } else if (tzPrecision <= 6) {
+                    return timestampData.getMillisecond() * 1000L
+                        + timestampData.getNanoOfMillisecond() / 1000;
+                  } else {
+                    return timestampData.getMillisecond() * 1_000_000L
+                        + timestampData.getNanoOfMillisecond();
+                  }
+                }
+              };

Review Comment:
   Can we extract this into a method? This is identical to lines 214-232.



-- 
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]

Reply via email to