You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by GitBox <gi...@apache.org> on 2020/07/29 17:52:22 UTC

[GitHub] [iceberg] chenjunjiedada opened a new pull request #1266: Flink: update parquet reader with schema visitor

chenjunjiedada opened a new pull request #1266:
URL: https://github.com/apache/iceberg/pull/1266


   This is sub PR of #1237, it changes the current FlinkParquetReader to use a schema visitor.


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] chenjunjiedada commented on a change in pull request #1266: Flink: update parquet reader with schema visitor

Posted by GitBox <gi...@apache.org>.
chenjunjiedada commented on a change in pull request #1266:
URL: https://github.com/apache/iceberg/pull/1266#discussion_r464453352



##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java
##########
@@ -19,64 +19,719 @@
 
 package org.apache.iceberg.flink.data;
 
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.time.Instant;
 import java.util.List;
-import org.apache.flink.types.Row;
+import java.util.Map;
+import org.apache.commons.lang3.ArrayUtils;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.MapData;
+import org.apache.flink.table.data.RawValueData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
 import org.apache.iceberg.Schema;
-import org.apache.iceberg.data.parquet.BaseParquetReaders;
+import org.apache.iceberg.parquet.ParquetSchemaUtil;
 import org.apache.iceberg.parquet.ParquetValueReader;
 import org.apache.iceberg.parquet.ParquetValueReaders;
+import org.apache.iceberg.parquet.TypeWithSchemaVisitor;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
 import org.apache.iceberg.types.Types;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.io.api.Binary;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.LogicalTypeAnnotation.DecimalLogicalTypeAnnotation;
 import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
 import org.apache.parquet.schema.Type;
 
-public class FlinkParquetReaders extends BaseParquetReaders<Row> {
+class FlinkParquetReaders {
+  private FlinkParquetReaders() {
+  }
 
-  private static final FlinkParquetReaders INSTANCE = new FlinkParquetReaders();
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema, MessageType fileSchema) {
+    return buildReader(expectedSchema, fileSchema, ImmutableMap.of());
+  }
 
-  private FlinkParquetReaders() {
+  @SuppressWarnings("unchecked")
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema,
+                                                        MessageType fileSchema,
+                                                        Map<Integer, ?> idToConstant) {
+    ReadBuilder builder = new ReadBuilder(fileSchema, idToConstant);
+    if (ParquetSchemaUtil.hasIds(fileSchema)) {
+      return (ParquetValueReader<RowData>)
+          TypeWithSchemaVisitor.visit(expectedSchema.asStruct(), fileSchema, builder);
+    } else {
+      return (ParquetValueReader<RowData>)
+          TypeWithSchemaVisitor.visit(expectedSchema.asStruct(), fileSchema,
+              new FallbackReadBuilder(builder));
+    }
+  }
+
+  private static class FallbackReadBuilder extends TypeWithSchemaVisitor<ParquetValueReader<?>> {
+    private MessageType type;
+    private final TypeWithSchemaVisitor<ParquetValueReader<?>> builder;
+
+    FallbackReadBuilder(TypeWithSchemaVisitor<ParquetValueReader<?>> builder) {
+      this.builder = builder;
+    }
+
+    @Override
+    public ParquetValueReader<?> message(Types.StructType expected, MessageType message,
+                                         List<ParquetValueReader<?>> fieldReaders) {
+      // the top level matches by ID, but the remaining IDs are missing
+      this.type = message;
+      return builder.struct(expected, message, fieldReaders);
+    }
+
+    @Override
+    public ParquetValueReader<?> struct(Types.StructType ignored, GroupType struct,
+                                        List<ParquetValueReader<?>> fieldReaders) {
+      // the expected struct is ignored because nested fields are never found when the IDs are missing
+      List<ParquetValueReader<?>> newFields = Lists.newArrayListWithExpectedSize(
+          fieldReaders.size());
+      List<Type> types = Lists.newArrayListWithExpectedSize(fieldReaders.size());
+      List<Type> fields = struct.getFields();
+      for (int i = 0; i < fields.size(); i += 1) {
+        Type fieldType = fields.get(i);
+        int fieldD = type.getMaxDefinitionLevel(path(fieldType.getName())) - 1;
+        newFields.add(ParquetValueReaders.option(fieldType, fieldD, fieldReaders.get(i)));
+        types.add(fieldType);
+      }
+
+      return new RowDataReader(types, newFields);
+    }
+  }
+
+  private static class ReadBuilder extends TypeWithSchemaVisitor<ParquetValueReader<?>> {

Review comment:
       Sound good to me, we can take a look at `WriterBuilder` as well after all these readers and writers get in.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1266: Flink: update parquet reader with schema visitor

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1266:
URL: https://github.com/apache/iceberg/pull/1266#discussion_r473473950



##########
File path: flink/src/test/java/org/apache/iceberg/flink/data/TestHelpers.java
##########
@@ -0,0 +1,198 @@
+/*
+ * 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.iceberg.flink.data;
+
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.OffsetDateTime;
+import java.time.ZoneOffset;
+import java.time.temporal.ChronoUnit;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import java.util.function.Supplier;
+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.MapType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+
+public class TestHelpers {
+  private TestHelpers() {}
+
+  private static final OffsetDateTime EPOCH = Instant.ofEpochMilli(0L).atOffset(ZoneOffset.UTC);
+  private static final LocalDate EPOCH_DAY = EPOCH.toLocalDate();
+
+  public static void assertRowData(Type type, LogicalType rowType, Record expectedRecord, RowData actualRowData) {
+    if (expectedRecord == null && actualRowData == null) {
+      return;
+    }
+
+    Assert.assertTrue("expected Record and actual RowData should be both null or not null",
+        expectedRecord != null && actualRowData != null);
+
+    List<Type> types = Lists.newArrayList();
+    for (Types.NestedField field : type.asStructType().fields()) {
+      types.add(field.type());
+    }
+
+    for (int i = 0; i < types.size(); i += 1) {
+      if (expectedRecord.get(i) == null) {
+        Assert.assertTrue(actualRowData.isNullAt(i));
+        continue;
+      }
+
+      Object expected = expectedRecord.get(i);
+      LogicalType logicalType = ((RowType) rowType).getTypeAt(i);
+
+      final int fieldPos = i;
+      assertEquals(types.get(i), logicalType, expected,
+          () -> RowData.createFieldGetter(logicalType, fieldPos).getFieldOrNull(actualRowData));
+    }
+  }
+
+  private static void assertEquals(Type type, LogicalType logicalType, Object expected, Supplier<Object> supplier) {

Review comment:
       Sorry, I meant to remove the supplier entirely. Why is it necessary to pass `Supplier<Object>` and not just `Object` to this method? As far as I can tell, `get` is called on the supplier immediately in all cases.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1266: Flink: update parquet reader with schema visitor

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1266:
URL: https://github.com/apache/iceberg/pull/1266#discussion_r469418285



##########
File path: flink/src/test/java/org/apache/iceberg/flink/data/TestHelpers.java
##########
@@ -0,0 +1,198 @@
+/*
+ * 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.iceberg.flink.data;
+
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.OffsetDateTime;
+import java.time.ZoneOffset;
+import java.time.temporal.ChronoUnit;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import java.util.function.Supplier;
+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.MapType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+
+public class TestHelpers {
+  private TestHelpers() {}
+
+  private static final OffsetDateTime EPOCH = Instant.ofEpochMilli(0L).atOffset(ZoneOffset.UTC);
+  private static final LocalDate EPOCH_DAY = EPOCH.toLocalDate();
+
+  public static void assertRowData(Type type, LogicalType rowType, Record expectedRecord, RowData actualRowData) {
+    if (expectedRecord == null && actualRowData == null) {
+      return;
+    }
+
+    Assert.assertTrue("expected Record and actual RowData should be both null or not null",
+        expectedRecord != null && actualRowData != null);
+
+    List<Type> types = Lists.newArrayList();
+    for (Types.NestedField field : type.asStructType().fields()) {
+      types.add(field.type());
+    }
+
+    for (int i = 0; i < types.size(); i += 1) {
+      if (expectedRecord.get(i) == null) {
+        Assert.assertTrue(actualRowData.isNullAt(i));
+        continue;
+      }
+
+      Object expected = expectedRecord.get(i);
+      LogicalType logicalType = ((RowType) rowType).getTypeAt(i);
+
+      final int fieldPos = i;
+      assertEquals(types.get(i), logicalType, expected,
+          () -> RowData.createFieldGetter(logicalType, fieldPos).getFieldOrNull(actualRowData));
+    }
+  }
+
+  private static void assertEquals(Type type, LogicalType logicalType, Object expected, Supplier<Object> supplier) {
+    switch (type.typeId()) {
+      case BOOLEAN:
+        Assert.assertEquals("boolean value should be equal", expected, supplier.get());
+        break;
+      case INTEGER:
+        Assert.assertEquals("int value should be equal", expected, supplier.get());
+        break;
+      case LONG:
+        Assert.assertEquals("long value should be equal", expected, supplier.get());
+        break;
+      case FLOAT:
+        Assert.assertEquals("float value should be equal", expected, supplier.get());
+        break;
+      case DOUBLE:
+        Assert.assertEquals("double value should be equal", expected, supplier.get());
+        break;
+      case STRING:
+        Assert.assertTrue("Should expect a CharSequence", expected instanceof CharSequence);
+        Assert.assertEquals("string should be equal",
+            String.valueOf(expected), supplier.get().toString());
+        break;
+      case DATE:
+        Assert.assertTrue("Should expect a Date", expected instanceof LocalDate);
+        LocalDate date = ChronoUnit.DAYS.addTo(EPOCH_DAY, (int) supplier.get());

Review comment:
       Can this use the utility methods to convert instead?




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1266: Flink: update parquet reader with schema visitor

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1266:
URL: https://github.com/apache/iceberg/pull/1266#discussion_r463768261



##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java
##########
@@ -19,64 +19,719 @@
 
 package org.apache.iceberg.flink.data;
 
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.time.Instant;
 import java.util.List;
-import org.apache.flink.types.Row;
+import java.util.Map;
+import org.apache.commons.lang3.ArrayUtils;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.MapData;
+import org.apache.flink.table.data.RawValueData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
 import org.apache.iceberg.Schema;
-import org.apache.iceberg.data.parquet.BaseParquetReaders;
+import org.apache.iceberg.parquet.ParquetSchemaUtil;
 import org.apache.iceberg.parquet.ParquetValueReader;
 import org.apache.iceberg.parquet.ParquetValueReaders;
+import org.apache.iceberg.parquet.TypeWithSchemaVisitor;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
 import org.apache.iceberg.types.Types;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.io.api.Binary;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.LogicalTypeAnnotation.DecimalLogicalTypeAnnotation;
 import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
 import org.apache.parquet.schema.Type;
 
-public class FlinkParquetReaders extends BaseParquetReaders<Row> {
+class FlinkParquetReaders {
+  private FlinkParquetReaders() {
+  }
 
-  private static final FlinkParquetReaders INSTANCE = new FlinkParquetReaders();
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema, MessageType fileSchema) {
+    return buildReader(expectedSchema, fileSchema, ImmutableMap.of());
+  }
 
-  private FlinkParquetReaders() {
+  @SuppressWarnings("unchecked")
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema,
+                                                        MessageType fileSchema,
+                                                        Map<Integer, ?> idToConstant) {
+    ReadBuilder builder = new ReadBuilder(fileSchema, idToConstant);
+    if (ParquetSchemaUtil.hasIds(fileSchema)) {
+      return (ParquetValueReader<RowData>)
+          TypeWithSchemaVisitor.visit(expectedSchema.asStruct(), fileSchema, builder);
+    } else {
+      return (ParquetValueReader<RowData>)
+          TypeWithSchemaVisitor.visit(expectedSchema.asStruct(), fileSchema,
+              new FallbackReadBuilder(builder));
+    }
+  }
+
+  private static class FallbackReadBuilder extends TypeWithSchemaVisitor<ParquetValueReader<?>> {
+    private MessageType type;
+    private final TypeWithSchemaVisitor<ParquetValueReader<?>> builder;
+
+    FallbackReadBuilder(TypeWithSchemaVisitor<ParquetValueReader<?>> builder) {
+      this.builder = builder;
+    }
+
+    @Override
+    public ParquetValueReader<?> message(Types.StructType expected, MessageType message,
+                                         List<ParquetValueReader<?>> fieldReaders) {
+      // the top level matches by ID, but the remaining IDs are missing
+      this.type = message;
+      return builder.struct(expected, message, fieldReaders);
+    }
+
+    @Override
+    public ParquetValueReader<?> struct(Types.StructType ignored, GroupType struct,
+                                        List<ParquetValueReader<?>> fieldReaders) {
+      // the expected struct is ignored because nested fields are never found when the IDs are missing
+      List<ParquetValueReader<?>> newFields = Lists.newArrayListWithExpectedSize(
+          fieldReaders.size());
+      List<Type> types = Lists.newArrayListWithExpectedSize(fieldReaders.size());
+      List<Type> fields = struct.getFields();
+      for (int i = 0; i < fields.size(); i += 1) {
+        Type fieldType = fields.get(i);
+        int fieldD = type.getMaxDefinitionLevel(path(fieldType.getName())) - 1;
+        newFields.add(ParquetValueReaders.option(fieldType, fieldD, fieldReaders.get(i)));
+        types.add(fieldType);
+      }
+
+      return new RowDataReader(types, newFields);
+    }
+  }
+
+  private static class ReadBuilder extends TypeWithSchemaVisitor<ParquetValueReader<?>> {

Review comment:
       Yes, I think we should refactor this but I'd prefer to do it separately. I'm okay continuing with the current approach in this PR and refactoring later or in parallel.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] chenjunjiedada commented on a change in pull request #1266: Flink: update parquet reader with schema visitor

Posted by GitBox <gi...@apache.org>.
chenjunjiedada commented on a change in pull request #1266:
URL: https://github.com/apache/iceberg/pull/1266#discussion_r465084047



##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java
##########
@@ -19,64 +19,719 @@
 
 package org.apache.iceberg.flink.data;
 
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.time.Instant;
 import java.util.List;
-import org.apache.flink.types.Row;
+import java.util.Map;
+import org.apache.commons.lang3.ArrayUtils;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.MapData;
+import org.apache.flink.table.data.RawValueData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
 import org.apache.iceberg.Schema;
-import org.apache.iceberg.data.parquet.BaseParquetReaders;
+import org.apache.iceberg.parquet.ParquetSchemaUtil;
 import org.apache.iceberg.parquet.ParquetValueReader;
 import org.apache.iceberg.parquet.ParquetValueReaders;
+import org.apache.iceberg.parquet.TypeWithSchemaVisitor;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
 import org.apache.iceberg.types.Types;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.io.api.Binary;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.LogicalTypeAnnotation.DecimalLogicalTypeAnnotation;
 import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
 import org.apache.parquet.schema.Type;
 
-public class FlinkParquetReaders extends BaseParquetReaders<Row> {
+class FlinkParquetReaders {
+  private FlinkParquetReaders() {
+  }
 
-  private static final FlinkParquetReaders INSTANCE = new FlinkParquetReaders();
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema, MessageType fileSchema) {
+    return buildReader(expectedSchema, fileSchema, ImmutableMap.of());
+  }
 
-  private FlinkParquetReaders() {
+  @SuppressWarnings("unchecked")
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema,
+                                                        MessageType fileSchema,
+                                                        Map<Integer, ?> idToConstant) {
+    ReadBuilder builder = new ReadBuilder(fileSchema, idToConstant);
+    if (ParquetSchemaUtil.hasIds(fileSchema)) {
+      return (ParquetValueReader<RowData>)
+          TypeWithSchemaVisitor.visit(expectedSchema.asStruct(), fileSchema, builder);
+    } else {
+      return (ParquetValueReader<RowData>)
+          TypeWithSchemaVisitor.visit(expectedSchema.asStruct(), fileSchema,
+              new FallbackReadBuilder(builder));
+    }
+  }
+
+  private static class FallbackReadBuilder extends TypeWithSchemaVisitor<ParquetValueReader<?>> {
+    private MessageType type;
+    private final TypeWithSchemaVisitor<ParquetValueReader<?>> builder;
+
+    FallbackReadBuilder(TypeWithSchemaVisitor<ParquetValueReader<?>> builder) {
+      this.builder = builder;
+    }
+
+    @Override
+    public ParquetValueReader<?> message(Types.StructType expected, MessageType message,
+                                         List<ParquetValueReader<?>> fieldReaders) {
+      // the top level matches by ID, but the remaining IDs are missing
+      this.type = message;
+      return builder.struct(expected, message, fieldReaders);
+    }
+
+    @Override
+    public ParquetValueReader<?> struct(Types.StructType ignored, GroupType struct,
+                                        List<ParquetValueReader<?>> fieldReaders) {
+      // the expected struct is ignored because nested fields are never found when the IDs are missing
+      List<ParquetValueReader<?>> newFields = Lists.newArrayListWithExpectedSize(
+          fieldReaders.size());
+      List<Type> types = Lists.newArrayListWithExpectedSize(fieldReaders.size());
+      List<Type> fields = struct.getFields();
+      for (int i = 0; i < fields.size(); i += 1) {
+        Type fieldType = fields.get(i);
+        int fieldD = type.getMaxDefinitionLevel(path(fieldType.getName())) - 1;
+        newFields.add(ParquetValueReaders.option(fieldType, fieldD, fieldReaders.get(i)));
+        types.add(fieldType);
+      }
+
+      return new RowDataReader(types, newFields);
+    }
+  }
+
+  private static class ReadBuilder extends TypeWithSchemaVisitor<ParquetValueReader<?>> {

Review comment:
       Created https://github.com/apache/iceberg/issues/1294.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] chenjunjiedada commented on a change in pull request #1266: Flink: update parquet reader with schema visitor

Posted by GitBox <gi...@apache.org>.
chenjunjiedada commented on a change in pull request #1266:
URL: https://github.com/apache/iceberg/pull/1266#discussion_r465086485



##########
File path: flink/src/test/java/org/apache/iceberg/flink/data/RandomData.java
##########
@@ -88,20 +93,46 @@ public Row next() {
     };
   }
 
+  private static Iterable<Record> generateIcebergGenerics(Schema schema, int numRecords,

Review comment:
       Make sense to mem moved all `Record` generators to RandomGenericData.

##########
File path: flink/src/test/java/org/apache/iceberg/flink/data/RandomData.java
##########
@@ -88,20 +93,46 @@ public Row next() {
     };
   }
 
+  private static Iterable<Record> generateIcebergGenerics(Schema schema, int numRecords,
+                                                   Supplier<RandomGenericData.RandomDataGenerator<Record>> supplier) {
+    return () -> new Iterator<Record>() {
+      private final RandomGenericData.RandomDataGenerator<Record> generator = supplier.get();
+      private int count = 0;
+
+      @Override
+      public boolean hasNext() {
+        return count < numRecords;
+      }
+
+      @Override
+      public Record next() {
+        if (!hasNext()) {
+          throw new NoSuchElementException();
+        }
+        ++count;
+        return (Record) TypeUtil.visit(schema, generator);
+      }
+    };
+  }
+
+
+  public static Iterable<Record> generateRecords(Schema schema, int numRecords, long seed) {

Review comment:
       Done.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi commented on a change in pull request #1266: Flink: update parquet reader with schema visitor

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on a change in pull request #1266:
URL: https://github.com/apache/iceberg/pull/1266#discussion_r466835571



##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java
##########
@@ -19,64 +19,701 @@
 
 package org.apache.iceberg.flink.data;
 
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.time.Instant;
+import java.time.ZoneOffset;
 import java.util.List;
-import org.apache.flink.types.Row;
+import java.util.Map;
+import org.apache.commons.lang3.ArrayUtils;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.MapData;
+import org.apache.flink.table.data.RawValueData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
 import org.apache.iceberg.Schema;
-import org.apache.iceberg.data.parquet.BaseParquetReaders;
 import org.apache.iceberg.parquet.ParquetValueReader;
 import org.apache.iceberg.parquet.ParquetValueReaders;
+import org.apache.iceberg.parquet.TypeWithSchemaVisitor;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
 import org.apache.iceberg.types.Types;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.io.api.Binary;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.LogicalTypeAnnotation.DecimalLogicalTypeAnnotation;
 import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
 import org.apache.parquet.schema.Type;
 
-public class FlinkParquetReaders extends BaseParquetReaders<Row> {
+class FlinkParquetReaders {
+  private FlinkParquetReaders() {
+  }
 
-  private static final FlinkParquetReaders INSTANCE = new FlinkParquetReaders();
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema, MessageType fileSchema) {
+    return buildReader(expectedSchema, fileSchema, ImmutableMap.of());
+  }
 
-  private FlinkParquetReaders() {
+  @SuppressWarnings("unchecked")
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema,
+                                                        MessageType fileSchema,
+                                                        Map<Integer, ?> idToConstant) {
+    return (ParquetValueReader<RowData>) TypeWithSchemaVisitor.visit(expectedSchema.asStruct(), fileSchema,
+        new ReadBuilder(fileSchema, idToConstant)
+    );
+  }
+
+  private static class ReadBuilder extends TypeWithSchemaVisitor<ParquetValueReader<?>> {
+    private final MessageType type;
+    private final Map<Integer, ?> idToConstant;
+
+    ReadBuilder(MessageType type, Map<Integer, ?> idToConstant) {
+      this.type = type;
+      this.idToConstant = idToConstant;
+    }
+
+    @Override
+    public ParquetValueReader<RowData> message(Types.StructType expected, MessageType message,
+                                               List<ParquetValueReader<?>> fieldReaders) {
+      return struct(expected, message.asGroupType(), fieldReaders);
+    }
+
+    @Override
+    public ParquetValueReader<RowData> struct(Types.StructType expected, GroupType struct,
+                                              List<ParquetValueReader<?>> fieldReaders) {
+      // match the expected struct's order
+      Map<Integer, ParquetValueReader<?>> readersById = Maps.newHashMap();
+      Map<Integer, Type> typesById = Maps.newHashMap();
+      List<Type> fields = struct.getFields();
+      for (int i = 0; i < fields.size(); i += 1) {
+        Type fieldType = fields.get(i);
+        int fieldD = type.getMaxDefinitionLevel(path(fieldType.getName())) - 1;
+        if (fieldType.getId() != null) {
+          int id = fieldType.getId().intValue();
+          readersById.put(id, ParquetValueReaders.option(fieldType, fieldD, fieldReaders.get(i)));
+          typesById.put(id, fieldType);
+        }
+      }
+
+      List<Types.NestedField> expectedFields = expected != null ?
+          expected.fields() : ImmutableList.of();
+      List<ParquetValueReader<?>> reorderedFields = Lists.newArrayListWithExpectedSize(
+          expectedFields.size());
+      List<Type> types = Lists.newArrayListWithExpectedSize(expectedFields.size());
+      for (Types.NestedField field : expectedFields) {
+        int id = field.fieldId();
+        if (idToConstant.containsKey(id)) {
+          // containsKey is used because the constant may be null
+          reorderedFields.add(ParquetValueReaders.constant(idToConstant.get(id)));
+          types.add(null);
+        } else {
+          ParquetValueReader<?> reader = readersById.get(id);
+          if (reader != null) {
+            reorderedFields.add(reader);
+            types.add(typesById.get(id));
+          } else {
+            reorderedFields.add(ParquetValueReaders.nulls());
+            types.add(null);
+          }
+        }
+      }
+
+      return new RowDataReader(types, reorderedFields);
+    }
+
+    @Override
+    public ParquetValueReader<?> list(Types.ListType expectedList, GroupType array,
+                                      ParquetValueReader<?> elementReader) {
+      GroupType repeated = array.getFields().get(0).asGroupType();
+      String[] repeatedPath = currentPath();
+
+      int repeatedD = type.getMaxDefinitionLevel(repeatedPath) - 1;
+      int repeatedR = type.getMaxRepetitionLevel(repeatedPath) - 1;
+
+      Type elementType = repeated.getType(0);
+      int elementD = type.getMaxDefinitionLevel(path(elementType.getName())) - 1;
+
+      return new ArrayReader<>(repeatedD, repeatedR, ParquetValueReaders.option(elementType, elementD, elementReader));
+    }
+
+    @Override
+    public ParquetValueReader<?> map(Types.MapType expectedMap, GroupType map,
+                                     ParquetValueReader<?> keyReader,
+                                     ParquetValueReader<?> valueReader) {
+      GroupType repeatedKeyValue = map.getFields().get(0).asGroupType();
+      String[] repeatedPath = currentPath();
+
+      int repeatedD = type.getMaxDefinitionLevel(repeatedPath) - 1;
+      int repeatedR = type.getMaxRepetitionLevel(repeatedPath) - 1;
+
+      Type keyType = repeatedKeyValue.getType(0);
+      int keyD = type.getMaxDefinitionLevel(path(keyType.getName())) - 1;
+      Type valueType = repeatedKeyValue.getType(1);
+      int valueD = type.getMaxDefinitionLevel(path(valueType.getName())) - 1;
+
+      return new MapReader<>(repeatedD, repeatedR,
+          ParquetValueReaders.option(keyType, keyD, keyReader),
+          ParquetValueReaders.option(valueType, valueD, valueReader));
+    }
+
+    @Override
+    public ParquetValueReader<?> primitive(org.apache.iceberg.types.Type.PrimitiveType expected,
+                                           PrimitiveType primitive) {
+      ColumnDescriptor desc = type.getColumnDescription(currentPath());
+
+      if (primitive.getOriginalType() != null) {
+        switch (primitive.getOriginalType()) {
+          case ENUM:
+          case JSON:
+          case UTF8:
+            return new StringReader(desc);
+          case INT_8:
+          case INT_16:
+          case INT_32:
+          case DATE:
+            if (expected != null && expected.typeId() == Types.LongType.get().typeId()) {
+              return new ParquetValueReaders.IntAsLongReader(desc);
+            } else {
+              return new ParquetValueReaders.UnboxedReader<>(desc);
+            }
+          case TIME_MICROS:
+            return new LossyMicrosToMillisTimeReader(desc);
+          case INT_64:
+            return new ParquetValueReaders.UnboxedReader<>(desc);
+          case TIMESTAMP_MICROS:
+            if (((Types.TimestampType) expected).shouldAdjustToUTC()) {
+              return new TimestampTzReader(desc);
+            } else {
+              return new TimestampReader(desc);
+            }
+          case DECIMAL:
+            DecimalLogicalTypeAnnotation decimal = (DecimalLogicalTypeAnnotation) primitive.getLogicalTypeAnnotation();
+            switch (primitive.getPrimitiveTypeName()) {
+              case BINARY:
+              case FIXED_LEN_BYTE_ARRAY:
+                return new BinaryDecimalReader(desc, decimal.getScale());
+              case INT64:
+                return new LongDecimalReader(desc, decimal.getPrecision(), decimal.getScale());
+              case INT32:
+                return new IntegerDecimalReader(desc, decimal.getPrecision(), decimal.getScale());
+              default:
+                throw new UnsupportedOperationException(
+                    "Unsupported base type for decimal: " + primitive.getPrimitiveTypeName());
+            }
+          case BSON:
+            return new ParquetValueReaders.ByteArrayReader(desc);
+          default:
+            throw new UnsupportedOperationException(
+                "Unsupported logical type: " + primitive.getOriginalType());
+        }
+      }
+
+      switch (primitive.getPrimitiveTypeName()) {
+        case FIXED_LEN_BYTE_ARRAY:
+        case BINARY:
+          return new ParquetValueReaders.ByteArrayReader(desc);
+        case INT32:
+          if (expected != null && expected.typeId() == org.apache.iceberg.types.Type.TypeID.LONG) {
+            return new ParquetValueReaders.IntAsLongReader(desc);
+          } else {
+            return new ParquetValueReaders.UnboxedReader<>(desc);
+          }
+        case FLOAT:
+          if (expected != null && expected.typeId() == org.apache.iceberg.types.Type.TypeID.DOUBLE) {
+            return new ParquetValueReaders.FloatAsDoubleReader(desc);
+          } else {
+            return new ParquetValueReaders.UnboxedReader<>(desc);
+          }
+        case BOOLEAN:
+        case INT64:
+        case DOUBLE:
+          return new ParquetValueReaders.UnboxedReader<>(desc);
+        default:
+          throw new UnsupportedOperationException("Unsupported type: " + primitive);
+      }
+    }
+  }
+
+  private static class BinaryDecimalReader extends ParquetValueReaders.PrimitiveReader<DecimalData> {
+    private final int scale;
+
+    BinaryDecimalReader(ColumnDescriptor desc, int scale) {
+      super(desc);
+      this.scale = scale;
+    }
+
+    @Override
+    public DecimalData read(DecimalData ignored) {
+      Binary binary = column.nextBinary();
+      BigDecimal bigDecimal = new BigDecimal(new BigInteger(binary.getBytes()), scale);
+      return DecimalData.fromBigDecimal(bigDecimal, bigDecimal.precision(), scale);

Review comment:
       @openinx is right, it should be same to `FlinkValueReaders.DecimalReader`.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] chenjunjiedada commented on a change in pull request #1266: Flink: update parquet reader with schema visitor

Posted by GitBox <gi...@apache.org>.
chenjunjiedada commented on a change in pull request #1266:
URL: https://github.com/apache/iceberg/pull/1266#discussion_r469700140



##########
File path: flink/src/test/java/org/apache/iceberg/flink/data/TestHelpers.java
##########
@@ -0,0 +1,198 @@
+/*
+ * 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.iceberg.flink.data;
+
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.OffsetDateTime;
+import java.time.ZoneOffset;
+import java.time.temporal.ChronoUnit;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import java.util.function.Supplier;
+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.MapType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+
+public class TestHelpers {
+  private TestHelpers() {}
+
+  private static final OffsetDateTime EPOCH = Instant.ofEpochMilli(0L).atOffset(ZoneOffset.UTC);
+  private static final LocalDate EPOCH_DAY = EPOCH.toLocalDate();
+
+  public static void assertRowData(Type type, LogicalType rowType, Record expectedRecord, RowData actualRowData) {
+    if (expectedRecord == null && actualRowData == null) {
+      return;
+    }
+
+    Assert.assertTrue("expected Record and actual RowData should be both null or not null",
+        expectedRecord != null && actualRowData != null);
+
+    List<Type> types = Lists.newArrayList();
+    for (Types.NestedField field : type.asStructType().fields()) {
+      types.add(field.type());
+    }
+
+    for (int i = 0; i < types.size(); i += 1) {
+      if (expectedRecord.get(i) == null) {
+        Assert.assertTrue(actualRowData.isNullAt(i));
+        continue;
+      }
+
+      Object expected = expectedRecord.get(i);
+      LogicalType logicalType = ((RowType) rowType).getTypeAt(i);
+
+      final int fieldPos = i;
+      assertEquals(types.get(i), logicalType, expected,
+          () -> RowData.createFieldGetter(logicalType, fieldPos).getFieldOrNull(actualRowData));
+    }
+  }
+
+  private static void assertEquals(Type type, LogicalType logicalType, Object expected, Supplier<Object> supplier) {
+    switch (type.typeId()) {
+      case BOOLEAN:
+        Assert.assertEquals("boolean value should be equal", expected, supplier.get());
+        break;
+      case INTEGER:
+        Assert.assertEquals("int value should be equal", expected, supplier.get());
+        break;
+      case LONG:
+        Assert.assertEquals("long value should be equal", expected, supplier.get());
+        break;
+      case FLOAT:
+        Assert.assertEquals("float value should be equal", expected, supplier.get());
+        break;
+      case DOUBLE:
+        Assert.assertEquals("double value should be equal", expected, supplier.get());
+        break;
+      case STRING:
+        Assert.assertTrue("Should expect a CharSequence", expected instanceof CharSequence);
+        Assert.assertEquals("string should be equal",
+            String.valueOf(expected), supplier.get().toString());
+        break;
+      case DATE:
+        Assert.assertTrue("Should expect a Date", expected instanceof LocalDate);
+        LocalDate date = ChronoUnit.DAYS.addTo(EPOCH_DAY, (int) supplier.get());
+        Assert.assertEquals("date should be equal", expected, date);
+        break;
+      case TIME:
+        Assert.assertTrue("Should expect a LocalTime", expected instanceof LocalTime);
+        int milliseconds = (int) (((LocalTime) expected).toNanoOfDay() / 1000_000);
+        Assert.assertEquals("time millis should be equal", milliseconds, supplier.get());
+        break;
+      case TIMESTAMP:
+        if (((Types.TimestampType) type).shouldAdjustToUTC()) {
+          Assert.assertTrue("Should expect a OffsetDataTime", expected instanceof OffsetDateTime);
+          OffsetDateTime ts = (OffsetDateTime) expected;
+          Assert.assertEquals("OffsetDataTime should be equal", ts.toLocalDateTime(),
+              ((TimestampData) supplier.get()).toLocalDateTime());
+        } else {
+          Assert.assertTrue("Should expect a LocalDataTime", expected instanceof LocalDateTime);
+          LocalDateTime ts = (LocalDateTime) expected;
+          Assert.assertEquals("LocalDataTime should be equal", ts,
+              ((TimestampData) supplier.get()).toLocalDateTime());
+        }
+        break;
+      case BINARY:
+        Assert.assertTrue("Should expect a ByteBuffer", expected instanceof ByteBuffer);
+        Assert.assertArrayEquals("binary should be equal", ((ByteBuffer) expected).array(), (byte[]) supplier.get());
+        break;
+      case DECIMAL:
+        Assert.assertTrue("Should expect a BigDecimal", expected instanceof BigDecimal);
+        BigDecimal bd = (BigDecimal) expected;
+        Assert.assertEquals("decimal value should be equal", bd,
+            ((DecimalData) supplier.get()).toBigDecimal());
+        break;
+      case LIST:
+        Assert.assertTrue("Should expect a Collection", expected instanceof Collection);
+        Collection<?> expectedArrayData = (Collection<?>) expected;
+        ArrayData actualArrayData = (ArrayData) supplier.get();
+        LogicalType elementType = ((ArrayType) logicalType).getElementType();
+        Assert.assertEquals("array length should be equal", expectedArrayData.size(), actualArrayData.size());
+        assertArrayValues(type.asListType().elementType(), elementType, expectedArrayData, actualArrayData);
+        break;
+      case MAP:
+        Assert.assertTrue("Should expect a Map", expected instanceof Map);
+        MapData actual = (MapData) supplier.get();
+        Assert.assertEquals("map size should be equal",
+            ((Map<?, ?>) expected).size(), actual.size());
+        Collection<?> expectedKeyArrayData = ((Map<?, ?>) expected).keySet();

Review comment:
       Fixed.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] chenjunjiedada commented on a change in pull request #1266: Flink: update parquet reader with schema visitor

Posted by GitBox <gi...@apache.org>.
chenjunjiedada commented on a change in pull request #1266:
URL: https://github.com/apache/iceberg/pull/1266#discussion_r463624525



##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java
##########
@@ -19,64 +19,719 @@
 
 package org.apache.iceberg.flink.data;
 
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.time.Instant;
 import java.util.List;
-import org.apache.flink.types.Row;
+import java.util.Map;
+import org.apache.commons.lang3.ArrayUtils;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.MapData;
+import org.apache.flink.table.data.RawValueData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
 import org.apache.iceberg.Schema;
-import org.apache.iceberg.data.parquet.BaseParquetReaders;
+import org.apache.iceberg.parquet.ParquetSchemaUtil;
 import org.apache.iceberg.parquet.ParquetValueReader;
 import org.apache.iceberg.parquet.ParquetValueReaders;
+import org.apache.iceberg.parquet.TypeWithSchemaVisitor;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
 import org.apache.iceberg.types.Types;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.io.api.Binary;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.LogicalTypeAnnotation.DecimalLogicalTypeAnnotation;
 import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
 import org.apache.parquet.schema.Type;
 
-public class FlinkParquetReaders extends BaseParquetReaders<Row> {
+class FlinkParquetReaders {
+  private FlinkParquetReaders() {
+  }
 
-  private static final FlinkParquetReaders INSTANCE = new FlinkParquetReaders();
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema, MessageType fileSchema) {
+    return buildReader(expectedSchema, fileSchema, ImmutableMap.of());
+  }
 
-  private FlinkParquetReaders() {
+  @SuppressWarnings("unchecked")
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema,
+                                                        MessageType fileSchema,
+                                                        Map<Integer, ?> idToConstant) {
+    ReadBuilder builder = new ReadBuilder(fileSchema, idToConstant);
+    if (ParquetSchemaUtil.hasIds(fileSchema)) {
+      return (ParquetValueReader<RowData>)
+          TypeWithSchemaVisitor.visit(expectedSchema.asStruct(), fileSchema, builder);
+    } else {
+      return (ParquetValueReader<RowData>)
+          TypeWithSchemaVisitor.visit(expectedSchema.asStruct(), fileSchema,
+              new FallbackReadBuilder(builder));
+    }
+  }
+
+  private static class FallbackReadBuilder extends TypeWithSchemaVisitor<ParquetValueReader<?>> {
+    private MessageType type;
+    private final TypeWithSchemaVisitor<ParquetValueReader<?>> builder;
+
+    FallbackReadBuilder(TypeWithSchemaVisitor<ParquetValueReader<?>> builder) {
+      this.builder = builder;
+    }
+
+    @Override
+    public ParquetValueReader<?> message(Types.StructType expected, MessageType message,
+                                         List<ParquetValueReader<?>> fieldReaders) {
+      // the top level matches by ID, but the remaining IDs are missing
+      this.type = message;
+      return builder.struct(expected, message, fieldReaders);
+    }
+
+    @Override
+    public ParquetValueReader<?> struct(Types.StructType ignored, GroupType struct,
+                                        List<ParquetValueReader<?>> fieldReaders) {
+      // the expected struct is ignored because nested fields are never found when the IDs are missing
+      List<ParquetValueReader<?>> newFields = Lists.newArrayListWithExpectedSize(
+          fieldReaders.size());
+      List<Type> types = Lists.newArrayListWithExpectedSize(fieldReaders.size());
+      List<Type> fields = struct.getFields();
+      for (int i = 0; i < fields.size(); i += 1) {
+        Type fieldType = fields.get(i);
+        int fieldD = type.getMaxDefinitionLevel(path(fieldType.getName())) - 1;
+        newFields.add(ParquetValueReaders.option(fieldType, fieldD, fieldReaders.get(i)));
+        types.add(fieldType);
+      }
+
+      return new RowDataReader(types, newFields);
+    }
+  }
+
+  private static class ReadBuilder extends TypeWithSchemaVisitor<ParquetValueReader<?>> {

Review comment:
       +1 to refactor this. The struct/list/map methods are mostly the same, only primitive methods are different from each other due to differences in data models. This should also apply to Avro and Orc `ReaderBuilder`.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] chenjunjiedada commented on a change in pull request #1266: Flink: update parquet reader with schema visitor

Posted by GitBox <gi...@apache.org>.
chenjunjiedada commented on a change in pull request #1266:
URL: https://github.com/apache/iceberg/pull/1266#discussion_r467028964



##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java
##########
@@ -19,64 +19,701 @@
 
 package org.apache.iceberg.flink.data;
 
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.time.Instant;
+import java.time.ZoneOffset;
 import java.util.List;
-import org.apache.flink.types.Row;
+import java.util.Map;
+import org.apache.commons.lang3.ArrayUtils;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.MapData;
+import org.apache.flink.table.data.RawValueData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
 import org.apache.iceberg.Schema;
-import org.apache.iceberg.data.parquet.BaseParquetReaders;
 import org.apache.iceberg.parquet.ParquetValueReader;
 import org.apache.iceberg.parquet.ParquetValueReaders;
+import org.apache.iceberg.parquet.TypeWithSchemaVisitor;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
 import org.apache.iceberg.types.Types;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.io.api.Binary;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.LogicalTypeAnnotation.DecimalLogicalTypeAnnotation;
 import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
 import org.apache.parquet.schema.Type;
 
-public class FlinkParquetReaders extends BaseParquetReaders<Row> {
+class FlinkParquetReaders {
+  private FlinkParquetReaders() {
+  }
 
-  private static final FlinkParquetReaders INSTANCE = new FlinkParquetReaders();
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema, MessageType fileSchema) {
+    return buildReader(expectedSchema, fileSchema, ImmutableMap.of());
+  }
 
-  private FlinkParquetReaders() {
+  @SuppressWarnings("unchecked")
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema,
+                                                        MessageType fileSchema,
+                                                        Map<Integer, ?> idToConstant) {
+    return (ParquetValueReader<RowData>) TypeWithSchemaVisitor.visit(expectedSchema.asStruct(), fileSchema,
+        new ReadBuilder(fileSchema, idToConstant)
+    );
+  }
+
+  private static class ReadBuilder extends TypeWithSchemaVisitor<ParquetValueReader<?>> {
+    private final MessageType type;
+    private final Map<Integer, ?> idToConstant;
+
+    ReadBuilder(MessageType type, Map<Integer, ?> idToConstant) {
+      this.type = type;
+      this.idToConstant = idToConstant;
+    }
+
+    @Override
+    public ParquetValueReader<RowData> message(Types.StructType expected, MessageType message,
+                                               List<ParquetValueReader<?>> fieldReaders) {
+      return struct(expected, message.asGroupType(), fieldReaders);
+    }
+
+    @Override
+    public ParquetValueReader<RowData> struct(Types.StructType expected, GroupType struct,
+                                              List<ParquetValueReader<?>> fieldReaders) {
+      // match the expected struct's order
+      Map<Integer, ParquetValueReader<?>> readersById = Maps.newHashMap();
+      Map<Integer, Type> typesById = Maps.newHashMap();
+      List<Type> fields = struct.getFields();
+      for (int i = 0; i < fields.size(); i += 1) {
+        Type fieldType = fields.get(i);
+        int fieldD = type.getMaxDefinitionLevel(path(fieldType.getName())) - 1;
+        if (fieldType.getId() != null) {
+          int id = fieldType.getId().intValue();
+          readersById.put(id, ParquetValueReaders.option(fieldType, fieldD, fieldReaders.get(i)));
+          typesById.put(id, fieldType);
+        }
+      }
+
+      List<Types.NestedField> expectedFields = expected != null ?
+          expected.fields() : ImmutableList.of();
+      List<ParquetValueReader<?>> reorderedFields = Lists.newArrayListWithExpectedSize(
+          expectedFields.size());
+      List<Type> types = Lists.newArrayListWithExpectedSize(expectedFields.size());
+      for (Types.NestedField field : expectedFields) {
+        int id = field.fieldId();
+        if (idToConstant.containsKey(id)) {
+          // containsKey is used because the constant may be null
+          reorderedFields.add(ParquetValueReaders.constant(idToConstant.get(id)));
+          types.add(null);
+        } else {
+          ParquetValueReader<?> reader = readersById.get(id);
+          if (reader != null) {
+            reorderedFields.add(reader);
+            types.add(typesById.get(id));
+          } else {
+            reorderedFields.add(ParquetValueReaders.nulls());
+            types.add(null);
+          }
+        }
+      }
+
+      return new RowDataReader(types, reorderedFields);
+    }
+
+    @Override
+    public ParquetValueReader<?> list(Types.ListType expectedList, GroupType array,
+                                      ParquetValueReader<?> elementReader) {
+      GroupType repeated = array.getFields().get(0).asGroupType();
+      String[] repeatedPath = currentPath();
+
+      int repeatedD = type.getMaxDefinitionLevel(repeatedPath) - 1;
+      int repeatedR = type.getMaxRepetitionLevel(repeatedPath) - 1;
+
+      Type elementType = repeated.getType(0);
+      int elementD = type.getMaxDefinitionLevel(path(elementType.getName())) - 1;
+
+      return new ArrayReader<>(repeatedD, repeatedR, ParquetValueReaders.option(elementType, elementD, elementReader));
+    }
+
+    @Override
+    public ParquetValueReader<?> map(Types.MapType expectedMap, GroupType map,
+                                     ParquetValueReader<?> keyReader,
+                                     ParquetValueReader<?> valueReader) {
+      GroupType repeatedKeyValue = map.getFields().get(0).asGroupType();
+      String[] repeatedPath = currentPath();
+
+      int repeatedD = type.getMaxDefinitionLevel(repeatedPath) - 1;
+      int repeatedR = type.getMaxRepetitionLevel(repeatedPath) - 1;
+
+      Type keyType = repeatedKeyValue.getType(0);
+      int keyD = type.getMaxDefinitionLevel(path(keyType.getName())) - 1;
+      Type valueType = repeatedKeyValue.getType(1);
+      int valueD = type.getMaxDefinitionLevel(path(valueType.getName())) - 1;
+
+      return new MapReader<>(repeatedD, repeatedR,
+          ParquetValueReaders.option(keyType, keyD, keyReader),
+          ParquetValueReaders.option(valueType, valueD, valueReader));
+    }
+
+    @Override
+    public ParquetValueReader<?> primitive(org.apache.iceberg.types.Type.PrimitiveType expected,
+                                           PrimitiveType primitive) {
+      ColumnDescriptor desc = type.getColumnDescription(currentPath());
+
+      if (primitive.getOriginalType() != null) {
+        switch (primitive.getOriginalType()) {
+          case ENUM:
+          case JSON:
+          case UTF8:
+            return new StringReader(desc);
+          case INT_8:
+          case INT_16:
+          case INT_32:
+          case DATE:
+            if (expected != null && expected.typeId() == Types.LongType.get().typeId()) {
+              return new ParquetValueReaders.IntAsLongReader(desc);
+            } else {
+              return new ParquetValueReaders.UnboxedReader<>(desc);
+            }
+          case TIME_MICROS:
+            return new LossyMicrosToMillisTimeReader(desc);
+          case INT_64:
+            return new ParquetValueReaders.UnboxedReader<>(desc);
+          case TIMESTAMP_MICROS:
+            if (((Types.TimestampType) expected).shouldAdjustToUTC()) {
+              return new TimestampTzReader(desc);
+            } else {
+              return new TimestampReader(desc);
+            }
+          case DECIMAL:
+            DecimalLogicalTypeAnnotation decimal = (DecimalLogicalTypeAnnotation) primitive.getLogicalTypeAnnotation();
+            switch (primitive.getPrimitiveTypeName()) {
+              case BINARY:
+              case FIXED_LEN_BYTE_ARRAY:
+                return new BinaryDecimalReader(desc, decimal.getScale());
+              case INT64:
+                return new LongDecimalReader(desc, decimal.getPrecision(), decimal.getScale());
+              case INT32:
+                return new IntegerDecimalReader(desc, decimal.getPrecision(), decimal.getScale());
+              default:
+                throw new UnsupportedOperationException(
+                    "Unsupported base type for decimal: " + primitive.getPrimitiveTypeName());
+            }
+          case BSON:
+            return new ParquetValueReaders.ByteArrayReader(desc);
+          default:
+            throw new UnsupportedOperationException(
+                "Unsupported logical type: " + primitive.getOriginalType());
+        }
+      }
+
+      switch (primitive.getPrimitiveTypeName()) {
+        case FIXED_LEN_BYTE_ARRAY:
+        case BINARY:
+          return new ParquetValueReaders.ByteArrayReader(desc);
+        case INT32:
+          if (expected != null && expected.typeId() == org.apache.iceberg.types.Type.TypeID.LONG) {
+            return new ParquetValueReaders.IntAsLongReader(desc);
+          } else {
+            return new ParquetValueReaders.UnboxedReader<>(desc);
+          }
+        case FLOAT:
+          if (expected != null && expected.typeId() == org.apache.iceberg.types.Type.TypeID.DOUBLE) {
+            return new ParquetValueReaders.FloatAsDoubleReader(desc);
+          } else {
+            return new ParquetValueReaders.UnboxedReader<>(desc);
+          }
+        case BOOLEAN:
+        case INT64:
+        case DOUBLE:
+          return new ParquetValueReaders.UnboxedReader<>(desc);
+        default:
+          throw new UnsupportedOperationException("Unsupported type: " + primitive);
+      }
+    }
+  }
+
+  private static class BinaryDecimalReader extends ParquetValueReaders.PrimitiveReader<DecimalData> {
+    private final int scale;
+
+    BinaryDecimalReader(ColumnDescriptor desc, int scale) {
+      super(desc);
+      this.scale = scale;
+    }
+
+    @Override
+    public DecimalData read(DecimalData ignored) {
+      Binary binary = column.nextBinary();
+      BigDecimal bigDecimal = new BigDecimal(new BigInteger(binary.getBytes()), scale);
+      return DecimalData.fromBigDecimal(bigDecimal, bigDecimal.precision(), scale);

Review comment:
       Done.

##########
File path: flink/src/test/java/org/apache/iceberg/flink/data/TestHelpers.java
##########
@@ -0,0 +1,199 @@
+/*
+ * 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.iceberg.flink.data;
+
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.OffsetDateTime;
+import java.time.ZoneOffset;
+import java.time.temporal.ChronoUnit;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import java.util.function.Supplier;
+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.MapType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+
+public class TestHelpers {
+  private TestHelpers() {}
+
+  private static final OffsetDateTime EPOCH = Instant.ofEpochMilli(0L).atOffset(ZoneOffset.UTC);
+  private static final LocalDate EPOCH_DAY = EPOCH.toLocalDate();
+
+  public static void assertRowData(Type type, LogicalType rowType, Record expectedRecord, RowData actualRowData) {
+    if (expectedRecord == null && actualRowData == null) {
+      return;
+    }
+
+    Assert.assertTrue("expected Record and actual RowData should be both null or not null",
+        expectedRecord != null && actualRowData != null);
+
+    List<Type> types = Lists.newArrayList();
+    for (Types.NestedField field : type.asStructType().fields()) {
+      types.add(field.type());
+    }
+
+    for (int i = 0; i < types.size(); i += 1) {
+      if (expectedRecord.get(i) == null) {
+        Assert.assertTrue(actualRowData.isNullAt(i));
+        continue;
+      }
+
+      Object expected = expectedRecord.get(i);
+      LogicalType logicalType = ((RowType) rowType).getTypeAt(i);
+
+      final int fieldPos = i;
+      assertEquals(types.get(i), logicalType, expected,
+          () -> RowData.createFieldGetter(logicalType, fieldPos).getFieldOrNull(actualRowData));
+    }
+
+  }
+
+  private static void assertEquals(Type type, LogicalType logicalType, Object expected, Supplier<Object> supplier) {
+    switch (type.typeId()) {
+      case BOOLEAN:
+        Assert.assertEquals("boolean value should be equal", expected, supplier.get());
+        break;
+      case INTEGER:
+        Assert.assertEquals("int value should be equal", expected, supplier.get());
+        break;
+      case LONG:
+        Assert.assertEquals("long value should be equal", expected, supplier.get());
+        break;
+      case FLOAT:
+        Assert.assertEquals("float value should be equal", expected, supplier.get());
+        break;
+      case DOUBLE:
+        Assert.assertEquals("double value should be equal", expected, supplier.get());
+        break;
+      case STRING:
+        Assert.assertTrue("Should expect a CharSequence", expected instanceof CharSequence);
+        Assert.assertEquals("string should be equal",
+            String.valueOf(expected), supplier.get().toString());
+        break;
+      case DATE:
+        Assert.assertTrue("Should expect a Date", expected instanceof LocalDate);
+        LocalDate date = ChronoUnit.DAYS.addTo(EPOCH_DAY, (int) supplier.get());
+        Assert.assertEquals("date should be equal", expected, date);
+        break;
+      case TIME:
+        Assert.assertTrue("Should expect a LocalTime", expected instanceof LocalTime);
+        int milliseconds = (int) (((LocalTime) expected).toNanoOfDay() / 1000_000);
+        Assert.assertEquals("time millis should be equal", milliseconds, supplier.get());
+        break;
+      case TIMESTAMP:
+        if (((Types.TimestampType) type.asPrimitiveType()).shouldAdjustToUTC()) {

Review comment:
       Done.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #1266: Flink: update parquet reader with schema visitor

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1266:
URL: https://github.com/apache/iceberg/pull/1266#discussion_r466923630



##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java
##########
@@ -19,64 +19,701 @@
 
 package org.apache.iceberg.flink.data;
 
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.time.Instant;
+import java.time.ZoneOffset;
 import java.util.List;
-import org.apache.flink.types.Row;
+import java.util.Map;
+import org.apache.commons.lang3.ArrayUtils;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.MapData;
+import org.apache.flink.table.data.RawValueData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
 import org.apache.iceberg.Schema;
-import org.apache.iceberg.data.parquet.BaseParquetReaders;
 import org.apache.iceberg.parquet.ParquetValueReader;
 import org.apache.iceberg.parquet.ParquetValueReaders;
+import org.apache.iceberg.parquet.TypeWithSchemaVisitor;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
 import org.apache.iceberg.types.Types;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.io.api.Binary;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.LogicalTypeAnnotation.DecimalLogicalTypeAnnotation;
 import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
 import org.apache.parquet.schema.Type;
 
-public class FlinkParquetReaders extends BaseParquetReaders<Row> {
+class FlinkParquetReaders {
+  private FlinkParquetReaders() {
+  }
 
-  private static final FlinkParquetReaders INSTANCE = new FlinkParquetReaders();
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema, MessageType fileSchema) {
+    return buildReader(expectedSchema, fileSchema, ImmutableMap.of());
+  }
 
-  private FlinkParquetReaders() {
+  @SuppressWarnings("unchecked")
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema,
+                                                        MessageType fileSchema,
+                                                        Map<Integer, ?> idToConstant) {
+    return (ParquetValueReader<RowData>) TypeWithSchemaVisitor.visit(expectedSchema.asStruct(), fileSchema,
+        new ReadBuilder(fileSchema, idToConstant)
+    );
+  }
+
+  private static class ReadBuilder extends TypeWithSchemaVisitor<ParquetValueReader<?>> {
+    private final MessageType type;
+    private final Map<Integer, ?> idToConstant;
+
+    ReadBuilder(MessageType type, Map<Integer, ?> idToConstant) {
+      this.type = type;
+      this.idToConstant = idToConstant;
+    }
+
+    @Override
+    public ParquetValueReader<RowData> message(Types.StructType expected, MessageType message,
+                                               List<ParquetValueReader<?>> fieldReaders) {
+      return struct(expected, message.asGroupType(), fieldReaders);
+    }
+
+    @Override
+    public ParquetValueReader<RowData> struct(Types.StructType expected, GroupType struct,
+                                              List<ParquetValueReader<?>> fieldReaders) {
+      // match the expected struct's order
+      Map<Integer, ParquetValueReader<?>> readersById = Maps.newHashMap();
+      Map<Integer, Type> typesById = Maps.newHashMap();
+      List<Type> fields = struct.getFields();
+      for (int i = 0; i < fields.size(); i += 1) {
+        Type fieldType = fields.get(i);
+        int fieldD = type.getMaxDefinitionLevel(path(fieldType.getName())) - 1;
+        if (fieldType.getId() != null) {
+          int id = fieldType.getId().intValue();
+          readersById.put(id, ParquetValueReaders.option(fieldType, fieldD, fieldReaders.get(i)));
+          typesById.put(id, fieldType);
+        }
+      }
+
+      List<Types.NestedField> expectedFields = expected != null ?
+          expected.fields() : ImmutableList.of();
+      List<ParquetValueReader<?>> reorderedFields = Lists.newArrayListWithExpectedSize(
+          expectedFields.size());
+      List<Type> types = Lists.newArrayListWithExpectedSize(expectedFields.size());
+      for (Types.NestedField field : expectedFields) {
+        int id = field.fieldId();
+        if (idToConstant.containsKey(id)) {
+          // containsKey is used because the constant may be null
+          reorderedFields.add(ParquetValueReaders.constant(idToConstant.get(id)));
+          types.add(null);
+        } else {
+          ParquetValueReader<?> reader = readersById.get(id);
+          if (reader != null) {
+            reorderedFields.add(reader);
+            types.add(typesById.get(id));
+          } else {
+            reorderedFields.add(ParquetValueReaders.nulls());
+            types.add(null);
+          }
+        }
+      }
+
+      return new RowDataReader(types, reorderedFields);
+    }
+
+    @Override
+    public ParquetValueReader<?> list(Types.ListType expectedList, GroupType array,
+                                      ParquetValueReader<?> elementReader) {
+      GroupType repeated = array.getFields().get(0).asGroupType();
+      String[] repeatedPath = currentPath();
+
+      int repeatedD = type.getMaxDefinitionLevel(repeatedPath) - 1;
+      int repeatedR = type.getMaxRepetitionLevel(repeatedPath) - 1;
+
+      Type elementType = repeated.getType(0);
+      int elementD = type.getMaxDefinitionLevel(path(elementType.getName())) - 1;
+
+      return new ArrayReader<>(repeatedD, repeatedR, ParquetValueReaders.option(elementType, elementD, elementReader));
+    }
+
+    @Override
+    public ParquetValueReader<?> map(Types.MapType expectedMap, GroupType map,
+                                     ParquetValueReader<?> keyReader,
+                                     ParquetValueReader<?> valueReader) {
+      GroupType repeatedKeyValue = map.getFields().get(0).asGroupType();
+      String[] repeatedPath = currentPath();
+
+      int repeatedD = type.getMaxDefinitionLevel(repeatedPath) - 1;
+      int repeatedR = type.getMaxRepetitionLevel(repeatedPath) - 1;
+
+      Type keyType = repeatedKeyValue.getType(0);
+      int keyD = type.getMaxDefinitionLevel(path(keyType.getName())) - 1;
+      Type valueType = repeatedKeyValue.getType(1);
+      int valueD = type.getMaxDefinitionLevel(path(valueType.getName())) - 1;
+
+      return new MapReader<>(repeatedD, repeatedR,
+          ParquetValueReaders.option(keyType, keyD, keyReader),
+          ParquetValueReaders.option(valueType, valueD, valueReader));
+    }
+
+    @Override
+    public ParquetValueReader<?> primitive(org.apache.iceberg.types.Type.PrimitiveType expected,
+                                           PrimitiveType primitive) {
+      ColumnDescriptor desc = type.getColumnDescription(currentPath());
+
+      if (primitive.getOriginalType() != null) {
+        switch (primitive.getOriginalType()) {
+          case ENUM:
+          case JSON:
+          case UTF8:
+            return new StringReader(desc);
+          case INT_8:
+          case INT_16:
+          case INT_32:
+          case DATE:
+            if (expected != null && expected.typeId() == Types.LongType.get().typeId()) {
+              return new ParquetValueReaders.IntAsLongReader(desc);
+            } else {
+              return new ParquetValueReaders.UnboxedReader<>(desc);
+            }
+          case TIME_MICROS:
+            return new LossyMicrosToMillisTimeReader(desc);
+          case INT_64:
+            return new ParquetValueReaders.UnboxedReader<>(desc);
+          case TIMESTAMP_MICROS:
+            if (((Types.TimestampType) expected).shouldAdjustToUTC()) {
+              return new TimestampTzReader(desc);
+            } else {
+              return new TimestampReader(desc);
+            }
+          case DECIMAL:
+            DecimalLogicalTypeAnnotation decimal = (DecimalLogicalTypeAnnotation) primitive.getLogicalTypeAnnotation();
+            switch (primitive.getPrimitiveTypeName()) {
+              case BINARY:
+              case FIXED_LEN_BYTE_ARRAY:
+                return new BinaryDecimalReader(desc, decimal.getScale());
+              case INT64:
+                return new LongDecimalReader(desc, decimal.getPrecision(), decimal.getScale());
+              case INT32:
+                return new IntegerDecimalReader(desc, decimal.getPrecision(), decimal.getScale());
+              default:
+                throw new UnsupportedOperationException(
+                    "Unsupported base type for decimal: " + primitive.getPrimitiveTypeName());
+            }
+          case BSON:
+            return new ParquetValueReaders.ByteArrayReader(desc);
+          default:
+            throw new UnsupportedOperationException(
+                "Unsupported logical type: " + primitive.getOriginalType());
+        }
+      }
+
+      switch (primitive.getPrimitiveTypeName()) {
+        case FIXED_LEN_BYTE_ARRAY:
+        case BINARY:
+          return new ParquetValueReaders.ByteArrayReader(desc);
+        case INT32:
+          if (expected != null && expected.typeId() == org.apache.iceberg.types.Type.TypeID.LONG) {
+            return new ParquetValueReaders.IntAsLongReader(desc);
+          } else {
+            return new ParquetValueReaders.UnboxedReader<>(desc);
+          }
+        case FLOAT:
+          if (expected != null && expected.typeId() == org.apache.iceberg.types.Type.TypeID.DOUBLE) {
+            return new ParquetValueReaders.FloatAsDoubleReader(desc);
+          } else {
+            return new ParquetValueReaders.UnboxedReader<>(desc);
+          }
+        case BOOLEAN:
+        case INT64:
+        case DOUBLE:
+          return new ParquetValueReaders.UnboxedReader<>(desc);
+        default:
+          throw new UnsupportedOperationException("Unsupported type: " + primitive);
+      }
+    }
+  }
+
+  private static class BinaryDecimalReader extends ParquetValueReaders.PrimitiveReader<DecimalData> {
+    private final int scale;
+
+    BinaryDecimalReader(ColumnDescriptor desc, int scale) {
+      super(desc);
+      this.scale = scale;
+    }
+
+    @Override
+    public DecimalData read(DecimalData ignored) {
+      Binary binary = column.nextBinary();
+      BigDecimal bigDecimal = new BigDecimal(new BigInteger(binary.getBytes()), scale);
+      return DecimalData.fromBigDecimal(bigDecimal, bigDecimal.precision(), scale);

Review comment:
       Yeah, make sense.  I think we may need a TODO comment here so that we won't forget that.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] chenjunjiedada commented on a change in pull request #1266: Flink: update parquet reader with schema visitor

Posted by GitBox <gi...@apache.org>.
chenjunjiedada commented on a change in pull request #1266:
URL: https://github.com/apache/iceberg/pull/1266#discussion_r469690242



##########
File path: flink/src/test/java/org/apache/iceberg/flink/data/TestHelpers.java
##########
@@ -0,0 +1,198 @@
+/*
+ * 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.iceberg.flink.data;
+
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.OffsetDateTime;
+import java.time.ZoneOffset;
+import java.time.temporal.ChronoUnit;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import java.util.function.Supplier;
+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.MapType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+
+public class TestHelpers {
+  private TestHelpers() {}
+
+  private static final OffsetDateTime EPOCH = Instant.ofEpochMilli(0L).atOffset(ZoneOffset.UTC);
+  private static final LocalDate EPOCH_DAY = EPOCH.toLocalDate();
+
+  public static void assertRowData(Type type, LogicalType rowType, Record expectedRecord, RowData actualRowData) {
+    if (expectedRecord == null && actualRowData == null) {
+      return;
+    }
+
+    Assert.assertTrue("expected Record and actual RowData should be both null or not null",
+        expectedRecord != null && actualRowData != null);
+
+    List<Type> types = Lists.newArrayList();
+    for (Types.NestedField field : type.asStructType().fields()) {
+      types.add(field.type());
+    }
+
+    for (int i = 0; i < types.size(); i += 1) {
+      if (expectedRecord.get(i) == null) {
+        Assert.assertTrue(actualRowData.isNullAt(i));
+        continue;
+      }
+
+      Object expected = expectedRecord.get(i);
+      LogicalType logicalType = ((RowType) rowType).getTypeAt(i);
+
+      final int fieldPos = i;
+      assertEquals(types.get(i), logicalType, expected,
+          () -> RowData.createFieldGetter(logicalType, fieldPos).getFieldOrNull(actualRowData));
+    }
+  }
+
+  private static void assertEquals(Type type, LogicalType logicalType, Object expected, Supplier<Object> supplier) {
+    switch (type.typeId()) {
+      case BOOLEAN:
+        Assert.assertEquals("boolean value should be equal", expected, supplier.get());
+        break;
+      case INTEGER:
+        Assert.assertEquals("int value should be equal", expected, supplier.get());
+        break;
+      case LONG:
+        Assert.assertEquals("long value should be equal", expected, supplier.get());
+        break;
+      case FLOAT:
+        Assert.assertEquals("float value should be equal", expected, supplier.get());
+        break;
+      case DOUBLE:
+        Assert.assertEquals("double value should be equal", expected, supplier.get());
+        break;
+      case STRING:
+        Assert.assertTrue("Should expect a CharSequence", expected instanceof CharSequence);
+        Assert.assertEquals("string should be equal",
+            String.valueOf(expected), supplier.get().toString());
+        break;
+      case DATE:
+        Assert.assertTrue("Should expect a Date", expected instanceof LocalDate);
+        LocalDate date = ChronoUnit.DAYS.addTo(EPOCH_DAY, (int) supplier.get());

Review comment:
       Sure, updated.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] chenjunjiedada commented on a change in pull request #1266: Flink: update parquet reader with schema visitor

Posted by GitBox <gi...@apache.org>.
chenjunjiedada commented on a change in pull request #1266:
URL: https://github.com/apache/iceberg/pull/1266#discussion_r469692416



##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java
##########
@@ -19,64 +19,678 @@
 
 package org.apache.iceberg.flink.data;
 
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
 import java.util.List;
-import org.apache.flink.types.Row;
+import java.util.Map;
+import org.apache.commons.lang3.ArrayUtils;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.MapData;
+import org.apache.flink.table.data.RawValueData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
 import org.apache.iceberg.Schema;
-import org.apache.iceberg.data.parquet.BaseParquetReaders;
 import org.apache.iceberg.parquet.ParquetValueReader;
 import org.apache.iceberg.parquet.ParquetValueReaders;
+import org.apache.iceberg.parquet.TypeWithSchemaVisitor;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
 import org.apache.iceberg.types.Types;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.io.api.Binary;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.LogicalTypeAnnotation.DecimalLogicalTypeAnnotation;
 import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
 import org.apache.parquet.schema.Type;
 
-public class FlinkParquetReaders extends BaseParquetReaders<Row> {
+class FlinkParquetReaders {
+  private FlinkParquetReaders() {
+  }
 
-  private static final FlinkParquetReaders INSTANCE = new FlinkParquetReaders();
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema, MessageType fileSchema) {
+    return buildReader(expectedSchema, fileSchema, ImmutableMap.of());
+  }
 
-  private FlinkParquetReaders() {
+  @SuppressWarnings("unchecked")
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema,
+                                                        MessageType fileSchema,
+                                                        Map<Integer, ?> idToConstant) {
+    return (ParquetValueReader<RowData>) TypeWithSchemaVisitor.visit(expectedSchema.asStruct(), fileSchema,
+        new ReadBuilder(fileSchema, idToConstant)
+    );
+  }
+
+  private static class ReadBuilder extends TypeWithSchemaVisitor<ParquetValueReader<?>> {
+    private final MessageType type;
+    private final Map<Integer, ?> idToConstant;
+
+    ReadBuilder(MessageType type, Map<Integer, ?> idToConstant) {
+      this.type = type;
+      this.idToConstant = idToConstant;
+    }
+
+    @Override
+    public ParquetValueReader<RowData> message(Types.StructType expected, MessageType message,
+                                               List<ParquetValueReader<?>> fieldReaders) {
+      return struct(expected, message.asGroupType(), fieldReaders);
+    }
+
+    @Override
+    public ParquetValueReader<RowData> struct(Types.StructType expected, GroupType struct,
+                                              List<ParquetValueReader<?>> fieldReaders) {
+      // match the expected struct's order
+      Map<Integer, ParquetValueReader<?>> readersById = Maps.newHashMap();
+      Map<Integer, Type> typesById = Maps.newHashMap();
+      List<Type> fields = struct.getFields();
+      for (int i = 0; i < fields.size(); i += 1) {
+        Type fieldType = fields.get(i);
+        int fieldD = type.getMaxDefinitionLevel(path(fieldType.getName())) - 1;
+        if (fieldType.getId() != null) {
+          int id = fieldType.getId().intValue();
+          readersById.put(id, ParquetValueReaders.option(fieldType, fieldD, fieldReaders.get(i)));
+          typesById.put(id, fieldType);
+        }
+      }
+
+      List<Types.NestedField> expectedFields = expected != null ?
+          expected.fields() : ImmutableList.of();
+      List<ParquetValueReader<?>> reorderedFields = Lists.newArrayListWithExpectedSize(
+          expectedFields.size());
+      List<Type> types = Lists.newArrayListWithExpectedSize(expectedFields.size());
+      for (Types.NestedField field : expectedFields) {
+        int id = field.fieldId();
+        if (idToConstant.containsKey(id)) {
+          // containsKey is used because the constant may be null
+          reorderedFields.add(ParquetValueReaders.constant(idToConstant.get(id)));
+          types.add(null);
+        } else {
+          ParquetValueReader<?> reader = readersById.get(id);
+          if (reader != null) {
+            reorderedFields.add(reader);
+            types.add(typesById.get(id));
+          } else {
+            reorderedFields.add(ParquetValueReaders.nulls());
+            types.add(null);
+          }
+        }
+      }
+
+      return new RowDataReader(types, reorderedFields);
+    }
+
+    @Override
+    public ParquetValueReader<?> list(Types.ListType expectedList, GroupType array,
+                                      ParquetValueReader<?> elementReader) {
+      GroupType repeated = array.getFields().get(0).asGroupType();
+      String[] repeatedPath = currentPath();
+
+      int repeatedD = type.getMaxDefinitionLevel(repeatedPath) - 1;
+      int repeatedR = type.getMaxRepetitionLevel(repeatedPath) - 1;
+
+      Type elementType = repeated.getType(0);
+      int elementD = type.getMaxDefinitionLevel(path(elementType.getName())) - 1;
+
+      return new ArrayReader<>(repeatedD, repeatedR, ParquetValueReaders.option(elementType, elementD, elementReader));
+    }
+
+    @Override
+    public ParquetValueReader<?> map(Types.MapType expectedMap, GroupType map,
+                                     ParquetValueReader<?> keyReader,
+                                     ParquetValueReader<?> valueReader) {
+      GroupType repeatedKeyValue = map.getFields().get(0).asGroupType();
+      String[] repeatedPath = currentPath();
+
+      int repeatedD = type.getMaxDefinitionLevel(repeatedPath) - 1;
+      int repeatedR = type.getMaxRepetitionLevel(repeatedPath) - 1;
+
+      Type keyType = repeatedKeyValue.getType(0);
+      int keyD = type.getMaxDefinitionLevel(path(keyType.getName())) - 1;
+      Type valueType = repeatedKeyValue.getType(1);
+      int valueD = type.getMaxDefinitionLevel(path(valueType.getName())) - 1;
+
+      return new MapReader<>(repeatedD, repeatedR,
+          ParquetValueReaders.option(keyType, keyD, keyReader),
+          ParquetValueReaders.option(valueType, valueD, valueReader));
+    }
+
+    @Override
+    public ParquetValueReader<?> primitive(org.apache.iceberg.types.Type.PrimitiveType expected,
+                                           PrimitiveType primitive) {
+      ColumnDescriptor desc = type.getColumnDescription(currentPath());
+
+      if (primitive.getOriginalType() != null) {
+        switch (primitive.getOriginalType()) {
+          case ENUM:
+          case JSON:
+          case UTF8:
+            return new StringReader(desc);
+          case INT_8:
+          case INT_16:
+          case INT_32:
+            if (expected != null && expected.typeId() == Types.LongType.get().typeId()) {
+              return new ParquetValueReaders.IntAsLongReader(desc);
+            } else {
+              return new ParquetValueReaders.UnboxedReader<>(desc);
+            }
+          case TIME_MICROS:
+            return new LossyMicrosToMillisTimeReader(desc);
+          case DATE:
+          case INT_64:
+            return new ParquetValueReaders.UnboxedReader<>(desc);
+          case TIMESTAMP_MICROS:
+            return new TimestampMicrosReader(desc);

Review comment:
       Updated.  @openinx, you might want to have a check here. I changed this back to two readers because the type in the imported parquet file could also have time zone info as well.

##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java
##########
@@ -19,64 +19,678 @@
 
 package org.apache.iceberg.flink.data;
 
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
 import java.util.List;
-import org.apache.flink.types.Row;
+import java.util.Map;
+import org.apache.commons.lang3.ArrayUtils;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.MapData;
+import org.apache.flink.table.data.RawValueData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
 import org.apache.iceberg.Schema;
-import org.apache.iceberg.data.parquet.BaseParquetReaders;
 import org.apache.iceberg.parquet.ParquetValueReader;
 import org.apache.iceberg.parquet.ParquetValueReaders;
+import org.apache.iceberg.parquet.TypeWithSchemaVisitor;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
 import org.apache.iceberg.types.Types;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.io.api.Binary;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.LogicalTypeAnnotation.DecimalLogicalTypeAnnotation;
 import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
 import org.apache.parquet.schema.Type;
 
-public class FlinkParquetReaders extends BaseParquetReaders<Row> {
+class FlinkParquetReaders {
+  private FlinkParquetReaders() {
+  }
 
-  private static final FlinkParquetReaders INSTANCE = new FlinkParquetReaders();
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema, MessageType fileSchema) {
+    return buildReader(expectedSchema, fileSchema, ImmutableMap.of());
+  }
 
-  private FlinkParquetReaders() {
+  @SuppressWarnings("unchecked")
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema,
+                                                        MessageType fileSchema,
+                                                        Map<Integer, ?> idToConstant) {
+    return (ParquetValueReader<RowData>) TypeWithSchemaVisitor.visit(expectedSchema.asStruct(), fileSchema,
+        new ReadBuilder(fileSchema, idToConstant)
+    );
+  }
+
+  private static class ReadBuilder extends TypeWithSchemaVisitor<ParquetValueReader<?>> {
+    private final MessageType type;
+    private final Map<Integer, ?> idToConstant;
+
+    ReadBuilder(MessageType type, Map<Integer, ?> idToConstant) {
+      this.type = type;
+      this.idToConstant = idToConstant;
+    }
+
+    @Override
+    public ParquetValueReader<RowData> message(Types.StructType expected, MessageType message,
+                                               List<ParquetValueReader<?>> fieldReaders) {
+      return struct(expected, message.asGroupType(), fieldReaders);
+    }
+
+    @Override
+    public ParquetValueReader<RowData> struct(Types.StructType expected, GroupType struct,
+                                              List<ParquetValueReader<?>> fieldReaders) {
+      // match the expected struct's order
+      Map<Integer, ParquetValueReader<?>> readersById = Maps.newHashMap();
+      Map<Integer, Type> typesById = Maps.newHashMap();
+      List<Type> fields = struct.getFields();
+      for (int i = 0; i < fields.size(); i += 1) {
+        Type fieldType = fields.get(i);
+        int fieldD = type.getMaxDefinitionLevel(path(fieldType.getName())) - 1;
+        if (fieldType.getId() != null) {
+          int id = fieldType.getId().intValue();
+          readersById.put(id, ParquetValueReaders.option(fieldType, fieldD, fieldReaders.get(i)));
+          typesById.put(id, fieldType);
+        }
+      }
+
+      List<Types.NestedField> expectedFields = expected != null ?
+          expected.fields() : ImmutableList.of();
+      List<ParquetValueReader<?>> reorderedFields = Lists.newArrayListWithExpectedSize(
+          expectedFields.size());
+      List<Type> types = Lists.newArrayListWithExpectedSize(expectedFields.size());
+      for (Types.NestedField field : expectedFields) {
+        int id = field.fieldId();
+        if (idToConstant.containsKey(id)) {
+          // containsKey is used because the constant may be null
+          reorderedFields.add(ParquetValueReaders.constant(idToConstant.get(id)));
+          types.add(null);
+        } else {
+          ParquetValueReader<?> reader = readersById.get(id);
+          if (reader != null) {
+            reorderedFields.add(reader);
+            types.add(typesById.get(id));
+          } else {
+            reorderedFields.add(ParquetValueReaders.nulls());
+            types.add(null);
+          }
+        }
+      }
+
+      return new RowDataReader(types, reorderedFields);
+    }
+
+    @Override
+    public ParquetValueReader<?> list(Types.ListType expectedList, GroupType array,
+                                      ParquetValueReader<?> elementReader) {
+      GroupType repeated = array.getFields().get(0).asGroupType();
+      String[] repeatedPath = currentPath();
+
+      int repeatedD = type.getMaxDefinitionLevel(repeatedPath) - 1;
+      int repeatedR = type.getMaxRepetitionLevel(repeatedPath) - 1;
+
+      Type elementType = repeated.getType(0);
+      int elementD = type.getMaxDefinitionLevel(path(elementType.getName())) - 1;
+
+      return new ArrayReader<>(repeatedD, repeatedR, ParquetValueReaders.option(elementType, elementD, elementReader));
+    }
+
+    @Override
+    public ParquetValueReader<?> map(Types.MapType expectedMap, GroupType map,
+                                     ParquetValueReader<?> keyReader,
+                                     ParquetValueReader<?> valueReader) {
+      GroupType repeatedKeyValue = map.getFields().get(0).asGroupType();
+      String[] repeatedPath = currentPath();
+
+      int repeatedD = type.getMaxDefinitionLevel(repeatedPath) - 1;
+      int repeatedR = type.getMaxRepetitionLevel(repeatedPath) - 1;
+
+      Type keyType = repeatedKeyValue.getType(0);
+      int keyD = type.getMaxDefinitionLevel(path(keyType.getName())) - 1;
+      Type valueType = repeatedKeyValue.getType(1);
+      int valueD = type.getMaxDefinitionLevel(path(valueType.getName())) - 1;
+
+      return new MapReader<>(repeatedD, repeatedR,
+          ParquetValueReaders.option(keyType, keyD, keyReader),
+          ParquetValueReaders.option(valueType, valueD, valueReader));
+    }
+
+    @Override
+    public ParquetValueReader<?> primitive(org.apache.iceberg.types.Type.PrimitiveType expected,
+                                           PrimitiveType primitive) {
+      ColumnDescriptor desc = type.getColumnDescription(currentPath());
+
+      if (primitive.getOriginalType() != null) {
+        switch (primitive.getOriginalType()) {
+          case ENUM:
+          case JSON:
+          case UTF8:
+            return new StringReader(desc);
+          case INT_8:
+          case INT_16:
+          case INT_32:
+            if (expected != null && expected.typeId() == Types.LongType.get().typeId()) {
+              return new ParquetValueReaders.IntAsLongReader(desc);
+            } else {
+              return new ParquetValueReaders.UnboxedReader<>(desc);
+            }
+          case TIME_MICROS:

Review comment:
       Added.

##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java
##########
@@ -19,64 +19,678 @@
 
 package org.apache.iceberg.flink.data;
 
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
 import java.util.List;
-import org.apache.flink.types.Row;
+import java.util.Map;
+import org.apache.commons.lang3.ArrayUtils;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.MapData;
+import org.apache.flink.table.data.RawValueData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
 import org.apache.iceberg.Schema;
-import org.apache.iceberg.data.parquet.BaseParquetReaders;
 import org.apache.iceberg.parquet.ParquetValueReader;
 import org.apache.iceberg.parquet.ParquetValueReaders;
+import org.apache.iceberg.parquet.TypeWithSchemaVisitor;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
 import org.apache.iceberg.types.Types;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.io.api.Binary;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.LogicalTypeAnnotation.DecimalLogicalTypeAnnotation;
 import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
 import org.apache.parquet.schema.Type;
 
-public class FlinkParquetReaders extends BaseParquetReaders<Row> {
+class FlinkParquetReaders {
+  private FlinkParquetReaders() {
+  }
 
-  private static final FlinkParquetReaders INSTANCE = new FlinkParquetReaders();
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema, MessageType fileSchema) {
+    return buildReader(expectedSchema, fileSchema, ImmutableMap.of());
+  }
 
-  private FlinkParquetReaders() {
+  @SuppressWarnings("unchecked")
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema,
+                                                        MessageType fileSchema,
+                                                        Map<Integer, ?> idToConstant) {
+    return (ParquetValueReader<RowData>) TypeWithSchemaVisitor.visit(expectedSchema.asStruct(), fileSchema,
+        new ReadBuilder(fileSchema, idToConstant)
+    );
+  }
+
+  private static class ReadBuilder extends TypeWithSchemaVisitor<ParquetValueReader<?>> {
+    private final MessageType type;
+    private final Map<Integer, ?> idToConstant;
+
+    ReadBuilder(MessageType type, Map<Integer, ?> idToConstant) {
+      this.type = type;
+      this.idToConstant = idToConstant;
+    }
+
+    @Override
+    public ParquetValueReader<RowData> message(Types.StructType expected, MessageType message,
+                                               List<ParquetValueReader<?>> fieldReaders) {
+      return struct(expected, message.asGroupType(), fieldReaders);
+    }
+
+    @Override
+    public ParquetValueReader<RowData> struct(Types.StructType expected, GroupType struct,
+                                              List<ParquetValueReader<?>> fieldReaders) {
+      // match the expected struct's order
+      Map<Integer, ParquetValueReader<?>> readersById = Maps.newHashMap();
+      Map<Integer, Type> typesById = Maps.newHashMap();
+      List<Type> fields = struct.getFields();
+      for (int i = 0; i < fields.size(); i += 1) {
+        Type fieldType = fields.get(i);
+        int fieldD = type.getMaxDefinitionLevel(path(fieldType.getName())) - 1;
+        if (fieldType.getId() != null) {
+          int id = fieldType.getId().intValue();
+          readersById.put(id, ParquetValueReaders.option(fieldType, fieldD, fieldReaders.get(i)));
+          typesById.put(id, fieldType);
+        }
+      }
+
+      List<Types.NestedField> expectedFields = expected != null ?
+          expected.fields() : ImmutableList.of();
+      List<ParquetValueReader<?>> reorderedFields = Lists.newArrayListWithExpectedSize(
+          expectedFields.size());
+      List<Type> types = Lists.newArrayListWithExpectedSize(expectedFields.size());
+      for (Types.NestedField field : expectedFields) {
+        int id = field.fieldId();
+        if (idToConstant.containsKey(id)) {
+          // containsKey is used because the constant may be null
+          reorderedFields.add(ParquetValueReaders.constant(idToConstant.get(id)));
+          types.add(null);
+        } else {
+          ParquetValueReader<?> reader = readersById.get(id);
+          if (reader != null) {
+            reorderedFields.add(reader);
+            types.add(typesById.get(id));
+          } else {
+            reorderedFields.add(ParquetValueReaders.nulls());
+            types.add(null);
+          }
+        }
+      }
+
+      return new RowDataReader(types, reorderedFields);
+    }
+
+    @Override
+    public ParquetValueReader<?> list(Types.ListType expectedList, GroupType array,
+                                      ParquetValueReader<?> elementReader) {
+      GroupType repeated = array.getFields().get(0).asGroupType();
+      String[] repeatedPath = currentPath();
+
+      int repeatedD = type.getMaxDefinitionLevel(repeatedPath) - 1;
+      int repeatedR = type.getMaxRepetitionLevel(repeatedPath) - 1;
+
+      Type elementType = repeated.getType(0);
+      int elementD = type.getMaxDefinitionLevel(path(elementType.getName())) - 1;
+
+      return new ArrayReader<>(repeatedD, repeatedR, ParquetValueReaders.option(elementType, elementD, elementReader));
+    }
+
+    @Override
+    public ParquetValueReader<?> map(Types.MapType expectedMap, GroupType map,
+                                     ParquetValueReader<?> keyReader,
+                                     ParquetValueReader<?> valueReader) {
+      GroupType repeatedKeyValue = map.getFields().get(0).asGroupType();
+      String[] repeatedPath = currentPath();
+
+      int repeatedD = type.getMaxDefinitionLevel(repeatedPath) - 1;
+      int repeatedR = type.getMaxRepetitionLevel(repeatedPath) - 1;
+
+      Type keyType = repeatedKeyValue.getType(0);
+      int keyD = type.getMaxDefinitionLevel(path(keyType.getName())) - 1;
+      Type valueType = repeatedKeyValue.getType(1);
+      int valueD = type.getMaxDefinitionLevel(path(valueType.getName())) - 1;
+
+      return new MapReader<>(repeatedD, repeatedR,
+          ParquetValueReaders.option(keyType, keyD, keyReader),
+          ParquetValueReaders.option(valueType, valueD, valueReader));
+    }
+
+    @Override
+    public ParquetValueReader<?> primitive(org.apache.iceberg.types.Type.PrimitiveType expected,
+                                           PrimitiveType primitive) {
+      ColumnDescriptor desc = type.getColumnDescription(currentPath());
+
+      if (primitive.getOriginalType() != null) {
+        switch (primitive.getOriginalType()) {
+          case ENUM:
+          case JSON:
+          case UTF8:
+            return new StringReader(desc);
+          case INT_8:
+          case INT_16:
+          case INT_32:
+            if (expected != null && expected.typeId() == Types.LongType.get().typeId()) {
+              return new ParquetValueReaders.IntAsLongReader(desc);
+            } else {
+              return new ParquetValueReaders.UnboxedReader<>(desc);
+            }
+          case TIME_MICROS:
+            return new LossyMicrosToMillisTimeReader(desc);
+          case DATE:
+          case INT_64:
+            return new ParquetValueReaders.UnboxedReader<>(desc);
+          case TIMESTAMP_MICROS:

Review comment:
       Added.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] chenjunjiedada commented on a change in pull request #1266: Flink: update parquet reader with schema visitor

Posted by GitBox <gi...@apache.org>.
chenjunjiedada commented on a change in pull request #1266:
URL: https://github.com/apache/iceberg/pull/1266#discussion_r463627281



##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java
##########
@@ -19,64 +19,719 @@
 
 package org.apache.iceberg.flink.data;
 
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.time.Instant;
 import java.util.List;
-import org.apache.flink.types.Row;
+import java.util.Map;
+import org.apache.commons.lang3.ArrayUtils;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.MapData;
+import org.apache.flink.table.data.RawValueData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
 import org.apache.iceberg.Schema;
-import org.apache.iceberg.data.parquet.BaseParquetReaders;
+import org.apache.iceberg.parquet.ParquetSchemaUtil;
 import org.apache.iceberg.parquet.ParquetValueReader;
 import org.apache.iceberg.parquet.ParquetValueReaders;
+import org.apache.iceberg.parquet.TypeWithSchemaVisitor;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
 import org.apache.iceberg.types.Types;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.io.api.Binary;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.LogicalTypeAnnotation.DecimalLogicalTypeAnnotation;
 import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
 import org.apache.parquet.schema.Type;
 
-public class FlinkParquetReaders extends BaseParquetReaders<Row> {
+class FlinkParquetReaders {
+  private FlinkParquetReaders() {
+  }
 
-  private static final FlinkParquetReaders INSTANCE = new FlinkParquetReaders();
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema, MessageType fileSchema) {
+    return buildReader(expectedSchema, fileSchema, ImmutableMap.of());
+  }
 
-  private FlinkParquetReaders() {
+  @SuppressWarnings("unchecked")
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema,
+                                                        MessageType fileSchema,
+                                                        Map<Integer, ?> idToConstant) {
+    ReadBuilder builder = new ReadBuilder(fileSchema, idToConstant);
+    if (ParquetSchemaUtil.hasIds(fileSchema)) {
+      return (ParquetValueReader<RowData>)
+          TypeWithSchemaVisitor.visit(expectedSchema.asStruct(), fileSchema, builder);
+    } else {
+      return (ParquetValueReader<RowData>)
+          TypeWithSchemaVisitor.visit(expectedSchema.asStruct(), fileSchema,
+              new FallbackReadBuilder(builder));
+    }
+  }
+
+  private static class FallbackReadBuilder extends TypeWithSchemaVisitor<ParquetValueReader<?>> {
+    private MessageType type;
+    private final TypeWithSchemaVisitor<ParquetValueReader<?>> builder;
+
+    FallbackReadBuilder(TypeWithSchemaVisitor<ParquetValueReader<?>> builder) {
+      this.builder = builder;
+    }
+
+    @Override
+    public ParquetValueReader<?> message(Types.StructType expected, MessageType message,
+                                         List<ParquetValueReader<?>> fieldReaders) {
+      // the top level matches by ID, but the remaining IDs are missing
+      this.type = message;
+      return builder.struct(expected, message, fieldReaders);
+    }
+
+    @Override
+    public ParquetValueReader<?> struct(Types.StructType ignored, GroupType struct,
+                                        List<ParquetValueReader<?>> fieldReaders) {
+      // the expected struct is ignored because nested fields are never found when the IDs are missing
+      List<ParquetValueReader<?>> newFields = Lists.newArrayListWithExpectedSize(
+          fieldReaders.size());
+      List<Type> types = Lists.newArrayListWithExpectedSize(fieldReaders.size());
+      List<Type> fields = struct.getFields();
+      for (int i = 0; i < fields.size(); i += 1) {
+        Type fieldType = fields.get(i);
+        int fieldD = type.getMaxDefinitionLevel(path(fieldType.getName())) - 1;
+        newFields.add(ParquetValueReaders.option(fieldType, fieldD, fieldReaders.get(i)));
+        types.add(fieldType);
+      }
+
+      return new RowDataReader(types, newFields);
+    }
+  }
+
+  private static class ReadBuilder extends TypeWithSchemaVisitor<ParquetValueReader<?>> {
+    private final MessageType type;
+    private final Map<Integer, ?> idToConstant;
+
+    ReadBuilder(MessageType type, Map<Integer, ?> idToConstant) {
+      this.type = type;
+      this.idToConstant = idToConstant;
+    }
+
+    @Override
+    public ParquetValueReader<?> message(Types.StructType expected, MessageType message,
+                                         List<ParquetValueReader<?>> fieldReaders) {
+      return struct(expected, message.asGroupType(), fieldReaders);
+    }
+
+    @Override
+    public ParquetValueReader<?> struct(Types.StructType expected, GroupType struct,
+                                        List<ParquetValueReader<?>> fieldReaders) {
+      // match the expected struct's order
+      Map<Integer, ParquetValueReader<?>> readersById = Maps.newHashMap();
+      Map<Integer, Type> typesById = Maps.newHashMap();
+      List<Type> fields = struct.getFields();
+      for (int i = 0; i < fields.size(); i += 1) {
+        Type fieldType = fields.get(i);
+        int fieldD = type.getMaxDefinitionLevel(path(fieldType.getName())) - 1;
+        if (fieldType.getId() != null) {
+          int id = fieldType.getId().intValue();
+          readersById.put(id, ParquetValueReaders.option(fieldType, fieldD, fieldReaders.get(i)));
+          typesById.put(id, fieldType);
+        }
+      }
+
+      List<Types.NestedField> expectedFields = expected != null ?
+          expected.fields() : ImmutableList.of();
+      List<ParquetValueReader<?>> reorderedFields = Lists.newArrayListWithExpectedSize(
+          expectedFields.size());
+      List<Type> types = Lists.newArrayListWithExpectedSize(expectedFields.size());
+      for (Types.NestedField field : expectedFields) {
+        int id = field.fieldId();
+        if (idToConstant.containsKey(id)) {
+          // containsKey is used because the constant may be null
+          reorderedFields.add(ParquetValueReaders.constant(idToConstant.get(id)));
+          types.add(null);
+        } else {
+          ParquetValueReader<?> reader = readersById.get(id);
+          if (reader != null) {
+            reorderedFields.add(reader);
+            types.add(typesById.get(id));
+          } else {
+            reorderedFields.add(ParquetValueReaders.nulls());
+            types.add(null);
+          }
+        }
+      }
+
+      return new RowDataReader(types, reorderedFields);
+    }
+
+    @Override
+    public ParquetValueReader<?> list(Types.ListType expectedList, GroupType array,
+                                      ParquetValueReader<?> elementReader) {
+      GroupType repeated = array.getFields().get(0).asGroupType();
+      String[] repeatedPath = currentPath();
+
+      int repeatedD = type.getMaxDefinitionLevel(repeatedPath) - 1;
+      int repeatedR = type.getMaxRepetitionLevel(repeatedPath) - 1;
+
+      Type elementType = repeated.getType(0);
+      int elementD = type.getMaxDefinitionLevel(path(elementType.getName())) - 1;
+
+      return new ArrayReader<>(repeatedD, repeatedR, ParquetValueReaders.option(elementType, elementD, elementReader));
+    }
+
+    @Override
+    public ParquetValueReader<?> map(Types.MapType expectedMap, GroupType map,
+                                     ParquetValueReader<?> keyReader,
+                                     ParquetValueReader<?> valueReader) {
+      GroupType repeatedKeyValue = map.getFields().get(0).asGroupType();
+      String[] repeatedPath = currentPath();
+
+      int repeatedD = type.getMaxDefinitionLevel(repeatedPath) - 1;
+      int repeatedR = type.getMaxRepetitionLevel(repeatedPath) - 1;
+
+      Type keyType = repeatedKeyValue.getType(0);
+      int keyD = type.getMaxDefinitionLevel(path(keyType.getName())) - 1;
+      Type valueType = repeatedKeyValue.getType(1);
+      int valueD = type.getMaxDefinitionLevel(path(valueType.getName())) - 1;
+
+      return new MapReader<>(repeatedD, repeatedR,
+          ParquetValueReaders.option(keyType, keyD, keyReader),
+          ParquetValueReaders.option(valueType, valueD, valueReader));
+    }
+
+    @Override
+    public ParquetValueReader<?> primitive(org.apache.iceberg.types.Type.PrimitiveType expected,
+                                           PrimitiveType primitive) {
+      ColumnDescriptor desc = type.getColumnDescription(currentPath());
+
+      if (primitive.getOriginalType() != null) {
+        switch (primitive.getOriginalType()) {
+          case ENUM:
+          case JSON:
+          case UTF8:
+            return new StringReader(desc);
+          case INT_8:
+          case INT_16:
+          case INT_32:
+          case DATE:
+            if (expected != null && expected.typeId() == Types.LongType.get().typeId()) {
+              return new ParquetValueReaders.IntAsLongReader(desc);
+            } else {
+              return new ParquetValueReaders.UnboxedReader<>(desc);
+            }
+          case TIME_MICROS:
+            return new TimeMillisReader(desc);
+          case INT_64:
+            return new ParquetValueReaders.UnboxedReader<>(desc);
+          case TIMESTAMP_MICROS:
+            return new TimestampMicroReader(desc);
+          case DECIMAL:
+            DecimalLogicalTypeAnnotation decimal = (DecimalLogicalTypeAnnotation) primitive.getLogicalTypeAnnotation();
+            switch (primitive.getPrimitiveTypeName()) {
+              case BINARY:
+              case FIXED_LEN_BYTE_ARRAY:
+                return new BinaryDecimalReader(desc, decimal.getScale());
+              case INT64:
+                return new LongDecimalReader(desc, decimal.getPrecision(), decimal.getScale());
+              case INT32:
+                return new IntegerDecimalReader(desc, decimal.getPrecision(), decimal.getScale());
+              default:
+                throw new UnsupportedOperationException(
+                    "Unsupported base type for decimal: " + primitive.getPrimitiveTypeName());
+            }
+          case BSON:
+            return new ParquetValueReaders.ByteArrayReader(desc);
+          default:
+            throw new UnsupportedOperationException(
+                "Unsupported logical type: " + primitive.getOriginalType());
+        }
+      }
+
+      switch (primitive.getPrimitiveTypeName()) {
+        case FIXED_LEN_BYTE_ARRAY:
+        case BINARY:
+          return new ParquetValueReaders.ByteArrayReader(desc);
+        case INT32:
+          if (expected != null && expected.typeId() == org.apache.iceberg.types.Type.TypeID.LONG) {
+            return new ParquetValueReaders.IntAsLongReader(desc);
+          } else {
+            return new ParquetValueReaders.UnboxedReader<>(desc);
+          }
+        case FLOAT:
+          if (expected != null && expected.typeId() == org.apache.iceberg.types.Type.TypeID.DOUBLE) {
+            return new ParquetValueReaders.FloatAsDoubleReader(desc);
+          } else {
+            return new ParquetValueReaders.UnboxedReader<>(desc);
+          }
+        case BOOLEAN:
+        case INT64:
+        case DOUBLE:
+          return new ParquetValueReaders.UnboxedReader<>(desc);
+        default:
+          throw new UnsupportedOperationException("Unsupported type: " + primitive);
+      }
+    }
+  }
+
+  private static class BinaryDecimalReader extends ParquetValueReaders.PrimitiveReader<DecimalData> {
+    private final int scale;
+
+    BinaryDecimalReader(ColumnDescriptor desc, int scale) {
+      super(desc);
+      this.scale = scale;
+    }
+
+    @Override
+    public DecimalData read(DecimalData ignored) {
+      Binary binary = column.nextBinary();
+      BigDecimal bigDecimal = new BigDecimal(new BigInteger(binary.getBytes()), scale);
+      return DecimalData.fromBigDecimal(bigDecimal, bigDecimal.precision(), scale);
+    }
+  }
+
+  private static class IntegerDecimalReader extends ParquetValueReaders.PrimitiveReader<DecimalData> {
+    private final int precision;
+    private final int scale;
+
+    IntegerDecimalReader(ColumnDescriptor desc, int precision, int scale) {
+      super(desc);
+      this.precision = precision;
+      this.scale = scale;
+    }
+
+    @Override
+    public DecimalData read(DecimalData ignored) {
+      return DecimalData.fromUnscaledLong(column.nextInteger(), precision, scale);
+    }
+  }
+
+  private static class LongDecimalReader extends ParquetValueReaders.PrimitiveReader<DecimalData> {
+    private final int precision;
+    private final int scale;
+
+    LongDecimalReader(ColumnDescriptor desc, int precision, int scale) {
+      super(desc);
+      this.precision = precision;
+      this.scale = scale;
+    }
+
+    @Override
+    public DecimalData read(DecimalData ignored) {
+      return DecimalData.fromUnscaledLong(column.nextLong(), precision, scale);
+    }
   }
 
-  public static ParquetValueReader<Row> buildReader(Schema expectedSchema, MessageType fileSchema) {
-    return INSTANCE.createReader(expectedSchema, fileSchema);
+  private static class TimestampMicroReader extends ParquetValueReaders.UnboxedReader<TimestampData> {
+    TimestampMicroReader(ColumnDescriptor desc) {
+      super(desc);
+    }
+
+    @Override
+    public TimestampData read(TimestampData ignored) {
+      long value = readLong();
+      return TimestampData.fromInstant(Instant.ofEpochSecond(value / 1000_000, (value % 1000_000) * 1000));

Review comment:
       Thanks for pointing this out. Updated this accordingly.
   




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1266: Flink: update parquet reader with schema visitor

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1266:
URL: https://github.com/apache/iceberg/pull/1266#discussion_r469417642



##########
File path: flink/src/test/java/org/apache/iceberg/flink/data/TestHelpers.java
##########
@@ -0,0 +1,198 @@
+/*
+ * 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.iceberg.flink.data;
+
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.OffsetDateTime;
+import java.time.ZoneOffset;
+import java.time.temporal.ChronoUnit;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import java.util.function.Supplier;
+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.MapType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+
+public class TestHelpers {
+  private TestHelpers() {}
+
+  private static final OffsetDateTime EPOCH = Instant.ofEpochMilli(0L).atOffset(ZoneOffset.UTC);
+  private static final LocalDate EPOCH_DAY = EPOCH.toLocalDate();
+
+  public static void assertRowData(Type type, LogicalType rowType, Record expectedRecord, RowData actualRowData) {
+    if (expectedRecord == null && actualRowData == null) {
+      return;
+    }
+
+    Assert.assertTrue("expected Record and actual RowData should be both null or not null",
+        expectedRecord != null && actualRowData != null);
+
+    List<Type> types = Lists.newArrayList();
+    for (Types.NestedField field : type.asStructType().fields()) {
+      types.add(field.type());
+    }
+
+    for (int i = 0; i < types.size(); i += 1) {
+      if (expectedRecord.get(i) == null) {
+        Assert.assertTrue(actualRowData.isNullAt(i));
+        continue;
+      }
+
+      Object expected = expectedRecord.get(i);
+      LogicalType logicalType = ((RowType) rowType).getTypeAt(i);
+
+      final int fieldPos = i;
+      assertEquals(types.get(i), logicalType, expected,
+          () -> RowData.createFieldGetter(logicalType, fieldPos).getFieldOrNull(actualRowData));
+    }
+  }
+
+  private static void assertEquals(Type type, LogicalType logicalType, Object expected, Supplier<Object> supplier) {

Review comment:
       Why call `supplier.get()` in every case when you could pass an object instead of a supplier?




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #1266: Flink: update parquet reader with schema visitor

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1266:
URL: https://github.com/apache/iceberg/pull/1266#discussion_r466807770



##########
File path: flink/src/test/java/org/apache/iceberg/flink/data/TestHelpers.java
##########
@@ -0,0 +1,271 @@
+/*
+ * 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.iceberg.flink.data;
+
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.OffsetDateTime;
+import java.time.ZoneOffset;
+import java.time.temporal.ChronoUnit;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.RowData;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+
+public class TestHelpers {
+  private TestHelpers() {}
+
+  private static final OffsetDateTime EPOCH = Instant.ofEpochMilli(0L).atOffset(ZoneOffset.UTC);
+  private static final LocalDate EPOCH_DAY = EPOCH.toLocalDate();
+
+  public static void assertRowData(Type type, Record expected, RowData actual) {
+    if (expected == null && actual == null) {
+      return;
+    }
+
+    List<Type> types = new ArrayList<>();
+    for (Types.NestedField field : type.asStructType().fields()) {
+      types.add(field.type());
+    }
+
+    for (int i = 0; i < types.size(); i += 1) {
+      if (expected.get(i) == null) {
+        Assert.assertTrue(actual.isNullAt(i));
+        continue;
+      }
+      Type.TypeID typeId = types.get(i).typeId();
+      Object value = expected.get(i);
+      switch (typeId) {
+        case BOOLEAN:
+          Assert.assertEquals("boolean value should be equal", value, actual.getBoolean(i));
+          break;
+        case INTEGER:
+          Assert.assertEquals("int value should be equal", value, actual.getInt(i));
+          break;
+        case LONG:
+          Assert.assertEquals("long value should be equal", value, actual.getLong(i));
+          break;
+        case FLOAT:
+          Assert.assertEquals("float value should be equal", value, actual.getFloat(i));
+          break;
+        case DOUBLE:
+          Assert.assertEquals("double should be equal", value, actual.getDouble(i));
+          break;
+        case STRING:
+          Assert.assertTrue("Should expect a CharSequence", value instanceof CharSequence);
+          Assert.assertEquals("string should be equal", String.valueOf(value), actual.getString(i).toString());
+          break;
+        case DATE:
+          Assert.assertTrue("Should expect a Date", value instanceof LocalDate);
+          LocalDate date = ChronoUnit.DAYS.addTo(EPOCH_DAY, actual.getInt(i));
+          Assert.assertEquals("date should be equal", value, date);
+          break;
+        case TIME:
+          Assert.assertTrue("Should expect a LocalTime", value instanceof LocalTime);
+          int milliseconds = (int) (((LocalTime) value).toNanoOfDay() / 1000_000);
+          Assert.assertEquals("time millis should be equal", milliseconds, actual.getInt(i));
+          break;
+        case TIMESTAMP:
+          if (((Types.TimestampType) type.asPrimitiveType()).shouldAdjustToUTC()) {
+            Assert.assertTrue("Should expect a OffsetDataTime", value instanceof OffsetDateTime);
+            OffsetDateTime ts = (OffsetDateTime) value;
+            Assert.assertEquals("OffsetDataTime should be equal", ts.toLocalDateTime(),
+                actual.getTimestamp(i, 6).toLocalDateTime());
+          } else {
+            Assert.assertTrue("Should expect a LocalDataTime", value instanceof LocalDateTime);
+            LocalDateTime ts = (LocalDateTime) value;
+            Assert.assertEquals("LocalDataTime should be equal", ts,
+                actual.getTimestamp(i, 6).toLocalDateTime());
+          }
+          break;
+        case FIXED:
+          Assert.assertTrue("Should expect byte[]", value instanceof byte[]);
+          Assert.assertArrayEquals("binary should be equal", (byte[]) value, actual.getBinary(i));
+          break;
+        case BINARY:
+          Assert.assertTrue("Should expect a ByteBuffer", value instanceof ByteBuffer);
+          Assert.assertArrayEquals("binary should be equal", ((ByteBuffer) value).array(), actual.getBinary(i));
+          break;
+        case DECIMAL:

Review comment:
       I tried to reuse those big switch-case,  How about the following way ? ( I did not finish all the code, but seems it should work to reuse that code). 
   
   ```java
   public class TestHelpers {
     private TestHelpers() {
     }
   
     private static final OffsetDateTime EPOCH = Instant.ofEpochMilli(0L).atOffset(ZoneOffset.UTC);
     private static final LocalDate EPOCH_DAY = EPOCH.toLocalDate();
   
     public static void assertRowData(Type type, RowType rowType, Record expectedRecord, RowData actualRowData) {
       if (expectedRecord == null && actualRowData == null) {
         return;
       }
   
       Assert.assertTrue("expected Record and actual RowData should be both null or not null",
           expectedRecord != null && actualRowData != null);
   
       List<Type> types = new ArrayList<>();
       for (Types.NestedField field : type.asStructType().fields()) {
         types.add(field.type());
       }
   
       for (int i = 0; i < types.size(); i += 1) {
         if (expectedRecord.get(i) == null) {
           Assert.assertTrue(actualRowData.isNullAt(i));
           continue;
         }
         Object expected = expectedRecord.get(i);
         LogicalType logicalType = rowType.getTypeAt(i);
   
         final int fieldPos = i;
         assertEquals(types.get(i), rowType.getTypeAt(i), expected,
             () -> RowData.createFieldGetter(logicalType, fieldPos).getFieldOrNull(actualRowData));
       }
     }
   
     private static void assertEquals(Type type, LogicalType logicalType, Object expected, Supplier<Object> supplier) {
       switch (type.typeId()) {
         case BOOLEAN:
           Assert.assertEquals("boolean value should be equal", expected, supplier.get());
           break;
         case INTEGER:
           Assert.assertEquals("int value should be equal", expected, supplier.get());
           break;
         case LONG:
           Assert.assertEquals("long value should be equal", expected, supplier.get());
           break;
         case FLOAT:
           Assert.assertEquals("float value should be equal", expected, supplier.get());
           break;
         case DOUBLE:
           Assert.assertEquals("double value should be equal", expected, supplier.get());
           break;
         case STRING:
           Assert.assertTrue("Should expect a CharSequence", expected instanceof CharSequence);
           Assert.assertEquals("string should be equal",
               String.valueOf(expected), supplier.get().toString());
           break;
         case DATE:
           Assert.assertTrue("Should expect a Date", expected instanceof LocalDate);
           LocalDate date = ChronoUnit.DAYS.addTo(EPOCH_DAY, (int) supplier.get());
           Assert.assertEquals("date should be equal", expected, date);
           break;
         case TIME:
           Assert.assertTrue("Should expect a LocalTime", expected instanceof LocalTime);
           int milliseconds = (int) (((LocalTime) expected).toNanoOfDay() / 1000_000);
           Assert.assertEquals("time millis should be equal", milliseconds, supplier.get());
           break;
         case TIMESTAMP:
           if (((Types.TimestampType) type.asPrimitiveType()).shouldAdjustToUTC()) {
             Assert.assertTrue("Should expect a OffsetDataTime", expected instanceof OffsetDateTime);
             OffsetDateTime ts = (OffsetDateTime) expected;
             Assert.assertEquals("OffsetDataTime should be equal", ts.toLocalDateTime(),
                 ((TimestampData) supplier.get()).toLocalDateTime());
           } else {
             Assert.assertTrue("Should expect a LocalDataTime", expected instanceof LocalDateTime);
             LocalDateTime ts = (LocalDateTime) expected;
             Assert.assertEquals("LocalDataTime should be equal", ts,
                 ((TimestampData) supplier.get()).toLocalDateTime());
           }
           break;
         case BINARY:
           Assert.assertTrue("Should expect a ByteBuffer", expected instanceof ByteBuffer);
           Assert.assertArrayEquals("binary should be equal", ((ByteBuffer) expected).array(), (byte[]) supplier.get());
           break;
         case DECIMAL:
           Assert.assertTrue("Should expect a BigDecimal", expected instanceof BigDecimal);
           BigDecimal bd = (BigDecimal) expected;
           Assert.assertEquals("decimal value should be equal", bd,
               ((DecimalData) supplier.get()).toBigDecimal());
           break;
         case LIST: // TODO call assertArrayValues.
         case MAP:
         case STRUCT:
         case UUID:
         case FIXED:
           // TODO;
         default:
           throw new IllegalArgumentException("Not a supported type: " + type);
       }
     }
   
     private static void assertArrayValues(Type type, LogicalType logicalType, Collection<?> expectedArray,
                                           ArrayData actualArray) {
       List<?> expectedElements = Lists.newArrayList(expectedArray);
       for (int i = 0; i < expectedArray.size(); i += 1) {
         if (expectedElements.get(i) == null) {
           Assert.assertTrue(actualArray.isNullAt(i));
           continue;
         }
   
         Object expected = expectedElements.get(i);
   
         final int pos = i;
         assertEquals(type, logicalType, expected,
             () -> ArrayData.createElementGetter(logicalType).getElementOrNull(actualArray, pos));
       }
     }
   }
   ```




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue merged pull request #1266: Flink: update parquet reader with schema visitor

Posted by GitBox <gi...@apache.org>.
rdblue merged pull request #1266:
URL: https://github.com/apache/iceberg/pull/1266


   


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] chenjunjiedada commented on a change in pull request #1266: Flink: update parquet reader with schema visitor

Posted by GitBox <gi...@apache.org>.
chenjunjiedada commented on a change in pull request #1266:
URL: https://github.com/apache/iceberg/pull/1266#discussion_r463625032



##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java
##########
@@ -19,64 +19,719 @@
 
 package org.apache.iceberg.flink.data;
 
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.time.Instant;
 import java.util.List;
-import org.apache.flink.types.Row;
+import java.util.Map;
+import org.apache.commons.lang3.ArrayUtils;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.MapData;
+import org.apache.flink.table.data.RawValueData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
 import org.apache.iceberg.Schema;
-import org.apache.iceberg.data.parquet.BaseParquetReaders;
+import org.apache.iceberg.parquet.ParquetSchemaUtil;
 import org.apache.iceberg.parquet.ParquetValueReader;
 import org.apache.iceberg.parquet.ParquetValueReaders;
+import org.apache.iceberg.parquet.TypeWithSchemaVisitor;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
 import org.apache.iceberg.types.Types;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.io.api.Binary;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.LogicalTypeAnnotation.DecimalLogicalTypeAnnotation;
 import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
 import org.apache.parquet.schema.Type;
 
-public class FlinkParquetReaders extends BaseParquetReaders<Row> {
+class FlinkParquetReaders {
+  private FlinkParquetReaders() {
+  }
 
-  private static final FlinkParquetReaders INSTANCE = new FlinkParquetReaders();
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema, MessageType fileSchema) {
+    return buildReader(expectedSchema, fileSchema, ImmutableMap.of());
+  }
 
-  private FlinkParquetReaders() {
+  @SuppressWarnings("unchecked")
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema,
+                                                        MessageType fileSchema,
+                                                        Map<Integer, ?> idToConstant) {
+    ReadBuilder builder = new ReadBuilder(fileSchema, idToConstant);
+    if (ParquetSchemaUtil.hasIds(fileSchema)) {
+      return (ParquetValueReader<RowData>)
+          TypeWithSchemaVisitor.visit(expectedSchema.asStruct(), fileSchema, builder);
+    } else {
+      return (ParquetValueReader<RowData>)
+          TypeWithSchemaVisitor.visit(expectedSchema.asStruct(), fileSchema,
+              new FallbackReadBuilder(builder));
+    }
+  }
+
+  private static class FallbackReadBuilder extends TypeWithSchemaVisitor<ParquetValueReader<?>> {
+    private MessageType type;
+    private final TypeWithSchemaVisitor<ParquetValueReader<?>> builder;
+
+    FallbackReadBuilder(TypeWithSchemaVisitor<ParquetValueReader<?>> builder) {
+      this.builder = builder;
+    }
+
+    @Override
+    public ParquetValueReader<?> message(Types.StructType expected, MessageType message,
+                                         List<ParquetValueReader<?>> fieldReaders) {
+      // the top level matches by ID, but the remaining IDs are missing
+      this.type = message;
+      return builder.struct(expected, message, fieldReaders);
+    }
+
+    @Override
+    public ParquetValueReader<?> struct(Types.StructType ignored, GroupType struct,

Review comment:
       You are right. Fixed.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] chenjunjiedada commented on a change in pull request #1266: Flink: update parquet reader with schema visitor

Posted by GitBox <gi...@apache.org>.
chenjunjiedada commented on a change in pull request #1266:
URL: https://github.com/apache/iceberg/pull/1266#discussion_r469699851



##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java
##########
@@ -19,64 +19,678 @@
 
 package org.apache.iceberg.flink.data;
 
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
 import java.util.List;
-import org.apache.flink.types.Row;
+import java.util.Map;
+import org.apache.commons.lang3.ArrayUtils;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.MapData;
+import org.apache.flink.table.data.RawValueData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
 import org.apache.iceberg.Schema;
-import org.apache.iceberg.data.parquet.BaseParquetReaders;
 import org.apache.iceberg.parquet.ParquetValueReader;
 import org.apache.iceberg.parquet.ParquetValueReaders;
+import org.apache.iceberg.parquet.TypeWithSchemaVisitor;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
 import org.apache.iceberg.types.Types;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.io.api.Binary;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.LogicalTypeAnnotation.DecimalLogicalTypeAnnotation;
 import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
 import org.apache.parquet.schema.Type;
 
-public class FlinkParquetReaders extends BaseParquetReaders<Row> {
+class FlinkParquetReaders {
+  private FlinkParquetReaders() {
+  }
 
-  private static final FlinkParquetReaders INSTANCE = new FlinkParquetReaders();
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema, MessageType fileSchema) {
+    return buildReader(expectedSchema, fileSchema, ImmutableMap.of());
+  }
 
-  private FlinkParquetReaders() {
+  @SuppressWarnings("unchecked")
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema,
+                                                        MessageType fileSchema,
+                                                        Map<Integer, ?> idToConstant) {
+    return (ParquetValueReader<RowData>) TypeWithSchemaVisitor.visit(expectedSchema.asStruct(), fileSchema,
+        new ReadBuilder(fileSchema, idToConstant)
+    );
+  }
+
+  private static class ReadBuilder extends TypeWithSchemaVisitor<ParquetValueReader<?>> {
+    private final MessageType type;
+    private final Map<Integer, ?> idToConstant;
+
+    ReadBuilder(MessageType type, Map<Integer, ?> idToConstant) {
+      this.type = type;
+      this.idToConstant = idToConstant;
+    }
+
+    @Override
+    public ParquetValueReader<RowData> message(Types.StructType expected, MessageType message,
+                                               List<ParquetValueReader<?>> fieldReaders) {
+      return struct(expected, message.asGroupType(), fieldReaders);
+    }
+
+    @Override
+    public ParquetValueReader<RowData> struct(Types.StructType expected, GroupType struct,
+                                              List<ParquetValueReader<?>> fieldReaders) {
+      // match the expected struct's order
+      Map<Integer, ParquetValueReader<?>> readersById = Maps.newHashMap();
+      Map<Integer, Type> typesById = Maps.newHashMap();
+      List<Type> fields = struct.getFields();
+      for (int i = 0; i < fields.size(); i += 1) {
+        Type fieldType = fields.get(i);
+        int fieldD = type.getMaxDefinitionLevel(path(fieldType.getName())) - 1;
+        if (fieldType.getId() != null) {
+          int id = fieldType.getId().intValue();
+          readersById.put(id, ParquetValueReaders.option(fieldType, fieldD, fieldReaders.get(i)));
+          typesById.put(id, fieldType);
+        }
+      }
+
+      List<Types.NestedField> expectedFields = expected != null ?
+          expected.fields() : ImmutableList.of();
+      List<ParquetValueReader<?>> reorderedFields = Lists.newArrayListWithExpectedSize(
+          expectedFields.size());
+      List<Type> types = Lists.newArrayListWithExpectedSize(expectedFields.size());
+      for (Types.NestedField field : expectedFields) {
+        int id = field.fieldId();
+        if (idToConstant.containsKey(id)) {
+          // containsKey is used because the constant may be null
+          reorderedFields.add(ParquetValueReaders.constant(idToConstant.get(id)));
+          types.add(null);
+        } else {
+          ParquetValueReader<?> reader = readersById.get(id);
+          if (reader != null) {
+            reorderedFields.add(reader);
+            types.add(typesById.get(id));
+          } else {
+            reorderedFields.add(ParquetValueReaders.nulls());
+            types.add(null);
+          }
+        }
+      }
+
+      return new RowDataReader(types, reorderedFields);
+    }
+
+    @Override
+    public ParquetValueReader<?> list(Types.ListType expectedList, GroupType array,
+                                      ParquetValueReader<?> elementReader) {
+      GroupType repeated = array.getFields().get(0).asGroupType();
+      String[] repeatedPath = currentPath();
+
+      int repeatedD = type.getMaxDefinitionLevel(repeatedPath) - 1;
+      int repeatedR = type.getMaxRepetitionLevel(repeatedPath) - 1;
+
+      Type elementType = repeated.getType(0);
+      int elementD = type.getMaxDefinitionLevel(path(elementType.getName())) - 1;
+
+      return new ArrayReader<>(repeatedD, repeatedR, ParquetValueReaders.option(elementType, elementD, elementReader));
+    }
+
+    @Override
+    public ParquetValueReader<?> map(Types.MapType expectedMap, GroupType map,
+                                     ParquetValueReader<?> keyReader,
+                                     ParquetValueReader<?> valueReader) {
+      GroupType repeatedKeyValue = map.getFields().get(0).asGroupType();
+      String[] repeatedPath = currentPath();
+
+      int repeatedD = type.getMaxDefinitionLevel(repeatedPath) - 1;
+      int repeatedR = type.getMaxRepetitionLevel(repeatedPath) - 1;
+
+      Type keyType = repeatedKeyValue.getType(0);
+      int keyD = type.getMaxDefinitionLevel(path(keyType.getName())) - 1;
+      Type valueType = repeatedKeyValue.getType(1);
+      int valueD = type.getMaxDefinitionLevel(path(valueType.getName())) - 1;
+
+      return new MapReader<>(repeatedD, repeatedR,
+          ParquetValueReaders.option(keyType, keyD, keyReader),
+          ParquetValueReaders.option(valueType, valueD, valueReader));
+    }
+
+    @Override
+    public ParquetValueReader<?> primitive(org.apache.iceberg.types.Type.PrimitiveType expected,
+                                           PrimitiveType primitive) {
+      ColumnDescriptor desc = type.getColumnDescription(currentPath());
+
+      if (primitive.getOriginalType() != null) {
+        switch (primitive.getOriginalType()) {
+          case ENUM:
+          case JSON:
+          case UTF8:
+            return new StringReader(desc);
+          case INT_8:
+          case INT_16:
+          case INT_32:
+            if (expected != null && expected.typeId() == Types.LongType.get().typeId()) {
+              return new ParquetValueReaders.IntAsLongReader(desc);
+            } else {
+              return new ParquetValueReaders.UnboxedReader<>(desc);
+            }
+          case TIME_MICROS:
+            return new LossyMicrosToMillisTimeReader(desc);
+          case DATE:
+          case INT_64:
+            return new ParquetValueReaders.UnboxedReader<>(desc);
+          case TIMESTAMP_MICROS:
+            return new TimestampMicrosReader(desc);
+          case DECIMAL:
+            DecimalLogicalTypeAnnotation decimal = (DecimalLogicalTypeAnnotation) primitive.getLogicalTypeAnnotation();
+            switch (primitive.getPrimitiveTypeName()) {
+              case BINARY:
+              case FIXED_LEN_BYTE_ARRAY:
+                return new BinaryDecimalReader(desc, decimal.getPrecision(), decimal.getScale());
+              case INT64:
+                return new LongDecimalReader(desc, decimal.getPrecision(), decimal.getScale());
+              case INT32:
+                return new IntegerDecimalReader(desc, decimal.getPrecision(), decimal.getScale());
+              default:
+                throw new UnsupportedOperationException(
+                    "Unsupported base type for decimal: " + primitive.getPrimitiveTypeName());
+            }
+          case BSON:
+            return new ParquetValueReaders.ByteArrayReader(desc);
+          default:
+            throw new UnsupportedOperationException(
+                "Unsupported logical type: " + primitive.getOriginalType());
+        }
+      }
+
+      switch (primitive.getPrimitiveTypeName()) {
+        case FIXED_LEN_BYTE_ARRAY:
+        case BINARY:
+          return new ParquetValueReaders.ByteArrayReader(desc);
+        case INT32:
+          if (expected != null && expected.typeId() == org.apache.iceberg.types.Type.TypeID.LONG) {
+            return new ParquetValueReaders.IntAsLongReader(desc);
+          } else {
+            return new ParquetValueReaders.UnboxedReader<>(desc);
+          }
+        case FLOAT:
+          if (expected != null && expected.typeId() == org.apache.iceberg.types.Type.TypeID.DOUBLE) {
+            return new ParquetValueReaders.FloatAsDoubleReader(desc);
+          } else {
+            return new ParquetValueReaders.UnboxedReader<>(desc);
+          }
+        case BOOLEAN:
+        case INT64:
+        case DOUBLE:
+          return new ParquetValueReaders.UnboxedReader<>(desc);
+        default:
+          throw new UnsupportedOperationException("Unsupported type: " + primitive);
+      }
+    }
+  }
+
+  private static class BinaryDecimalReader extends ParquetValueReaders.PrimitiveReader<DecimalData> {
+    private final int precision;
+    private final int scale;
+
+    BinaryDecimalReader(ColumnDescriptor desc, int precision, int scale) {
+      super(desc);
+      this.precision = precision;
+      this.scale = scale;
+    }
+
+    @Override
+    public DecimalData read(DecimalData ignored) {
+      Binary binary = column.nextBinary();
+      BigDecimal bigDecimal = new BigDecimal(new BigInteger(binary.getBytes()), scale);
+      // TODO: need a unit test to write-read-validate decimal via FlinkParquetWrite/Reader
+      return DecimalData.fromBigDecimal(bigDecimal, precision, scale);
+    }
+  }
+
+  private static class IntegerDecimalReader extends ParquetValueReaders.PrimitiveReader<DecimalData> {
+    private final int precision;
+    private final int scale;
+
+    IntegerDecimalReader(ColumnDescriptor desc, int precision, int scale) {
+      super(desc);
+      this.precision = precision;
+      this.scale = scale;
+    }
+
+    @Override
+    public DecimalData read(DecimalData ignored) {
+      return DecimalData.fromUnscaledLong(column.nextInteger(), precision, scale);
+    }
   }
 
-  public static ParquetValueReader<Row> buildReader(Schema expectedSchema, MessageType fileSchema) {
-    return INSTANCE.createReader(expectedSchema, fileSchema);
+  private static class LongDecimalReader extends ParquetValueReaders.PrimitiveReader<DecimalData> {
+    private final int precision;
+    private final int scale;
+
+    LongDecimalReader(ColumnDescriptor desc, int precision, int scale) {
+      super(desc);
+      this.precision = precision;
+      this.scale = scale;
+    }
+
+    @Override
+    public DecimalData read(DecimalData ignored) {
+      return DecimalData.fromUnscaledLong(column.nextLong(), precision, scale);
+    }
   }
 
-  @Override
-  protected ParquetValueReader<Row> createStructReader(List<Type> types,
-                                                       List<ParquetValueReader<?>> fieldReaders,
-                                                       Types.StructType structType) {
-    return new RowReader(types, fieldReaders, structType);
+  private static class TimestampMicrosReader extends ParquetValueReaders.UnboxedReader<TimestampData> {
+    TimestampMicrosReader(ColumnDescriptor desc) {
+      super(desc);
+    }
+
+    @Override
+    public TimestampData read(TimestampData ignored) {
+      long micros = readLong();
+      return TimestampData.fromEpochMillis(Math.floorDiv(micros, 1_000),
+          (int) Math.floorMod(micros, 1_000) * 1_000);
+    }
+
+    @Override
+    public long readLong() {
+      return column.nextLong();
+    }
+  }
+
+  private static class StringReader extends ParquetValueReaders.PrimitiveReader<StringData> {
+    StringReader(ColumnDescriptor desc) {
+      super(desc);
+    }
+
+    @Override
+    public StringData read(StringData ignored) {
+      Binary binary = column.nextBinary();
+      ByteBuffer buffer = binary.toByteBuffer();
+      if (buffer.hasArray()) {
+        return StringData.fromBytes(
+            buffer.array(), buffer.arrayOffset() + buffer.position(), buffer.remaining());
+      } else {
+        return StringData.fromBytes(binary.getBytes());
+      }
+    }
+  }
+
+  private static class LossyMicrosToMillisTimeReader extends ParquetValueReaders.PrimitiveReader<Integer> {
+    LossyMicrosToMillisTimeReader(ColumnDescriptor desc) {
+      super(desc);
+    }
+
+    @Override
+    public Integer read(Integer reuse) {
+      // Discard microseconds since Flink uses millisecond unit for TIME type.
+      return (int) Math.floorDiv(column.nextLong(), 1000);
+    }
+  }
+
+  private static class ArrayReader<E> extends ParquetValueReaders.RepeatedReader<ArrayData, ReusableArrayData, E> {
+    private int readPos = 0;
+    private int writePos = 0;
+
+    ArrayReader(int definitionLevel, int repetitionLevel, ParquetValueReader<E> reader) {
+      super(definitionLevel, repetitionLevel, reader);
+    }
+
+    @Override
+    protected ReusableArrayData newListData(ArrayData reuse) {
+      this.readPos = 0;
+      this.writePos = 0;
+
+      if (reuse instanceof ReusableArrayData) {
+        return (ReusableArrayData) reuse;
+      } else {
+        return new ReusableArrayData();
+      }
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    protected E getElement(ReusableArrayData list) {
+      E value = null;
+      if (readPos < list.capacity()) {
+        value = (E) list.values[readPos];
+      }
+
+      readPos += 1;
+
+      return value;
+    }
+
+    @Override
+    protected void addElement(ReusableArrayData reused, E element) {
+      if (writePos >= reused.capacity()) {
+        reused.grow();
+      }
+
+      reused.values[writePos] = element;
+
+      writePos += 1;
+    }
+
+    @Override
+    protected ArrayData buildList(ReusableArrayData list) {
+      list.setNumElements(writePos);
+      return list;
+    }
+  }
+
+  private static class MapReader<K, V> extends
+      ParquetValueReaders.RepeatedKeyValueReader<MapData, ReusableMapData, K, V> {
+    private int readPos = 0;
+    private int writePos = 0;
+
+    private final ParquetValueReaders.ReusableEntry<K, V> entry = new ParquetValueReaders.ReusableEntry<>();
+    private final ParquetValueReaders.ReusableEntry<K, V> nullEntry = new ParquetValueReaders.ReusableEntry<>();
+
+    MapReader(int definitionLevel, int repetitionLevel,
+              ParquetValueReader<K> keyReader, ParquetValueReader<V> valueReader) {
+      super(definitionLevel, repetitionLevel, keyReader, valueReader);
+    }
+
+    @Override
+    protected ReusableMapData newMapData(MapData reuse) {
+      this.readPos = 0;
+      this.writePos = 0;
+
+      if (reuse instanceof ReusableMapData) {
+        return (ReusableMapData) reuse;
+      } else {
+        return new ReusableMapData();
+      }
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    protected Map.Entry<K, V> getPair(ReusableMapData map) {
+      Map.Entry<K, V> kv = nullEntry;
+      if (readPos < map.capacity()) {
+        entry.set((K) map.keys.values[readPos], (V) map.values.values[readPos]);
+        kv = entry;
+      }
+
+      readPos += 1;
+
+      return kv;
+    }
+
+    @Override
+    protected void addPair(ReusableMapData map, K key, V value) {
+      if (writePos >= map.capacity()) {
+        map.grow();
+      }
+
+      map.keys.values[writePos] = key;
+      map.values.values[writePos] = value;
+
+      writePos += 1;
+    }
+
+    @Override
+    protected MapData buildMap(ReusableMapData map) {
+      map.setNumElements(writePos);
+      return map;
+    }
   }
 
-  private static class RowReader extends ParquetValueReaders.StructReader<Row, Row> {
-    private final Types.StructType structType;
+  private static class RowDataReader extends ParquetValueReaders.StructReader<RowData, GenericRowData> {
+    private final int numFields;
 
-    RowReader(List<Type> types, List<ParquetValueReader<?>> readers, Types.StructType struct) {
+    RowDataReader(List<Type> types, List<ParquetValueReader<?>> readers) {
       super(types, readers);
-      this.structType = struct;
+      this.numFields = readers.size();
     }
 
     @Override
-    protected Row newStructData(Row reuse) {
-      if (reuse != null) {
-        return reuse;
+    protected GenericRowData newStructData(RowData reuse) {
+      if (reuse instanceof GenericRowData) {
+        return (GenericRowData) reuse;
       } else {
-        return new Row(structType.fields().size());
+        return new GenericRowData(numFields);
       }
     }
 
     @Override
-    protected Object getField(Row row, int pos) {
-      return row.getField(pos);
+    protected Object getField(GenericRowData intermediate, int pos) {
+      return intermediate.getField(pos);
+    }
+
+    @Override
+    protected RowData buildStruct(GenericRowData struct) {
+      return struct;
+    }
+
+    @Override
+    protected void set(GenericRowData row, int pos, Object value) {
+      row.setField(pos, value);
+    }
+
+    @Override
+    protected void setNull(GenericRowData row, int pos) {
+      row.setField(pos, null);
+    }
+
+    @Override
+    protected void setBoolean(GenericRowData row, int pos, boolean value) {
+      row.setField(pos, value);
+    }
+
+    @Override
+    protected void setInteger(GenericRowData row, int pos, int value) {
+      row.setField(pos, value);
+    }
+
+    @Override
+    protected void setLong(GenericRowData row, int pos, long value) {
+      row.setField(pos, value);
     }
 
     @Override
-    protected Row buildStruct(Row row) {
-      return row;
+    protected void setFloat(GenericRowData row, int pos, float value) {
+      row.setField(pos, value);
     }
 
     @Override
-    protected void set(Row row, int pos, Object value) {
+    protected void setDouble(GenericRowData row, int pos, double value) {
       row.setField(pos, value);
     }
   }
+
+  private static class ReusableMapData implements MapData {

Review comment:
       Opened https://github.com/apache/iceberg/issues/1331.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #1266: Flink: update parquet reader with schema visitor

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1266:
URL: https://github.com/apache/iceberg/pull/1266#discussion_r464911911



##########
File path: data/src/test/java/org/apache/iceberg/data/RandomGenericData.java
##########
@@ -55,8 +55,8 @@ private RandomGenericData() {}
     return records;
   }
 
-  private static class RandomRecordGenerator extends RandomDataGenerator<Record> {
-    private RandomRecordGenerator(long seed) {
+  public static class RandomRecordGenerator extends RandomDataGenerator<Record> {

Review comment:
       This don't have to be public, see comment https://github.com/apache/iceberg/pull/1266/files#r464903356




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] chenjunjiedada commented on pull request #1266: Flink: update parquet reader with schema visitor

Posted by GitBox <gi...@apache.org>.
chenjunjiedada commented on pull request #1266:
URL: https://github.com/apache/iceberg/pull/1266#issuecomment-668341653


   @openinx @rdblue, would you please help to take another look? Just removed the `FallbackReaderBuilder` and renamed the confusing variable.


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] chenjunjiedada commented on a change in pull request #1266: Flink: update parquet reader with schema visitor

Posted by GitBox <gi...@apache.org>.
chenjunjiedada commented on a change in pull request #1266:
URL: https://github.com/apache/iceberg/pull/1266#discussion_r466905629



##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java
##########
@@ -19,64 +19,701 @@
 
 package org.apache.iceberg.flink.data;
 
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.time.Instant;
+import java.time.ZoneOffset;
 import java.util.List;
-import org.apache.flink.types.Row;
+import java.util.Map;
+import org.apache.commons.lang3.ArrayUtils;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.MapData;
+import org.apache.flink.table.data.RawValueData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
 import org.apache.iceberg.Schema;
-import org.apache.iceberg.data.parquet.BaseParquetReaders;
 import org.apache.iceberg.parquet.ParquetValueReader;
 import org.apache.iceberg.parquet.ParquetValueReaders;
+import org.apache.iceberg.parquet.TypeWithSchemaVisitor;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
 import org.apache.iceberg.types.Types;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.io.api.Binary;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.LogicalTypeAnnotation.DecimalLogicalTypeAnnotation;
 import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
 import org.apache.parquet.schema.Type;
 
-public class FlinkParquetReaders extends BaseParquetReaders<Row> {
+class FlinkParquetReaders {
+  private FlinkParquetReaders() {
+  }
 
-  private static final FlinkParquetReaders INSTANCE = new FlinkParquetReaders();
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema, MessageType fileSchema) {
+    return buildReader(expectedSchema, fileSchema, ImmutableMap.of());
+  }
 
-  private FlinkParquetReaders() {
+  @SuppressWarnings("unchecked")
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema,
+                                                        MessageType fileSchema,
+                                                        Map<Integer, ?> idToConstant) {
+    return (ParquetValueReader<RowData>) TypeWithSchemaVisitor.visit(expectedSchema.asStruct(), fileSchema,
+        new ReadBuilder(fileSchema, idToConstant)
+    );
+  }
+
+  private static class ReadBuilder extends TypeWithSchemaVisitor<ParquetValueReader<?>> {
+    private final MessageType type;
+    private final Map<Integer, ?> idToConstant;
+
+    ReadBuilder(MessageType type, Map<Integer, ?> idToConstant) {
+      this.type = type;
+      this.idToConstant = idToConstant;
+    }
+
+    @Override
+    public ParquetValueReader<RowData> message(Types.StructType expected, MessageType message,
+                                               List<ParquetValueReader<?>> fieldReaders) {
+      return struct(expected, message.asGroupType(), fieldReaders);
+    }
+
+    @Override
+    public ParquetValueReader<RowData> struct(Types.StructType expected, GroupType struct,
+                                              List<ParquetValueReader<?>> fieldReaders) {
+      // match the expected struct's order
+      Map<Integer, ParquetValueReader<?>> readersById = Maps.newHashMap();
+      Map<Integer, Type> typesById = Maps.newHashMap();
+      List<Type> fields = struct.getFields();
+      for (int i = 0; i < fields.size(); i += 1) {
+        Type fieldType = fields.get(i);
+        int fieldD = type.getMaxDefinitionLevel(path(fieldType.getName())) - 1;
+        if (fieldType.getId() != null) {
+          int id = fieldType.getId().intValue();
+          readersById.put(id, ParquetValueReaders.option(fieldType, fieldD, fieldReaders.get(i)));
+          typesById.put(id, fieldType);
+        }
+      }
+
+      List<Types.NestedField> expectedFields = expected != null ?
+          expected.fields() : ImmutableList.of();
+      List<ParquetValueReader<?>> reorderedFields = Lists.newArrayListWithExpectedSize(
+          expectedFields.size());
+      List<Type> types = Lists.newArrayListWithExpectedSize(expectedFields.size());
+      for (Types.NestedField field : expectedFields) {
+        int id = field.fieldId();
+        if (idToConstant.containsKey(id)) {
+          // containsKey is used because the constant may be null
+          reorderedFields.add(ParquetValueReaders.constant(idToConstant.get(id)));
+          types.add(null);
+        } else {
+          ParquetValueReader<?> reader = readersById.get(id);
+          if (reader != null) {
+            reorderedFields.add(reader);
+            types.add(typesById.get(id));
+          } else {
+            reorderedFields.add(ParquetValueReaders.nulls());
+            types.add(null);
+          }
+        }
+      }
+
+      return new RowDataReader(types, reorderedFields);
+    }
+
+    @Override
+    public ParquetValueReader<?> list(Types.ListType expectedList, GroupType array,
+                                      ParquetValueReader<?> elementReader) {
+      GroupType repeated = array.getFields().get(0).asGroupType();
+      String[] repeatedPath = currentPath();
+
+      int repeatedD = type.getMaxDefinitionLevel(repeatedPath) - 1;
+      int repeatedR = type.getMaxRepetitionLevel(repeatedPath) - 1;
+
+      Type elementType = repeated.getType(0);
+      int elementD = type.getMaxDefinitionLevel(path(elementType.getName())) - 1;
+
+      return new ArrayReader<>(repeatedD, repeatedR, ParquetValueReaders.option(elementType, elementD, elementReader));
+    }
+
+    @Override
+    public ParquetValueReader<?> map(Types.MapType expectedMap, GroupType map,
+                                     ParquetValueReader<?> keyReader,
+                                     ParquetValueReader<?> valueReader) {
+      GroupType repeatedKeyValue = map.getFields().get(0).asGroupType();
+      String[] repeatedPath = currentPath();
+
+      int repeatedD = type.getMaxDefinitionLevel(repeatedPath) - 1;
+      int repeatedR = type.getMaxRepetitionLevel(repeatedPath) - 1;
+
+      Type keyType = repeatedKeyValue.getType(0);
+      int keyD = type.getMaxDefinitionLevel(path(keyType.getName())) - 1;
+      Type valueType = repeatedKeyValue.getType(1);
+      int valueD = type.getMaxDefinitionLevel(path(valueType.getName())) - 1;
+
+      return new MapReader<>(repeatedD, repeatedR,
+          ParquetValueReaders.option(keyType, keyD, keyReader),
+          ParquetValueReaders.option(valueType, valueD, valueReader));
+    }
+
+    @Override
+    public ParquetValueReader<?> primitive(org.apache.iceberg.types.Type.PrimitiveType expected,
+                                           PrimitiveType primitive) {
+      ColumnDescriptor desc = type.getColumnDescription(currentPath());
+
+      if (primitive.getOriginalType() != null) {
+        switch (primitive.getOriginalType()) {
+          case ENUM:
+          case JSON:
+          case UTF8:
+            return new StringReader(desc);
+          case INT_8:
+          case INT_16:
+          case INT_32:
+          case DATE:
+            if (expected != null && expected.typeId() == Types.LongType.get().typeId()) {
+              return new ParquetValueReaders.IntAsLongReader(desc);
+            } else {
+              return new ParquetValueReaders.UnboxedReader<>(desc);
+            }
+          case TIME_MICROS:
+            return new LossyMicrosToMillisTimeReader(desc);
+          case INT_64:
+            return new ParquetValueReaders.UnboxedReader<>(desc);
+          case TIMESTAMP_MICROS:
+            if (((Types.TimestampType) expected).shouldAdjustToUTC()) {
+              return new TimestampTzReader(desc);
+            } else {
+              return new TimestampReader(desc);
+            }
+          case DECIMAL:
+            DecimalLogicalTypeAnnotation decimal = (DecimalLogicalTypeAnnotation) primitive.getLogicalTypeAnnotation();
+            switch (primitive.getPrimitiveTypeName()) {
+              case BINARY:
+              case FIXED_LEN_BYTE_ARRAY:
+                return new BinaryDecimalReader(desc, decimal.getScale());
+              case INT64:
+                return new LongDecimalReader(desc, decimal.getPrecision(), decimal.getScale());
+              case INT32:
+                return new IntegerDecimalReader(desc, decimal.getPrecision(), decimal.getScale());
+              default:
+                throw new UnsupportedOperationException(
+                    "Unsupported base type for decimal: " + primitive.getPrimitiveTypeName());
+            }
+          case BSON:
+            return new ParquetValueReaders.ByteArrayReader(desc);
+          default:
+            throw new UnsupportedOperationException(
+                "Unsupported logical type: " + primitive.getOriginalType());
+        }
+      }
+
+      switch (primitive.getPrimitiveTypeName()) {
+        case FIXED_LEN_BYTE_ARRAY:
+        case BINARY:
+          return new ParquetValueReaders.ByteArrayReader(desc);
+        case INT32:
+          if (expected != null && expected.typeId() == org.apache.iceberg.types.Type.TypeID.LONG) {
+            return new ParquetValueReaders.IntAsLongReader(desc);
+          } else {
+            return new ParquetValueReaders.UnboxedReader<>(desc);
+          }
+        case FLOAT:
+          if (expected != null && expected.typeId() == org.apache.iceberg.types.Type.TypeID.DOUBLE) {
+            return new ParquetValueReaders.FloatAsDoubleReader(desc);
+          } else {
+            return new ParquetValueReaders.UnboxedReader<>(desc);
+          }
+        case BOOLEAN:
+        case INT64:
+        case DOUBLE:
+          return new ParquetValueReaders.UnboxedReader<>(desc);
+        default:
+          throw new UnsupportedOperationException("Unsupported type: " + primitive);
+      }
+    }
+  }
+
+  private static class BinaryDecimalReader extends ParquetValueReaders.PrimitiveReader<DecimalData> {
+    private final int scale;
+
+    BinaryDecimalReader(ColumnDescriptor desc, int scale) {
+      super(desc);
+      this.scale = scale;
+    }
+
+    @Override
+    public DecimalData read(DecimalData ignored) {
+      Binary binary = column.nextBinary();
+      BigDecimal bigDecimal = new BigDecimal(new BigInteger(binary.getBytes()), scale);
+      return DecimalData.fromBigDecimal(bigDecimal, bigDecimal.precision(), scale);
+    }
+  }
+
+  private static class IntegerDecimalReader extends ParquetValueReaders.PrimitiveReader<DecimalData> {
+    private final int precision;
+    private final int scale;
+
+    IntegerDecimalReader(ColumnDescriptor desc, int precision, int scale) {
+      super(desc);
+      this.precision = precision;
+      this.scale = scale;
+    }
+
+    @Override
+    public DecimalData read(DecimalData ignored) {
+      return DecimalData.fromUnscaledLong(column.nextInteger(), precision, scale);
+    }
+  }
+
+  private static class LongDecimalReader extends ParquetValueReaders.PrimitiveReader<DecimalData> {
+    private final int precision;
+    private final int scale;
+
+    LongDecimalReader(ColumnDescriptor desc, int precision, int scale) {
+      super(desc);
+      this.precision = precision;
+      this.scale = scale;
+    }
+
+    @Override
+    public DecimalData read(DecimalData ignored) {
+      return DecimalData.fromUnscaledLong(column.nextLong(), precision, scale);
+    }
+  }
+
+  private static class TimestampTzReader extends ParquetValueReaders.UnboxedReader<TimestampData> {
+    TimestampTzReader(ColumnDescriptor desc) {
+      super(desc);
+    }
+
+    @Override
+    public TimestampData read(TimestampData ignored) {
+      long value = readLong();
+      return TimestampData.fromLocalDateTime(Instant.ofEpochSecond(Math.floorDiv(value, 1000_000),
+          Math.floorMod(value, 1000_000) * 1000)
+          .atOffset(ZoneOffset.UTC)
+          .toLocalDateTime());
+    }
+
+    @Override
+    public long readLong() {
+      return column.nextLong();
+    }
   }
 
-  public static ParquetValueReader<Row> buildReader(Schema expectedSchema, MessageType fileSchema) {
-    return INSTANCE.createReader(expectedSchema, fileSchema);
+  private static class TimestampReader extends ParquetValueReaders.UnboxedReader<TimestampData> {
+    TimestampReader(ColumnDescriptor desc) {
+      super(desc);
+    }
+
+    @Override
+    public TimestampData read(TimestampData ignored) {
+      long value = readLong();
+      return TimestampData.fromInstant(Instant.ofEpochSecond(Math.floorDiv(value, 1000_000),
+          Math.floorMod(value, 1000_000) * 1000));
+    }
+
+    @Override
+    public long readLong() {
+      return column.nextLong();
+    }
   }
 
-  @Override
-  protected ParquetValueReader<Row> createStructReader(List<Type> types,
-                                                       List<ParquetValueReader<?>> fieldReaders,
-                                                       Types.StructType structType) {
-    return new RowReader(types, fieldReaders, structType);
+  private static class StringReader extends ParquetValueReaders.PrimitiveReader<StringData> {
+    StringReader(ColumnDescriptor desc) {
+      super(desc);
+    }
+
+    @Override
+    public StringData read(StringData ignored) {
+      Binary binary = column.nextBinary();
+      ByteBuffer buffer = binary.toByteBuffer();
+      if (buffer.hasArray()) {
+        return StringData.fromBytes(
+            buffer.array(), buffer.arrayOffset() + buffer.position(), buffer.remaining());
+      } else {
+        return StringData.fromBytes(binary.getBytes());
+      }
+    }
+  }
+
+  private static class LossyMicrosToMillisTimeReader extends ParquetValueReaders.PrimitiveReader<Integer> {
+    LossyMicrosToMillisTimeReader(ColumnDescriptor desc) {
+      super(desc);
+    }
+
+    @Override
+    public Integer read(Integer reuse) {
+      // Discard microseconds since Flink uses millisecond unit for TIME type.
+      return (int) Math.floorDiv(column.nextLong(), 1000);
+    }
+  }
+
+  private static class ArrayReader<E> extends ParquetValueReaders.RepeatedReader<ArrayData, ReusableArrayData, E> {
+    private int readPos = 0;
+    private int writePos = 0;
+
+    ArrayReader(int definitionLevel, int repetitionLevel, ParquetValueReader<E> reader) {
+      super(definitionLevel, repetitionLevel, reader);
+    }
+
+    @Override
+    protected ReusableArrayData newListData(ArrayData reuse) {
+      this.readPos = 0;
+      this.writePos = 0;
+
+      if (reuse instanceof ReusableArrayData) {
+        return (ReusableArrayData) reuse;
+      } else {
+        return new ReusableArrayData();
+      }
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    protected E getElement(ReusableArrayData list) {
+      E value = null;
+      if (readPos < list.capacity()) {
+        value = (E) list.values[readPos];
+      }
+
+      readPos += 1;

Review comment:
       The `addElement` will put new read element to the grown buffer, and that should not be reused.  for example:
   
   when `capacity = 8`, `readPos = 8`,  list will grow the capacity to 16, and `addElement` will fill the `list[8]` with new read obejct, which should not be overwritten.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1266: Flink: update parquet reader with schema visitor

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1266:
URL: https://github.com/apache/iceberg/pull/1266#discussion_r469420690



##########
File path: flink/src/test/java/org/apache/iceberg/flink/data/TestHelpers.java
##########
@@ -0,0 +1,198 @@
+/*
+ * 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.iceberg.flink.data;
+
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.OffsetDateTime;
+import java.time.ZoneOffset;
+import java.time.temporal.ChronoUnit;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import java.util.function.Supplier;
+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.MapType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+
+public class TestHelpers {
+  private TestHelpers() {}
+
+  private static final OffsetDateTime EPOCH = Instant.ofEpochMilli(0L).atOffset(ZoneOffset.UTC);
+  private static final LocalDate EPOCH_DAY = EPOCH.toLocalDate();
+
+  public static void assertRowData(Type type, LogicalType rowType, Record expectedRecord, RowData actualRowData) {
+    if (expectedRecord == null && actualRowData == null) {
+      return;
+    }
+
+    Assert.assertTrue("expected Record and actual RowData should be both null or not null",
+        expectedRecord != null && actualRowData != null);
+
+    List<Type> types = Lists.newArrayList();
+    for (Types.NestedField field : type.asStructType().fields()) {
+      types.add(field.type());
+    }
+
+    for (int i = 0; i < types.size(); i += 1) {
+      if (expectedRecord.get(i) == null) {
+        Assert.assertTrue(actualRowData.isNullAt(i));
+        continue;
+      }
+
+      Object expected = expectedRecord.get(i);
+      LogicalType logicalType = ((RowType) rowType).getTypeAt(i);
+
+      final int fieldPos = i;
+      assertEquals(types.get(i), logicalType, expected,
+          () -> RowData.createFieldGetter(logicalType, fieldPos).getFieldOrNull(actualRowData));
+    }
+  }
+
+  private static void assertEquals(Type type, LogicalType logicalType, Object expected, Supplier<Object> supplier) {
+    switch (type.typeId()) {
+      case BOOLEAN:
+        Assert.assertEquals("boolean value should be equal", expected, supplier.get());
+        break;
+      case INTEGER:
+        Assert.assertEquals("int value should be equal", expected, supplier.get());
+        break;
+      case LONG:
+        Assert.assertEquals("long value should be equal", expected, supplier.get());
+        break;
+      case FLOAT:
+        Assert.assertEquals("float value should be equal", expected, supplier.get());
+        break;
+      case DOUBLE:
+        Assert.assertEquals("double value should be equal", expected, supplier.get());
+        break;
+      case STRING:
+        Assert.assertTrue("Should expect a CharSequence", expected instanceof CharSequence);
+        Assert.assertEquals("string should be equal",
+            String.valueOf(expected), supplier.get().toString());
+        break;
+      case DATE:
+        Assert.assertTrue("Should expect a Date", expected instanceof LocalDate);
+        LocalDate date = ChronoUnit.DAYS.addTo(EPOCH_DAY, (int) supplier.get());
+        Assert.assertEquals("date should be equal", expected, date);
+        break;
+      case TIME:
+        Assert.assertTrue("Should expect a LocalTime", expected instanceof LocalTime);
+        int milliseconds = (int) (((LocalTime) expected).toNanoOfDay() / 1000_000);
+        Assert.assertEquals("time millis should be equal", milliseconds, supplier.get());
+        break;
+      case TIMESTAMP:
+        if (((Types.TimestampType) type).shouldAdjustToUTC()) {
+          Assert.assertTrue("Should expect a OffsetDataTime", expected instanceof OffsetDateTime);
+          OffsetDateTime ts = (OffsetDateTime) expected;
+          Assert.assertEquals("OffsetDataTime should be equal", ts.toLocalDateTime(),
+              ((TimestampData) supplier.get()).toLocalDateTime());
+        } else {
+          Assert.assertTrue("Should expect a LocalDataTime", expected instanceof LocalDateTime);
+          LocalDateTime ts = (LocalDateTime) expected;
+          Assert.assertEquals("LocalDataTime should be equal", ts,
+              ((TimestampData) supplier.get()).toLocalDateTime());
+        }
+        break;
+      case BINARY:
+        Assert.assertTrue("Should expect a ByteBuffer", expected instanceof ByteBuffer);
+        Assert.assertArrayEquals("binary should be equal", ((ByteBuffer) expected).array(), (byte[]) supplier.get());

Review comment:
       This test is brittle because it assumes:
   1. ByteBuffer is a HeapByteBuffer
   2. The position and limit of the ByteBuffer are `0` and `buffer.array().length`
   3. The buffer's `arrayOffset` is `0`
   
   It would be better to convert the byte array to a `ByteBuffer` and validate equality with buffers.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #1266: Flink: update parquet reader with schema visitor

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1266:
URL: https://github.com/apache/iceberg/pull/1266#discussion_r466948097



##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java
##########
@@ -19,64 +19,703 @@
 
 package org.apache.iceberg.flink.data;
 
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.time.Instant;
+import java.time.ZoneOffset;
 import java.util.List;
-import org.apache.flink.types.Row;
+import java.util.Map;
+import org.apache.commons.lang3.ArrayUtils;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.MapData;
+import org.apache.flink.table.data.RawValueData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
 import org.apache.iceberg.Schema;
-import org.apache.iceberg.data.parquet.BaseParquetReaders;
 import org.apache.iceberg.parquet.ParquetValueReader;
 import org.apache.iceberg.parquet.ParquetValueReaders;
+import org.apache.iceberg.parquet.TypeWithSchemaVisitor;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
 import org.apache.iceberg.types.Types;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.io.api.Binary;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.LogicalTypeAnnotation.DecimalLogicalTypeAnnotation;
 import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
 import org.apache.parquet.schema.Type;
 
-public class FlinkParquetReaders extends BaseParquetReaders<Row> {
+class FlinkParquetReaders {
+  private FlinkParquetReaders() {
+  }
 
-  private static final FlinkParquetReaders INSTANCE = new FlinkParquetReaders();
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema, MessageType fileSchema) {
+    return buildReader(expectedSchema, fileSchema, ImmutableMap.of());
+  }
 
-  private FlinkParquetReaders() {
+  @SuppressWarnings("unchecked")
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema,
+                                                        MessageType fileSchema,
+                                                        Map<Integer, ?> idToConstant) {
+    return (ParquetValueReader<RowData>) TypeWithSchemaVisitor.visit(expectedSchema.asStruct(), fileSchema,
+        new ReadBuilder(fileSchema, idToConstant)
+    );
+  }
+
+  private static class ReadBuilder extends TypeWithSchemaVisitor<ParquetValueReader<?>> {
+    private final MessageType type;
+    private final Map<Integer, ?> idToConstant;
+
+    ReadBuilder(MessageType type, Map<Integer, ?> idToConstant) {
+      this.type = type;
+      this.idToConstant = idToConstant;
+    }
+
+    @Override
+    public ParquetValueReader<RowData> message(Types.StructType expected, MessageType message,
+                                               List<ParquetValueReader<?>> fieldReaders) {
+      return struct(expected, message.asGroupType(), fieldReaders);
+    }
+
+    @Override
+    public ParquetValueReader<RowData> struct(Types.StructType expected, GroupType struct,
+                                              List<ParquetValueReader<?>> fieldReaders) {
+      // match the expected struct's order
+      Map<Integer, ParquetValueReader<?>> readersById = Maps.newHashMap();
+      Map<Integer, Type> typesById = Maps.newHashMap();
+      List<Type> fields = struct.getFields();
+      for (int i = 0; i < fields.size(); i += 1) {
+        Type fieldType = fields.get(i);
+        int fieldD = type.getMaxDefinitionLevel(path(fieldType.getName())) - 1;
+        if (fieldType.getId() != null) {
+          int id = fieldType.getId().intValue();
+          readersById.put(id, ParquetValueReaders.option(fieldType, fieldD, fieldReaders.get(i)));
+          typesById.put(id, fieldType);
+        }
+      }
+
+      List<Types.NestedField> expectedFields = expected != null ?
+          expected.fields() : ImmutableList.of();
+      List<ParquetValueReader<?>> reorderedFields = Lists.newArrayListWithExpectedSize(
+          expectedFields.size());
+      List<Type> types = Lists.newArrayListWithExpectedSize(expectedFields.size());
+      for (Types.NestedField field : expectedFields) {
+        int id = field.fieldId();
+        if (idToConstant.containsKey(id)) {
+          // containsKey is used because the constant may be null
+          reorderedFields.add(ParquetValueReaders.constant(idToConstant.get(id)));
+          types.add(null);
+        } else {
+          ParquetValueReader<?> reader = readersById.get(id);
+          if (reader != null) {
+            reorderedFields.add(reader);
+            types.add(typesById.get(id));
+          } else {
+            reorderedFields.add(ParquetValueReaders.nulls());
+            types.add(null);
+          }
+        }
+      }
+
+      return new RowDataReader(types, reorderedFields);
+    }
+
+    @Override
+    public ParquetValueReader<?> list(Types.ListType expectedList, GroupType array,
+                                      ParquetValueReader<?> elementReader) {
+      GroupType repeated = array.getFields().get(0).asGroupType();
+      String[] repeatedPath = currentPath();
+
+      int repeatedD = type.getMaxDefinitionLevel(repeatedPath) - 1;
+      int repeatedR = type.getMaxRepetitionLevel(repeatedPath) - 1;
+
+      Type elementType = repeated.getType(0);
+      int elementD = type.getMaxDefinitionLevel(path(elementType.getName())) - 1;
+
+      return new ArrayReader<>(repeatedD, repeatedR, ParquetValueReaders.option(elementType, elementD, elementReader));
+    }
+
+    @Override
+    public ParquetValueReader<?> map(Types.MapType expectedMap, GroupType map,
+                                     ParquetValueReader<?> keyReader,
+                                     ParquetValueReader<?> valueReader) {
+      GroupType repeatedKeyValue = map.getFields().get(0).asGroupType();
+      String[] repeatedPath = currentPath();
+
+      int repeatedD = type.getMaxDefinitionLevel(repeatedPath) - 1;
+      int repeatedR = type.getMaxRepetitionLevel(repeatedPath) - 1;
+
+      Type keyType = repeatedKeyValue.getType(0);
+      int keyD = type.getMaxDefinitionLevel(path(keyType.getName())) - 1;
+      Type valueType = repeatedKeyValue.getType(1);
+      int valueD = type.getMaxDefinitionLevel(path(valueType.getName())) - 1;
+
+      return new MapReader<>(repeatedD, repeatedR,
+          ParquetValueReaders.option(keyType, keyD, keyReader),
+          ParquetValueReaders.option(valueType, valueD, valueReader));
+    }
+
+    @Override
+    public ParquetValueReader<?> primitive(org.apache.iceberg.types.Type.PrimitiveType expected,
+                                           PrimitiveType primitive) {
+      ColumnDescriptor desc = type.getColumnDescription(currentPath());
+
+      if (primitive.getOriginalType() != null) {
+        switch (primitive.getOriginalType()) {
+          case ENUM:
+          case JSON:
+          case UTF8:
+            return new StringReader(desc);
+          case INT_8:
+          case INT_16:
+          case INT_32:
+            if (expected != null && expected.typeId() == Types.LongType.get().typeId()) {
+              return new ParquetValueReaders.IntAsLongReader(desc);
+            } else {
+              return new ParquetValueReaders.UnboxedReader<>(desc);
+            }
+          case TIME_MICROS:
+            return new LossyMicrosToMillisTimeReader(desc);
+          case DATE:
+          case INT_64:
+            return new ParquetValueReaders.UnboxedReader<>(desc);
+          case TIMESTAMP_MICROS:
+            if (((Types.TimestampType) expected).shouldAdjustToUTC()) {
+              return new TimestampTzReader(desc);
+            } else {
+              return new TimestampReader(desc);
+            }
+          case DECIMAL:
+            DecimalLogicalTypeAnnotation decimal = (DecimalLogicalTypeAnnotation) primitive.getLogicalTypeAnnotation();
+            switch (primitive.getPrimitiveTypeName()) {
+              case BINARY:
+              case FIXED_LEN_BYTE_ARRAY:
+                return new BinaryDecimalReader(desc, decimal.getPrecision(), decimal.getScale());
+              case INT64:
+                return new LongDecimalReader(desc, decimal.getPrecision(), decimal.getScale());
+              case INT32:
+                return new IntegerDecimalReader(desc, decimal.getPrecision(), decimal.getScale());
+              default:
+                throw new UnsupportedOperationException(
+                    "Unsupported base type for decimal: " + primitive.getPrimitiveTypeName());
+            }
+          case BSON:
+            return new ParquetValueReaders.ByteArrayReader(desc);
+          default:
+            throw new UnsupportedOperationException(
+                "Unsupported logical type: " + primitive.getOriginalType());
+        }
+      }
+
+      switch (primitive.getPrimitiveTypeName()) {
+        case FIXED_LEN_BYTE_ARRAY:
+        case BINARY:
+          return new ParquetValueReaders.ByteArrayReader(desc);
+        case INT32:
+          if (expected != null && expected.typeId() == org.apache.iceberg.types.Type.TypeID.LONG) {
+            return new ParquetValueReaders.IntAsLongReader(desc);
+          } else {
+            return new ParquetValueReaders.UnboxedReader<>(desc);
+          }
+        case FLOAT:
+          if (expected != null && expected.typeId() == org.apache.iceberg.types.Type.TypeID.DOUBLE) {
+            return new ParquetValueReaders.FloatAsDoubleReader(desc);
+          } else {
+            return new ParquetValueReaders.UnboxedReader<>(desc);
+          }
+        case BOOLEAN:
+        case INT64:
+        case DOUBLE:
+          return new ParquetValueReaders.UnboxedReader<>(desc);
+        default:
+          throw new UnsupportedOperationException("Unsupported type: " + primitive);
+      }
+    }
+  }
+
+  private static class BinaryDecimalReader extends ParquetValueReaders.PrimitiveReader<DecimalData> {
+    private final int precision;
+    private final int scale;
+
+    BinaryDecimalReader(ColumnDescriptor desc, int precision, int scale) {
+      super(desc);
+      this.precision = precision;
+      this.scale = scale;
+    }
+
+    @Override
+    public DecimalData read(DecimalData ignored) {
+      Binary binary = column.nextBinary();
+      BigDecimal bigDecimal = new BigDecimal(new BigInteger(binary.getBytes()), scale);
+      return DecimalData.fromBigDecimal(bigDecimal, precision, scale);
+    }
+  }
+
+  private static class IntegerDecimalReader extends ParquetValueReaders.PrimitiveReader<DecimalData> {
+    private final int precision;
+    private final int scale;
+
+    IntegerDecimalReader(ColumnDescriptor desc, int precision, int scale) {
+      super(desc);
+      this.precision = precision;
+      this.scale = scale;
+    }
+
+    @Override
+    public DecimalData read(DecimalData ignored) {
+      return DecimalData.fromUnscaledLong(column.nextInteger(), precision, scale);
+    }
+  }
+
+  private static class LongDecimalReader extends ParquetValueReaders.PrimitiveReader<DecimalData> {
+    private final int precision;
+    private final int scale;
+
+    LongDecimalReader(ColumnDescriptor desc, int precision, int scale) {
+      super(desc);
+      this.precision = precision;
+      this.scale = scale;
+    }
+
+    @Override
+    public DecimalData read(DecimalData ignored) {
+      return DecimalData.fromUnscaledLong(column.nextLong(), precision, scale);
+    }
+  }
+
+  private static class TimestampTzReader extends ParquetValueReaders.UnboxedReader<TimestampData> {
+    TimestampTzReader(ColumnDescriptor desc) {
+      super(desc);
+    }
+
+    @Override
+    public TimestampData read(TimestampData ignored) {
+      long value = readLong();
+      return TimestampData.fromLocalDateTime(Instant.ofEpochSecond(Math.floorDiv(value, 1000_000),
+          Math.floorMod(value, 1000_000) * 1000)
+          .atOffset(ZoneOffset.UTC)
+          .toLocalDateTime());
+    }
+
+    @Override
+    public long readLong() {
+      return column.nextLong();
+    }
   }
 
-  public static ParquetValueReader<Row> buildReader(Schema expectedSchema, MessageType fileSchema) {
-    return INSTANCE.createReader(expectedSchema, fileSchema);
+  private static class TimestampReader extends ParquetValueReaders.UnboxedReader<TimestampData> {
+    TimestampReader(ColumnDescriptor desc) {
+      super(desc);
+    }
+
+    @Override
+    public TimestampData read(TimestampData ignored) {
+      long value = readLong();
+      return TimestampData.fromInstant(Instant.ofEpochSecond(Math.floorDiv(value, 1000_000),
+          Math.floorMod(value, 1000_000) * 1000));
+    }
+
+    @Override
+    public long readLong() {
+      return column.nextLong();
+    }
   }
 
-  @Override
-  protected ParquetValueReader<Row> createStructReader(List<Type> types,
-                                                       List<ParquetValueReader<?>> fieldReaders,
-                                                       Types.StructType structType) {
-    return new RowReader(types, fieldReaders, structType);
+  private static class StringReader extends ParquetValueReaders.PrimitiveReader<StringData> {
+    StringReader(ColumnDescriptor desc) {
+      super(desc);
+    }
+
+    @Override
+    public StringData read(StringData ignored) {
+      Binary binary = column.nextBinary();
+      ByteBuffer buffer = binary.toByteBuffer();
+      if (buffer.hasArray()) {
+        return StringData.fromBytes(
+            buffer.array(), buffer.arrayOffset() + buffer.position(), buffer.remaining());
+      } else {
+        return StringData.fromBytes(binary.getBytes());
+      }
+    }
+  }
+
+  private static class LossyMicrosToMillisTimeReader extends ParquetValueReaders.PrimitiveReader<Integer> {
+    LossyMicrosToMillisTimeReader(ColumnDescriptor desc) {
+      super(desc);
+    }
+
+    @Override
+    public Integer read(Integer reuse) {
+      // Discard microseconds since Flink uses millisecond unit for TIME type.
+      return (int) Math.floorDiv(column.nextLong(), 1000);
+    }
+  }
+
+  private static class ArrayReader<E> extends ParquetValueReaders.RepeatedReader<ArrayData, ReusableArrayData, E> {
+    private int readPos = 0;
+    private int writePos = 0;
+
+    ArrayReader(int definitionLevel, int repetitionLevel, ParquetValueReader<E> reader) {
+      super(definitionLevel, repetitionLevel, reader);
+    }
+
+    @Override
+    protected ReusableArrayData newListData(ArrayData reuse) {
+      this.readPos = 0;
+      this.writePos = 0;
+
+      if (reuse instanceof ReusableArrayData) {
+        return (ReusableArrayData) reuse;
+      } else {
+        return new ReusableArrayData();
+      }
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    protected E getElement(ReusableArrayData list) {
+      E value = null;
+      if (readPos < list.capacity()) {
+        value = (E) list.values[readPos];
+      }
+
+      readPos += 1;
+
+      return value;
+    }
+
+    @Override
+    protected void addElement(ReusableArrayData reused, E element) {
+      if (writePos >= reused.capacity()) {
+        reused.grow();
+      }
+
+      reused.values[writePos] = element;
+
+      writePos += 1;
+    }
+
+    @Override
+    protected ArrayData buildList(ReusableArrayData list) {
+      list.setNumElements(writePos);
+      return list;
+    }
   }
 
-  private static class RowReader extends ParquetValueReaders.StructReader<Row, Row> {
-    private final Types.StructType structType;
+  private static class MapReader<K, V> extends
+      ParquetValueReaders.RepeatedKeyValueReader<MapData, ReusableMapData, K, V> {
+    private int readPos = 0;
+    private int writePos = 0;
+
+    private final ParquetValueReaders.ReusableEntry<K, V> entry = new ParquetValueReaders.ReusableEntry<>();
+    private final ParquetValueReaders.ReusableEntry<K, V> nullEntry = new ParquetValueReaders.ReusableEntry<>();
+
+    MapReader(int definitionLevel, int repetitionLevel,
+              ParquetValueReader<K> keyReader, ParquetValueReader<V> valueReader) {
+      super(definitionLevel, repetitionLevel, keyReader, valueReader);
+    }
+
+    @Override
+    protected ReusableMapData newMapData(MapData reuse) {
+      this.readPos = 0;
+      this.writePos = 0;
+
+      if (reuse instanceof ReusableMapData) {
+        return (ReusableMapData) reuse;
+      } else {
+        return new ReusableMapData();
+      }
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    protected Map.Entry<K, V> getPair(ReusableMapData map) {
+      Map.Entry<K, V> kv = nullEntry;
+      if (readPos < map.capacity()) {
+        entry.set((K) map.keys.values[readPos], (V) map.values.values[readPos]);
+        kv = entry;
+      }
+
+      readPos += 1;
+
+      return kv;
+    }
+
+    @Override
+    protected void addPair(ReusableMapData map, K key, V value) {
+      if (writePos >= map.capacity()) {
+        map.grow();
+      }
+
+      map.keys.values[writePos] = key;
+      map.values.values[writePos] = value;
+
+      writePos += 1;
+    }
 
-    RowReader(List<Type> types, List<ParquetValueReader<?>> readers, Types.StructType struct) {
+    @Override
+    protected MapData buildMap(ReusableMapData map) {
+      map.setNumElements(writePos);
+      return map;
+    }
+  }
+
+  private static class RowDataReader extends ParquetValueReaders.StructReader<RowData, GenericRowData> {
+    private final int numFields;
+
+    RowDataReader(List<Type> types, List<ParquetValueReader<?>> readers) {
       super(types, readers);
-      this.structType = struct;
+      this.numFields = readers.size();
     }
 
     @Override
-    protected Row newStructData(Row reuse) {
-      if (reuse != null) {
-        return reuse;
+    protected GenericRowData newStructData(RowData reuse) {
+      if (reuse instanceof GenericRowData) {
+        return (GenericRowData) reuse;
       } else {
-        return new Row(structType.fields().size());
+        return new GenericRowData(numFields);
       }
     }
 
     @Override
-    protected Object getField(Row row, int pos) {
-      return row.getField(pos);
+    protected Object getField(GenericRowData intermediate, int pos) {
+      return intermediate.getField(pos);
     }
 
     @Override
-    protected Row buildStruct(Row row) {
-      return row;
+    protected RowData buildStruct(GenericRowData struct) {
+      return struct;
     }
 
     @Override
-    protected void set(Row row, int pos, Object value) {
+    protected void set(GenericRowData row, int pos, Object value) {
       row.setField(pos, value);
     }
+
+    @Override
+    protected void setNull(GenericRowData row, int pos) {
+      row.setField(pos, null);
+    }
+
+    @Override
+    protected void setBoolean(GenericRowData row, int pos, boolean value) {
+      row.setField(pos, value);
+    }
+
+    @Override
+    protected void setInteger(GenericRowData row, int pos, int value) {
+      row.setField(pos, value);
+    }
+
+    @Override
+    protected void setLong(GenericRowData row, int pos, long value) {
+      row.setField(pos, value);
+    }
+
+    @Override
+    protected void setFloat(GenericRowData row, int pos, float value) {
+      row.setField(pos, value);
+    }
+
+    @Override
+    protected void setDouble(GenericRowData row, int pos, double value) {
+      row.setField(pos, value);
+    }

Review comment:
       For flink `GenericRowData`,  we don't provide setInteger/setLong/setFloat interfaces etc, so seems we don't have to override those methods from `ParquetValueReaders.StructReader`, because all of them have the default implementation, for example: 
   
   ```java
       protected void setBoolean(I struct, int pos, boolean value) {
         set(struct, pos, value);
       }
   ```
   
   I mean override the `set(I struct, int pos, Object value)` is enough.
   
   BTW, I saw Setter interface in `ParquetValueReaders.StructReader` don't have meaningful usage. Is it necessary to create a separate pull request to remove all methods related to `Setter`  ?  cc @rdblue .  




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] chenjunjiedada commented on a change in pull request #1266: Flink: update parquet reader with schema visitor

Posted by GitBox <gi...@apache.org>.
chenjunjiedada commented on a change in pull request #1266:
URL: https://github.com/apache/iceberg/pull/1266#discussion_r466824006



##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java
##########
@@ -19,64 +19,701 @@
 
 package org.apache.iceberg.flink.data;
 
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.time.Instant;
+import java.time.ZoneOffset;
 import java.util.List;
-import org.apache.flink.types.Row;
+import java.util.Map;
+import org.apache.commons.lang3.ArrayUtils;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.MapData;
+import org.apache.flink.table.data.RawValueData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
 import org.apache.iceberg.Schema;
-import org.apache.iceberg.data.parquet.BaseParquetReaders;
 import org.apache.iceberg.parquet.ParquetValueReader;
 import org.apache.iceberg.parquet.ParquetValueReaders;
+import org.apache.iceberg.parquet.TypeWithSchemaVisitor;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
 import org.apache.iceberg.types.Types;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.io.api.Binary;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.LogicalTypeAnnotation.DecimalLogicalTypeAnnotation;
 import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
 import org.apache.parquet.schema.Type;
 
-public class FlinkParquetReaders extends BaseParquetReaders<Row> {
+class FlinkParquetReaders {
+  private FlinkParquetReaders() {
+  }
 
-  private static final FlinkParquetReaders INSTANCE = new FlinkParquetReaders();
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema, MessageType fileSchema) {
+    return buildReader(expectedSchema, fileSchema, ImmutableMap.of());
+  }
 
-  private FlinkParquetReaders() {
+  @SuppressWarnings("unchecked")
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema,
+                                                        MessageType fileSchema,
+                                                        Map<Integer, ?> idToConstant) {
+    return (ParquetValueReader<RowData>) TypeWithSchemaVisitor.visit(expectedSchema.asStruct(), fileSchema,
+        new ReadBuilder(fileSchema, idToConstant)
+    );
+  }
+
+  private static class ReadBuilder extends TypeWithSchemaVisitor<ParquetValueReader<?>> {
+    private final MessageType type;
+    private final Map<Integer, ?> idToConstant;
+
+    ReadBuilder(MessageType type, Map<Integer, ?> idToConstant) {
+      this.type = type;
+      this.idToConstant = idToConstant;
+    }
+
+    @Override
+    public ParquetValueReader<RowData> message(Types.StructType expected, MessageType message,
+                                               List<ParquetValueReader<?>> fieldReaders) {
+      return struct(expected, message.asGroupType(), fieldReaders);
+    }
+
+    @Override
+    public ParquetValueReader<RowData> struct(Types.StructType expected, GroupType struct,
+                                              List<ParquetValueReader<?>> fieldReaders) {
+      // match the expected struct's order
+      Map<Integer, ParquetValueReader<?>> readersById = Maps.newHashMap();
+      Map<Integer, Type> typesById = Maps.newHashMap();
+      List<Type> fields = struct.getFields();
+      for (int i = 0; i < fields.size(); i += 1) {
+        Type fieldType = fields.get(i);
+        int fieldD = type.getMaxDefinitionLevel(path(fieldType.getName())) - 1;
+        if (fieldType.getId() != null) {
+          int id = fieldType.getId().intValue();
+          readersById.put(id, ParquetValueReaders.option(fieldType, fieldD, fieldReaders.get(i)));
+          typesById.put(id, fieldType);
+        }
+      }
+
+      List<Types.NestedField> expectedFields = expected != null ?
+          expected.fields() : ImmutableList.of();
+      List<ParquetValueReader<?>> reorderedFields = Lists.newArrayListWithExpectedSize(
+          expectedFields.size());
+      List<Type> types = Lists.newArrayListWithExpectedSize(expectedFields.size());
+      for (Types.NestedField field : expectedFields) {
+        int id = field.fieldId();
+        if (idToConstant.containsKey(id)) {
+          // containsKey is used because the constant may be null
+          reorderedFields.add(ParquetValueReaders.constant(idToConstant.get(id)));
+          types.add(null);
+        } else {
+          ParquetValueReader<?> reader = readersById.get(id);
+          if (reader != null) {
+            reorderedFields.add(reader);
+            types.add(typesById.get(id));
+          } else {
+            reorderedFields.add(ParquetValueReaders.nulls());
+            types.add(null);
+          }
+        }
+      }
+
+      return new RowDataReader(types, reorderedFields);
+    }
+
+    @Override
+    public ParquetValueReader<?> list(Types.ListType expectedList, GroupType array,
+                                      ParquetValueReader<?> elementReader) {
+      GroupType repeated = array.getFields().get(0).asGroupType();
+      String[] repeatedPath = currentPath();
+
+      int repeatedD = type.getMaxDefinitionLevel(repeatedPath) - 1;
+      int repeatedR = type.getMaxRepetitionLevel(repeatedPath) - 1;
+
+      Type elementType = repeated.getType(0);
+      int elementD = type.getMaxDefinitionLevel(path(elementType.getName())) - 1;
+
+      return new ArrayReader<>(repeatedD, repeatedR, ParquetValueReaders.option(elementType, elementD, elementReader));
+    }
+
+    @Override
+    public ParquetValueReader<?> map(Types.MapType expectedMap, GroupType map,
+                                     ParquetValueReader<?> keyReader,
+                                     ParquetValueReader<?> valueReader) {
+      GroupType repeatedKeyValue = map.getFields().get(0).asGroupType();
+      String[] repeatedPath = currentPath();
+
+      int repeatedD = type.getMaxDefinitionLevel(repeatedPath) - 1;
+      int repeatedR = type.getMaxRepetitionLevel(repeatedPath) - 1;
+
+      Type keyType = repeatedKeyValue.getType(0);
+      int keyD = type.getMaxDefinitionLevel(path(keyType.getName())) - 1;
+      Type valueType = repeatedKeyValue.getType(1);
+      int valueD = type.getMaxDefinitionLevel(path(valueType.getName())) - 1;
+
+      return new MapReader<>(repeatedD, repeatedR,
+          ParquetValueReaders.option(keyType, keyD, keyReader),
+          ParquetValueReaders.option(valueType, valueD, valueReader));
+    }
+
+    @Override
+    public ParquetValueReader<?> primitive(org.apache.iceberg.types.Type.PrimitiveType expected,
+                                           PrimitiveType primitive) {
+      ColumnDescriptor desc = type.getColumnDescription(currentPath());
+
+      if (primitive.getOriginalType() != null) {
+        switch (primitive.getOriginalType()) {
+          case ENUM:
+          case JSON:
+          case UTF8:
+            return new StringReader(desc);
+          case INT_8:
+          case INT_16:
+          case INT_32:
+          case DATE:
+            if (expected != null && expected.typeId() == Types.LongType.get().typeId()) {
+              return new ParquetValueReaders.IntAsLongReader(desc);
+            } else {
+              return new ParquetValueReaders.UnboxedReader<>(desc);
+            }
+          case TIME_MICROS:
+            return new LossyMicrosToMillisTimeReader(desc);
+          case INT_64:
+            return new ParquetValueReaders.UnboxedReader<>(desc);
+          case TIMESTAMP_MICROS:
+            if (((Types.TimestampType) expected).shouldAdjustToUTC()) {
+              return new TimestampTzReader(desc);
+            } else {
+              return new TimestampReader(desc);
+            }
+          case DECIMAL:
+            DecimalLogicalTypeAnnotation decimal = (DecimalLogicalTypeAnnotation) primitive.getLogicalTypeAnnotation();
+            switch (primitive.getPrimitiveTypeName()) {
+              case BINARY:
+              case FIXED_LEN_BYTE_ARRAY:
+                return new BinaryDecimalReader(desc, decimal.getScale());
+              case INT64:
+                return new LongDecimalReader(desc, decimal.getPrecision(), decimal.getScale());
+              case INT32:
+                return new IntegerDecimalReader(desc, decimal.getPrecision(), decimal.getScale());
+              default:
+                throw new UnsupportedOperationException(
+                    "Unsupported base type for decimal: " + primitive.getPrimitiveTypeName());
+            }
+          case BSON:
+            return new ParquetValueReaders.ByteArrayReader(desc);
+          default:
+            throw new UnsupportedOperationException(
+                "Unsupported logical type: " + primitive.getOriginalType());
+        }
+      }
+
+      switch (primitive.getPrimitiveTypeName()) {
+        case FIXED_LEN_BYTE_ARRAY:
+        case BINARY:
+          return new ParquetValueReaders.ByteArrayReader(desc);
+        case INT32:
+          if (expected != null && expected.typeId() == org.apache.iceberg.types.Type.TypeID.LONG) {
+            return new ParquetValueReaders.IntAsLongReader(desc);
+          } else {
+            return new ParquetValueReaders.UnboxedReader<>(desc);
+          }
+        case FLOAT:
+          if (expected != null && expected.typeId() == org.apache.iceberg.types.Type.TypeID.DOUBLE) {
+            return new ParquetValueReaders.FloatAsDoubleReader(desc);
+          } else {
+            return new ParquetValueReaders.UnboxedReader<>(desc);
+          }
+        case BOOLEAN:
+        case INT64:
+        case DOUBLE:
+          return new ParquetValueReaders.UnboxedReader<>(desc);
+        default:
+          throw new UnsupportedOperationException("Unsupported type: " + primitive);
+      }
+    }
+  }
+
+  private static class BinaryDecimalReader extends ParquetValueReaders.PrimitiveReader<DecimalData> {
+    private final int scale;
+
+    BinaryDecimalReader(ColumnDescriptor desc, int scale) {
+      super(desc);
+      this.scale = scale;
+    }
+
+    @Override
+    public DecimalData read(DecimalData ignored) {
+      Binary binary = column.nextBinary();
+      BigDecimal bigDecimal = new BigDecimal(new BigInteger(binary.getBytes()), scale);
+      return DecimalData.fromBigDecimal(bigDecimal, bigDecimal.precision(), scale);
+    }
+  }
+
+  private static class IntegerDecimalReader extends ParquetValueReaders.PrimitiveReader<DecimalData> {
+    private final int precision;
+    private final int scale;
+
+    IntegerDecimalReader(ColumnDescriptor desc, int precision, int scale) {
+      super(desc);
+      this.precision = precision;
+      this.scale = scale;
+    }
+
+    @Override
+    public DecimalData read(DecimalData ignored) {
+      return DecimalData.fromUnscaledLong(column.nextInteger(), precision, scale);
+    }
+  }
+
+  private static class LongDecimalReader extends ParquetValueReaders.PrimitiveReader<DecimalData> {
+    private final int precision;
+    private final int scale;
+
+    LongDecimalReader(ColumnDescriptor desc, int precision, int scale) {
+      super(desc);
+      this.precision = precision;
+      this.scale = scale;
+    }
+
+    @Override
+    public DecimalData read(DecimalData ignored) {
+      return DecimalData.fromUnscaledLong(column.nextLong(), precision, scale);
+    }
+  }
+
+  private static class TimestampTzReader extends ParquetValueReaders.UnboxedReader<TimestampData> {
+    TimestampTzReader(ColumnDescriptor desc) {
+      super(desc);
+    }
+
+    @Override
+    public TimestampData read(TimestampData ignored) {
+      long value = readLong();
+      return TimestampData.fromLocalDateTime(Instant.ofEpochSecond(Math.floorDiv(value, 1000_000),
+          Math.floorMod(value, 1000_000) * 1000)
+          .atOffset(ZoneOffset.UTC)
+          .toLocalDateTime());

Review comment:
       I think here is constructing the `TimestampData` from `LocalDataTime`, but not converting to LocalDataTime. 




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi commented on a change in pull request #1266: Flink: update parquet reader with schema visitor

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on a change in pull request #1266:
URL: https://github.com/apache/iceberg/pull/1266#discussion_r465457223



##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java
##########
@@ -19,64 +19,719 @@
 
 package org.apache.iceberg.flink.data;
 
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.time.Instant;
 import java.util.List;
-import org.apache.flink.types.Row;
+import java.util.Map;
+import org.apache.commons.lang3.ArrayUtils;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.MapData;
+import org.apache.flink.table.data.RawValueData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
 import org.apache.iceberg.Schema;
-import org.apache.iceberg.data.parquet.BaseParquetReaders;
+import org.apache.iceberg.parquet.ParquetSchemaUtil;
 import org.apache.iceberg.parquet.ParquetValueReader;
 import org.apache.iceberg.parquet.ParquetValueReaders;
+import org.apache.iceberg.parquet.TypeWithSchemaVisitor;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
 import org.apache.iceberg.types.Types;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.io.api.Binary;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.LogicalTypeAnnotation.DecimalLogicalTypeAnnotation;
 import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
 import org.apache.parquet.schema.Type;
 
-public class FlinkParquetReaders extends BaseParquetReaders<Row> {
+class FlinkParquetReaders {
+  private FlinkParquetReaders() {
+  }
 
-  private static final FlinkParquetReaders INSTANCE = new FlinkParquetReaders();
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema, MessageType fileSchema) {
+    return buildReader(expectedSchema, fileSchema, ImmutableMap.of());
+  }
 
-  private FlinkParquetReaders() {
+  @SuppressWarnings("unchecked")
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema,
+                                                        MessageType fileSchema,
+                                                        Map<Integer, ?> idToConstant) {
+    ReadBuilder builder = new ReadBuilder(fileSchema, idToConstant);
+    if (ParquetSchemaUtil.hasIds(fileSchema)) {
+      return (ParquetValueReader<RowData>)
+          TypeWithSchemaVisitor.visit(expectedSchema.asStruct(), fileSchema, builder);
+    } else {
+      return (ParquetValueReader<RowData>)
+          TypeWithSchemaVisitor.visit(expectedSchema.asStruct(), fileSchema,
+              new FallbackReadBuilder(builder));
+    }
+  }
+
+  private static class FallbackReadBuilder extends TypeWithSchemaVisitor<ParquetValueReader<?>> {
+    private MessageType type;
+    private final TypeWithSchemaVisitor<ParquetValueReader<?>> builder;
+
+    FallbackReadBuilder(TypeWithSchemaVisitor<ParquetValueReader<?>> builder) {
+      this.builder = builder;
+    }
+
+    @Override
+    public ParquetValueReader<?> message(Types.StructType expected, MessageType message,
+                                         List<ParquetValueReader<?>> fieldReaders) {
+      // the top level matches by ID, but the remaining IDs are missing
+      this.type = message;
+      return builder.struct(expected, message, fieldReaders);
+    }
+
+    @Override
+    public ParquetValueReader<?> struct(Types.StructType ignored, GroupType struct,
+                                        List<ParquetValueReader<?>> fieldReaders) {
+      // the expected struct is ignored because nested fields are never found when the IDs are missing
+      List<ParquetValueReader<?>> newFields = Lists.newArrayListWithExpectedSize(
+          fieldReaders.size());
+      List<Type> types = Lists.newArrayListWithExpectedSize(fieldReaders.size());
+      List<Type> fields = struct.getFields();
+      for (int i = 0; i < fields.size(); i += 1) {
+        Type fieldType = fields.get(i);
+        int fieldD = type.getMaxDefinitionLevel(path(fieldType.getName())) - 1;
+        newFields.add(ParquetValueReaders.option(fieldType, fieldD, fieldReaders.get(i)));
+        types.add(fieldType);
+      }
+
+      return new RowDataReader(types, newFields);
+    }
+  }
+
+  private static class ReadBuilder extends TypeWithSchemaVisitor<ParquetValueReader<?>> {

Review comment:
       Hi @chenjunjiedada , I have no idea about Avro, it is simple enough, looks like most of codes are related to Flink data structures. It may not be worth refactoring.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1266: Flink: update parquet reader with schema visitor

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1266:
URL: https://github.com/apache/iceberg/pull/1266#discussion_r469419256



##########
File path: flink/src/test/java/org/apache/iceberg/flink/data/TestHelpers.java
##########
@@ -0,0 +1,198 @@
+/*
+ * 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.iceberg.flink.data;
+
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.OffsetDateTime;
+import java.time.ZoneOffset;
+import java.time.temporal.ChronoUnit;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import java.util.function.Supplier;
+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.MapType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+
+public class TestHelpers {
+  private TestHelpers() {}
+
+  private static final OffsetDateTime EPOCH = Instant.ofEpochMilli(0L).atOffset(ZoneOffset.UTC);
+  private static final LocalDate EPOCH_DAY = EPOCH.toLocalDate();
+
+  public static void assertRowData(Type type, LogicalType rowType, Record expectedRecord, RowData actualRowData) {
+    if (expectedRecord == null && actualRowData == null) {
+      return;
+    }
+
+    Assert.assertTrue("expected Record and actual RowData should be both null or not null",
+        expectedRecord != null && actualRowData != null);
+
+    List<Type> types = Lists.newArrayList();
+    for (Types.NestedField field : type.asStructType().fields()) {
+      types.add(field.type());
+    }
+
+    for (int i = 0; i < types.size(); i += 1) {
+      if (expectedRecord.get(i) == null) {
+        Assert.assertTrue(actualRowData.isNullAt(i));
+        continue;
+      }
+
+      Object expected = expectedRecord.get(i);
+      LogicalType logicalType = ((RowType) rowType).getTypeAt(i);
+
+      final int fieldPos = i;
+      assertEquals(types.get(i), logicalType, expected,
+          () -> RowData.createFieldGetter(logicalType, fieldPos).getFieldOrNull(actualRowData));
+    }
+  }
+
+  private static void assertEquals(Type type, LogicalType logicalType, Object expected, Supplier<Object> supplier) {
+    switch (type.typeId()) {
+      case BOOLEAN:
+        Assert.assertEquals("boolean value should be equal", expected, supplier.get());
+        break;
+      case INTEGER:
+        Assert.assertEquals("int value should be equal", expected, supplier.get());
+        break;
+      case LONG:
+        Assert.assertEquals("long value should be equal", expected, supplier.get());
+        break;
+      case FLOAT:
+        Assert.assertEquals("float value should be equal", expected, supplier.get());
+        break;
+      case DOUBLE:
+        Assert.assertEquals("double value should be equal", expected, supplier.get());
+        break;
+      case STRING:
+        Assert.assertTrue("Should expect a CharSequence", expected instanceof CharSequence);
+        Assert.assertEquals("string should be equal",
+            String.valueOf(expected), supplier.get().toString());
+        break;
+      case DATE:
+        Assert.assertTrue("Should expect a Date", expected instanceof LocalDate);
+        LocalDate date = ChronoUnit.DAYS.addTo(EPOCH_DAY, (int) supplier.get());
+        Assert.assertEquals("date should be equal", expected, date);
+        break;
+      case TIME:
+        Assert.assertTrue("Should expect a LocalTime", expected instanceof LocalTime);
+        int milliseconds = (int) (((LocalTime) expected).toNanoOfDay() / 1000_000);
+        Assert.assertEquals("time millis should be equal", milliseconds, supplier.get());
+        break;
+      case TIMESTAMP:
+        if (((Types.TimestampType) type).shouldAdjustToUTC()) {
+          Assert.assertTrue("Should expect a OffsetDataTime", expected instanceof OffsetDateTime);
+          OffsetDateTime ts = (OffsetDateTime) expected;
+          Assert.assertEquals("OffsetDataTime should be equal", ts.toLocalDateTime(),
+              ((TimestampData) supplier.get()).toLocalDateTime());
+        } else {
+          Assert.assertTrue("Should expect a LocalDataTime", expected instanceof LocalDateTime);
+          LocalDateTime ts = (LocalDateTime) expected;
+          Assert.assertEquals("LocalDataTime should be equal", ts,
+              ((TimestampData) supplier.get()).toLocalDateTime());

Review comment:
       It is good that this uses a different conversion to `LocalDateTime`.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] chenjunjiedada commented on a change in pull request #1266: Flink: update parquet reader with schema visitor

Posted by GitBox <gi...@apache.org>.
chenjunjiedada commented on a change in pull request #1266:
URL: https://github.com/apache/iceberg/pull/1266#discussion_r469690543



##########
File path: flink/src/test/java/org/apache/iceberg/flink/data/TestHelpers.java
##########
@@ -0,0 +1,198 @@
+/*
+ * 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.iceberg.flink.data;
+
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.OffsetDateTime;
+import java.time.ZoneOffset;
+import java.time.temporal.ChronoUnit;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import java.util.function.Supplier;
+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.MapType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+
+public class TestHelpers {
+  private TestHelpers() {}
+
+  private static final OffsetDateTime EPOCH = Instant.ofEpochMilli(0L).atOffset(ZoneOffset.UTC);
+  private static final LocalDate EPOCH_DAY = EPOCH.toLocalDate();
+
+  public static void assertRowData(Type type, LogicalType rowType, Record expectedRecord, RowData actualRowData) {
+    if (expectedRecord == null && actualRowData == null) {
+      return;
+    }
+
+    Assert.assertTrue("expected Record and actual RowData should be both null or not null",
+        expectedRecord != null && actualRowData != null);
+
+    List<Type> types = Lists.newArrayList();
+    for (Types.NestedField field : type.asStructType().fields()) {
+      types.add(field.type());
+    }
+
+    for (int i = 0; i < types.size(); i += 1) {
+      if (expectedRecord.get(i) == null) {
+        Assert.assertTrue(actualRowData.isNullAt(i));
+        continue;
+      }
+
+      Object expected = expectedRecord.get(i);
+      LogicalType logicalType = ((RowType) rowType).getTypeAt(i);
+
+      final int fieldPos = i;
+      assertEquals(types.get(i), logicalType, expected,
+          () -> RowData.createFieldGetter(logicalType, fieldPos).getFieldOrNull(actualRowData));
+    }
+  }
+
+  private static void assertEquals(Type type, LogicalType logicalType, Object expected, Supplier<Object> supplier) {
+    switch (type.typeId()) {
+      case BOOLEAN:
+        Assert.assertEquals("boolean value should be equal", expected, supplier.get());
+        break;
+      case INTEGER:
+        Assert.assertEquals("int value should be equal", expected, supplier.get());
+        break;
+      case LONG:
+        Assert.assertEquals("long value should be equal", expected, supplier.get());
+        break;
+      case FLOAT:
+        Assert.assertEquals("float value should be equal", expected, supplier.get());
+        break;
+      case DOUBLE:
+        Assert.assertEquals("double value should be equal", expected, supplier.get());
+        break;
+      case STRING:
+        Assert.assertTrue("Should expect a CharSequence", expected instanceof CharSequence);
+        Assert.assertEquals("string should be equal",
+            String.valueOf(expected), supplier.get().toString());
+        break;
+      case DATE:
+        Assert.assertTrue("Should expect a Date", expected instanceof LocalDate);
+        LocalDate date = ChronoUnit.DAYS.addTo(EPOCH_DAY, (int) supplier.get());
+        Assert.assertEquals("date should be equal", expected, date);
+        break;
+      case TIME:
+        Assert.assertTrue("Should expect a LocalTime", expected instanceof LocalTime);
+        int milliseconds = (int) (((LocalTime) expected).toNanoOfDay() / 1000_000);
+        Assert.assertEquals("time millis should be equal", milliseconds, supplier.get());
+        break;
+      case TIMESTAMP:
+        if (((Types.TimestampType) type).shouldAdjustToUTC()) {
+          Assert.assertTrue("Should expect a OffsetDataTime", expected instanceof OffsetDateTime);
+          OffsetDateTime ts = (OffsetDateTime) expected;
+          Assert.assertEquals("OffsetDataTime should be equal", ts.toLocalDateTime(),
+              ((TimestampData) supplier.get()).toLocalDateTime());

Review comment:
       There is no direct method to convert `TimestampData` to `OffsetDateTime`.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #1266: Flink: update parquet reader with schema visitor

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1266:
URL: https://github.com/apache/iceberg/pull/1266#discussion_r466798700



##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java
##########
@@ -19,64 +19,701 @@
 
 package org.apache.iceberg.flink.data;
 
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.time.Instant;
+import java.time.ZoneOffset;
 import java.util.List;
-import org.apache.flink.types.Row;
+import java.util.Map;
+import org.apache.commons.lang3.ArrayUtils;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.MapData;
+import org.apache.flink.table.data.RawValueData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
 import org.apache.iceberg.Schema;
-import org.apache.iceberg.data.parquet.BaseParquetReaders;
 import org.apache.iceberg.parquet.ParquetValueReader;
 import org.apache.iceberg.parquet.ParquetValueReaders;
+import org.apache.iceberg.parquet.TypeWithSchemaVisitor;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
 import org.apache.iceberg.types.Types;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.io.api.Binary;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.LogicalTypeAnnotation.DecimalLogicalTypeAnnotation;
 import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
 import org.apache.parquet.schema.Type;
 
-public class FlinkParquetReaders extends BaseParquetReaders<Row> {
+class FlinkParquetReaders {
+  private FlinkParquetReaders() {
+  }
 
-  private static final FlinkParquetReaders INSTANCE = new FlinkParquetReaders();
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema, MessageType fileSchema) {
+    return buildReader(expectedSchema, fileSchema, ImmutableMap.of());
+  }
 
-  private FlinkParquetReaders() {
+  @SuppressWarnings("unchecked")
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema,
+                                                        MessageType fileSchema,
+                                                        Map<Integer, ?> idToConstant) {
+    return (ParquetValueReader<RowData>) TypeWithSchemaVisitor.visit(expectedSchema.asStruct(), fileSchema,
+        new ReadBuilder(fileSchema, idToConstant)
+    );
+  }
+
+  private static class ReadBuilder extends TypeWithSchemaVisitor<ParquetValueReader<?>> {
+    private final MessageType type;
+    private final Map<Integer, ?> idToConstant;
+
+    ReadBuilder(MessageType type, Map<Integer, ?> idToConstant) {
+      this.type = type;
+      this.idToConstant = idToConstant;
+    }
+
+    @Override
+    public ParquetValueReader<RowData> message(Types.StructType expected, MessageType message,
+                                               List<ParquetValueReader<?>> fieldReaders) {
+      return struct(expected, message.asGroupType(), fieldReaders);
+    }
+
+    @Override
+    public ParquetValueReader<RowData> struct(Types.StructType expected, GroupType struct,
+                                              List<ParquetValueReader<?>> fieldReaders) {
+      // match the expected struct's order
+      Map<Integer, ParquetValueReader<?>> readersById = Maps.newHashMap();
+      Map<Integer, Type> typesById = Maps.newHashMap();
+      List<Type> fields = struct.getFields();
+      for (int i = 0; i < fields.size(); i += 1) {
+        Type fieldType = fields.get(i);
+        int fieldD = type.getMaxDefinitionLevel(path(fieldType.getName())) - 1;
+        if (fieldType.getId() != null) {
+          int id = fieldType.getId().intValue();
+          readersById.put(id, ParquetValueReaders.option(fieldType, fieldD, fieldReaders.get(i)));
+          typesById.put(id, fieldType);
+        }
+      }
+
+      List<Types.NestedField> expectedFields = expected != null ?
+          expected.fields() : ImmutableList.of();
+      List<ParquetValueReader<?>> reorderedFields = Lists.newArrayListWithExpectedSize(
+          expectedFields.size());
+      List<Type> types = Lists.newArrayListWithExpectedSize(expectedFields.size());
+      for (Types.NestedField field : expectedFields) {
+        int id = field.fieldId();
+        if (idToConstant.containsKey(id)) {
+          // containsKey is used because the constant may be null
+          reorderedFields.add(ParquetValueReaders.constant(idToConstant.get(id)));
+          types.add(null);
+        } else {
+          ParquetValueReader<?> reader = readersById.get(id);
+          if (reader != null) {
+            reorderedFields.add(reader);
+            types.add(typesById.get(id));
+          } else {
+            reorderedFields.add(ParquetValueReaders.nulls());
+            types.add(null);
+          }
+        }
+      }
+
+      return new RowDataReader(types, reorderedFields);
+    }
+
+    @Override
+    public ParquetValueReader<?> list(Types.ListType expectedList, GroupType array,
+                                      ParquetValueReader<?> elementReader) {
+      GroupType repeated = array.getFields().get(0).asGroupType();
+      String[] repeatedPath = currentPath();
+
+      int repeatedD = type.getMaxDefinitionLevel(repeatedPath) - 1;
+      int repeatedR = type.getMaxRepetitionLevel(repeatedPath) - 1;
+
+      Type elementType = repeated.getType(0);
+      int elementD = type.getMaxDefinitionLevel(path(elementType.getName())) - 1;
+
+      return new ArrayReader<>(repeatedD, repeatedR, ParquetValueReaders.option(elementType, elementD, elementReader));
+    }
+
+    @Override
+    public ParquetValueReader<?> map(Types.MapType expectedMap, GroupType map,
+                                     ParquetValueReader<?> keyReader,
+                                     ParquetValueReader<?> valueReader) {
+      GroupType repeatedKeyValue = map.getFields().get(0).asGroupType();
+      String[] repeatedPath = currentPath();
+
+      int repeatedD = type.getMaxDefinitionLevel(repeatedPath) - 1;
+      int repeatedR = type.getMaxRepetitionLevel(repeatedPath) - 1;
+
+      Type keyType = repeatedKeyValue.getType(0);
+      int keyD = type.getMaxDefinitionLevel(path(keyType.getName())) - 1;
+      Type valueType = repeatedKeyValue.getType(1);
+      int valueD = type.getMaxDefinitionLevel(path(valueType.getName())) - 1;
+
+      return new MapReader<>(repeatedD, repeatedR,
+          ParquetValueReaders.option(keyType, keyD, keyReader),
+          ParquetValueReaders.option(valueType, valueD, valueReader));
+    }
+
+    @Override
+    public ParquetValueReader<?> primitive(org.apache.iceberg.types.Type.PrimitiveType expected,
+                                           PrimitiveType primitive) {
+      ColumnDescriptor desc = type.getColumnDescription(currentPath());
+
+      if (primitive.getOriginalType() != null) {
+        switch (primitive.getOriginalType()) {
+          case ENUM:
+          case JSON:
+          case UTF8:
+            return new StringReader(desc);
+          case INT_8:
+          case INT_16:
+          case INT_32:
+          case DATE:
+            if (expected != null && expected.typeId() == Types.LongType.get().typeId()) {
+              return new ParquetValueReaders.IntAsLongReader(desc);
+            } else {
+              return new ParquetValueReaders.UnboxedReader<>(desc);
+            }
+          case TIME_MICROS:
+            return new LossyMicrosToMillisTimeReader(desc);
+          case INT_64:
+            return new ParquetValueReaders.UnboxedReader<>(desc);
+          case TIMESTAMP_MICROS:
+            if (((Types.TimestampType) expected).shouldAdjustToUTC()) {
+              return new TimestampTzReader(desc);
+            } else {
+              return new TimestampReader(desc);
+            }
+          case DECIMAL:
+            DecimalLogicalTypeAnnotation decimal = (DecimalLogicalTypeAnnotation) primitive.getLogicalTypeAnnotation();
+            switch (primitive.getPrimitiveTypeName()) {
+              case BINARY:
+              case FIXED_LEN_BYTE_ARRAY:
+                return new BinaryDecimalReader(desc, decimal.getScale());
+              case INT64:
+                return new LongDecimalReader(desc, decimal.getPrecision(), decimal.getScale());
+              case INT32:
+                return new IntegerDecimalReader(desc, decimal.getPrecision(), decimal.getScale());
+              default:
+                throw new UnsupportedOperationException(
+                    "Unsupported base type for decimal: " + primitive.getPrimitiveTypeName());
+            }
+          case BSON:
+            return new ParquetValueReaders.ByteArrayReader(desc);
+          default:
+            throw new UnsupportedOperationException(
+                "Unsupported logical type: " + primitive.getOriginalType());
+        }
+      }
+
+      switch (primitive.getPrimitiveTypeName()) {
+        case FIXED_LEN_BYTE_ARRAY:
+        case BINARY:
+          return new ParquetValueReaders.ByteArrayReader(desc);
+        case INT32:
+          if (expected != null && expected.typeId() == org.apache.iceberg.types.Type.TypeID.LONG) {
+            return new ParquetValueReaders.IntAsLongReader(desc);
+          } else {
+            return new ParquetValueReaders.UnboxedReader<>(desc);
+          }
+        case FLOAT:
+          if (expected != null && expected.typeId() == org.apache.iceberg.types.Type.TypeID.DOUBLE) {
+            return new ParquetValueReaders.FloatAsDoubleReader(desc);
+          } else {
+            return new ParquetValueReaders.UnboxedReader<>(desc);
+          }
+        case BOOLEAN:
+        case INT64:
+        case DOUBLE:
+          return new ParquetValueReaders.UnboxedReader<>(desc);
+        default:
+          throw new UnsupportedOperationException("Unsupported type: " + primitive);
+      }
+    }
+  }
+
+  private static class BinaryDecimalReader extends ParquetValueReaders.PrimitiveReader<DecimalData> {
+    private final int scale;
+
+    BinaryDecimalReader(ColumnDescriptor desc, int scale) {
+      super(desc);
+      this.scale = scale;
+    }
+
+    @Override
+    public DecimalData read(DecimalData ignored) {
+      Binary binary = column.nextBinary();
+      BigDecimal bigDecimal = new BigDecimal(new BigInteger(binary.getBytes()), scale);
+      return DecimalData.fromBigDecimal(bigDecimal, bigDecimal.precision(), scale);

Review comment:
       Here we pass the `bigDecimal.precision()` to `DecimalData.fromBigDecimal`,  I think it's incorrect.  Because the precision expected for DecimalData is the precision of data type,  it will use this value to do `ROUND`  etc. You could see the comments in `DecimalData`: 
   
   ```java
   	// The semantics of the fields are as follows:
   	//  - `precision` and `scale` represent the precision and scale of SQL decimal type
   	//  - If `decimalVal` is set, it represents the whole decimal value
   	//  - Otherwise, the decimal value is longVal/(10^scale).
   	//
   	// Note that the (precision, scale) must be correct.
   	// if precision > MAX_COMPACT_PRECISION,
   	//   `decimalVal` represents the value. `longVal` is undefined
   	// otherwise, (longVal, scale) represents the value
   	//   `decimalVal` may be set and cached
   
   	final int precision;
   	final int scale;
   ``` 
   
   I think we need to pass the decimal type's precision and scale,   and the use the `DecimalData.fromUnscaledBytes(binary.getBytes(), precision, scale);`  to construct the DecimalData.
   

##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java
##########
@@ -19,64 +19,701 @@
 
 package org.apache.iceberg.flink.data;
 
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.time.Instant;
+import java.time.ZoneOffset;
 import java.util.List;
-import org.apache.flink.types.Row;
+import java.util.Map;
+import org.apache.commons.lang3.ArrayUtils;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.MapData;
+import org.apache.flink.table.data.RawValueData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
 import org.apache.iceberg.Schema;
-import org.apache.iceberg.data.parquet.BaseParquetReaders;
 import org.apache.iceberg.parquet.ParquetValueReader;
 import org.apache.iceberg.parquet.ParquetValueReaders;
+import org.apache.iceberg.parquet.TypeWithSchemaVisitor;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
 import org.apache.iceberg.types.Types;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.io.api.Binary;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.LogicalTypeAnnotation.DecimalLogicalTypeAnnotation;
 import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
 import org.apache.parquet.schema.Type;
 
-public class FlinkParquetReaders extends BaseParquetReaders<Row> {
+class FlinkParquetReaders {
+  private FlinkParquetReaders() {
+  }
 
-  private static final FlinkParquetReaders INSTANCE = new FlinkParquetReaders();
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema, MessageType fileSchema) {
+    return buildReader(expectedSchema, fileSchema, ImmutableMap.of());
+  }
 
-  private FlinkParquetReaders() {
+  @SuppressWarnings("unchecked")
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema,
+                                                        MessageType fileSchema,
+                                                        Map<Integer, ?> idToConstant) {
+    return (ParquetValueReader<RowData>) TypeWithSchemaVisitor.visit(expectedSchema.asStruct(), fileSchema,
+        new ReadBuilder(fileSchema, idToConstant)
+    );
+  }
+
+  private static class ReadBuilder extends TypeWithSchemaVisitor<ParquetValueReader<?>> {
+    private final MessageType type;
+    private final Map<Integer, ?> idToConstant;
+
+    ReadBuilder(MessageType type, Map<Integer, ?> idToConstant) {
+      this.type = type;
+      this.idToConstant = idToConstant;
+    }
+
+    @Override
+    public ParquetValueReader<RowData> message(Types.StructType expected, MessageType message,
+                                               List<ParquetValueReader<?>> fieldReaders) {
+      return struct(expected, message.asGroupType(), fieldReaders);
+    }
+
+    @Override
+    public ParquetValueReader<RowData> struct(Types.StructType expected, GroupType struct,
+                                              List<ParquetValueReader<?>> fieldReaders) {
+      // match the expected struct's order
+      Map<Integer, ParquetValueReader<?>> readersById = Maps.newHashMap();
+      Map<Integer, Type> typesById = Maps.newHashMap();
+      List<Type> fields = struct.getFields();
+      for (int i = 0; i < fields.size(); i += 1) {
+        Type fieldType = fields.get(i);
+        int fieldD = type.getMaxDefinitionLevel(path(fieldType.getName())) - 1;
+        if (fieldType.getId() != null) {
+          int id = fieldType.getId().intValue();
+          readersById.put(id, ParquetValueReaders.option(fieldType, fieldD, fieldReaders.get(i)));
+          typesById.put(id, fieldType);
+        }
+      }
+
+      List<Types.NestedField> expectedFields = expected != null ?
+          expected.fields() : ImmutableList.of();
+      List<ParquetValueReader<?>> reorderedFields = Lists.newArrayListWithExpectedSize(
+          expectedFields.size());
+      List<Type> types = Lists.newArrayListWithExpectedSize(expectedFields.size());
+      for (Types.NestedField field : expectedFields) {
+        int id = field.fieldId();
+        if (idToConstant.containsKey(id)) {
+          // containsKey is used because the constant may be null
+          reorderedFields.add(ParquetValueReaders.constant(idToConstant.get(id)));
+          types.add(null);
+        } else {
+          ParquetValueReader<?> reader = readersById.get(id);
+          if (reader != null) {
+            reorderedFields.add(reader);
+            types.add(typesById.get(id));
+          } else {
+            reorderedFields.add(ParquetValueReaders.nulls());
+            types.add(null);
+          }
+        }
+      }
+
+      return new RowDataReader(types, reorderedFields);
+    }
+
+    @Override
+    public ParquetValueReader<?> list(Types.ListType expectedList, GroupType array,
+                                      ParquetValueReader<?> elementReader) {
+      GroupType repeated = array.getFields().get(0).asGroupType();
+      String[] repeatedPath = currentPath();
+
+      int repeatedD = type.getMaxDefinitionLevel(repeatedPath) - 1;
+      int repeatedR = type.getMaxRepetitionLevel(repeatedPath) - 1;
+
+      Type elementType = repeated.getType(0);
+      int elementD = type.getMaxDefinitionLevel(path(elementType.getName())) - 1;
+
+      return new ArrayReader<>(repeatedD, repeatedR, ParquetValueReaders.option(elementType, elementD, elementReader));
+    }
+
+    @Override
+    public ParquetValueReader<?> map(Types.MapType expectedMap, GroupType map,
+                                     ParquetValueReader<?> keyReader,
+                                     ParquetValueReader<?> valueReader) {
+      GroupType repeatedKeyValue = map.getFields().get(0).asGroupType();
+      String[] repeatedPath = currentPath();
+
+      int repeatedD = type.getMaxDefinitionLevel(repeatedPath) - 1;
+      int repeatedR = type.getMaxRepetitionLevel(repeatedPath) - 1;
+
+      Type keyType = repeatedKeyValue.getType(0);
+      int keyD = type.getMaxDefinitionLevel(path(keyType.getName())) - 1;
+      Type valueType = repeatedKeyValue.getType(1);
+      int valueD = type.getMaxDefinitionLevel(path(valueType.getName())) - 1;
+
+      return new MapReader<>(repeatedD, repeatedR,
+          ParquetValueReaders.option(keyType, keyD, keyReader),
+          ParquetValueReaders.option(valueType, valueD, valueReader));
+    }
+
+    @Override
+    public ParquetValueReader<?> primitive(org.apache.iceberg.types.Type.PrimitiveType expected,
+                                           PrimitiveType primitive) {
+      ColumnDescriptor desc = type.getColumnDescription(currentPath());
+
+      if (primitive.getOriginalType() != null) {
+        switch (primitive.getOriginalType()) {
+          case ENUM:
+          case JSON:
+          case UTF8:
+            return new StringReader(desc);
+          case INT_8:
+          case INT_16:
+          case INT_32:
+          case DATE:
+            if (expected != null && expected.typeId() == Types.LongType.get().typeId()) {
+              return new ParquetValueReaders.IntAsLongReader(desc);
+            } else {
+              return new ParquetValueReaders.UnboxedReader<>(desc);
+            }
+          case TIME_MICROS:
+            return new LossyMicrosToMillisTimeReader(desc);
+          case INT_64:
+            return new ParquetValueReaders.UnboxedReader<>(desc);
+          case TIMESTAMP_MICROS:
+            if (((Types.TimestampType) expected).shouldAdjustToUTC()) {
+              return new TimestampTzReader(desc);
+            } else {
+              return new TimestampReader(desc);
+            }
+          case DECIMAL:
+            DecimalLogicalTypeAnnotation decimal = (DecimalLogicalTypeAnnotation) primitive.getLogicalTypeAnnotation();
+            switch (primitive.getPrimitiveTypeName()) {
+              case BINARY:
+              case FIXED_LEN_BYTE_ARRAY:
+                return new BinaryDecimalReader(desc, decimal.getScale());
+              case INT64:
+                return new LongDecimalReader(desc, decimal.getPrecision(), decimal.getScale());
+              case INT32:
+                return new IntegerDecimalReader(desc, decimal.getPrecision(), decimal.getScale());
+              default:
+                throw new UnsupportedOperationException(
+                    "Unsupported base type for decimal: " + primitive.getPrimitiveTypeName());
+            }
+          case BSON:
+            return new ParquetValueReaders.ByteArrayReader(desc);
+          default:
+            throw new UnsupportedOperationException(
+                "Unsupported logical type: " + primitive.getOriginalType());
+        }
+      }
+
+      switch (primitive.getPrimitiveTypeName()) {
+        case FIXED_LEN_BYTE_ARRAY:
+        case BINARY:
+          return new ParquetValueReaders.ByteArrayReader(desc);
+        case INT32:
+          if (expected != null && expected.typeId() == org.apache.iceberg.types.Type.TypeID.LONG) {
+            return new ParquetValueReaders.IntAsLongReader(desc);
+          } else {
+            return new ParquetValueReaders.UnboxedReader<>(desc);
+          }
+        case FLOAT:
+          if (expected != null && expected.typeId() == org.apache.iceberg.types.Type.TypeID.DOUBLE) {
+            return new ParquetValueReaders.FloatAsDoubleReader(desc);
+          } else {
+            return new ParquetValueReaders.UnboxedReader<>(desc);
+          }
+        case BOOLEAN:
+        case INT64:
+        case DOUBLE:
+          return new ParquetValueReaders.UnboxedReader<>(desc);
+        default:
+          throw new UnsupportedOperationException("Unsupported type: " + primitive);
+      }
+    }
+  }
+
+  private static class BinaryDecimalReader extends ParquetValueReaders.PrimitiveReader<DecimalData> {
+    private final int scale;
+
+    BinaryDecimalReader(ColumnDescriptor desc, int scale) {
+      super(desc);
+      this.scale = scale;
+    }
+
+    @Override
+    public DecimalData read(DecimalData ignored) {
+      Binary binary = column.nextBinary();
+      BigDecimal bigDecimal = new BigDecimal(new BigInteger(binary.getBytes()), scale);
+      return DecimalData.fromBigDecimal(bigDecimal, bigDecimal.precision(), scale);
+    }
+  }
+
+  private static class IntegerDecimalReader extends ParquetValueReaders.PrimitiveReader<DecimalData> {
+    private final int precision;
+    private final int scale;
+
+    IntegerDecimalReader(ColumnDescriptor desc, int precision, int scale) {
+      super(desc);
+      this.precision = precision;
+      this.scale = scale;
+    }
+
+    @Override
+    public DecimalData read(DecimalData ignored) {
+      return DecimalData.fromUnscaledLong(column.nextInteger(), precision, scale);
+    }
+  }
+
+  private static class LongDecimalReader extends ParquetValueReaders.PrimitiveReader<DecimalData> {
+    private final int precision;
+    private final int scale;
+
+    LongDecimalReader(ColumnDescriptor desc, int precision, int scale) {
+      super(desc);
+      this.precision = precision;
+      this.scale = scale;
+    }
+
+    @Override
+    public DecimalData read(DecimalData ignored) {
+      return DecimalData.fromUnscaledLong(column.nextLong(), precision, scale);
+    }
+  }
+
+  private static class TimestampTzReader extends ParquetValueReaders.UnboxedReader<TimestampData> {
+    TimestampTzReader(ColumnDescriptor desc) {
+      super(desc);
+    }
+
+    @Override
+    public TimestampData read(TimestampData ignored) {
+      long value = readLong();
+      return TimestampData.fromLocalDateTime(Instant.ofEpochSecond(Math.floorDiv(value, 1000_000),
+          Math.floorMod(value, 1000_000) * 1000)
+          .atOffset(ZoneOffset.UTC)
+          .toLocalDateTime());
+    }
+
+    @Override
+    public long readLong() {
+      return column.nextLong();
+    }
   }
 
-  public static ParquetValueReader<Row> buildReader(Schema expectedSchema, MessageType fileSchema) {
-    return INSTANCE.createReader(expectedSchema, fileSchema);
+  private static class TimestampReader extends ParquetValueReaders.UnboxedReader<TimestampData> {
+    TimestampReader(ColumnDescriptor desc) {
+      super(desc);
+    }
+
+    @Override
+    public TimestampData read(TimestampData ignored) {
+      long value = readLong();
+      return TimestampData.fromInstant(Instant.ofEpochSecond(Math.floorDiv(value, 1000_000),
+          Math.floorMod(value, 1000_000) * 1000));
+    }
+
+    @Override
+    public long readLong() {
+      return column.nextLong();
+    }
   }
 
-  @Override
-  protected ParquetValueReader<Row> createStructReader(List<Type> types,
-                                                       List<ParquetValueReader<?>> fieldReaders,
-                                                       Types.StructType structType) {
-    return new RowReader(types, fieldReaders, structType);
+  private static class StringReader extends ParquetValueReaders.PrimitiveReader<StringData> {
+    StringReader(ColumnDescriptor desc) {
+      super(desc);
+    }
+
+    @Override
+    public StringData read(StringData ignored) {
+      Binary binary = column.nextBinary();
+      ByteBuffer buffer = binary.toByteBuffer();
+      if (buffer.hasArray()) {
+        return StringData.fromBytes(
+            buffer.array(), buffer.arrayOffset() + buffer.position(), buffer.remaining());
+      } else {
+        return StringData.fromBytes(binary.getBytes());
+      }
+    }
+  }
+
+  private static class LossyMicrosToMillisTimeReader extends ParquetValueReaders.PrimitiveReader<Integer> {
+    LossyMicrosToMillisTimeReader(ColumnDescriptor desc) {
+      super(desc);
+    }
+
+    @Override
+    public Integer read(Integer reuse) {
+      // Discard microseconds since Flink uses millisecond unit for TIME type.
+      return (int) Math.floorDiv(column.nextLong(), 1000);
+    }
+  }
+
+  private static class ArrayReader<E> extends ParquetValueReaders.RepeatedReader<ArrayData, ReusableArrayData, E> {
+    private int readPos = 0;
+    private int writePos = 0;
+
+    ArrayReader(int definitionLevel, int repetitionLevel, ParquetValueReader<E> reader) {
+      super(definitionLevel, repetitionLevel, reader);
+    }
+
+    @Override
+    protected ReusableArrayData newListData(ArrayData reuse) {
+      this.readPos = 0;
+      this.writePos = 0;
+
+      if (reuse instanceof ReusableArrayData) {
+        return (ReusableArrayData) reuse;
+      } else {
+        return new ReusableArrayData();
+      }
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    protected E getElement(ReusableArrayData list) {
+      E value = null;
+      if (readPos < list.capacity()) {
+        value = (E) list.values[readPos];
+      }
+
+      readPos += 1;

Review comment:
       Q: when the case `readPos >= list.capacity()`, seems we don't need to increment the readPos ?  For my understanding,  in that case the `getElement` should return a null, means that we will read the value without a provided reuse element.  Then we also shouldn't move the readPos because we actually don't reuse any object from the list.  
   
   Of course, if we increment readPos here,  the logic is also correct. but seem have less possibility to reuse the object in array. 
   
   How do you think ? 

##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java
##########
@@ -19,64 +19,701 @@
 
 package org.apache.iceberg.flink.data;
 
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.time.Instant;
+import java.time.ZoneOffset;
 import java.util.List;
-import org.apache.flink.types.Row;
+import java.util.Map;
+import org.apache.commons.lang3.ArrayUtils;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.MapData;
+import org.apache.flink.table.data.RawValueData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
 import org.apache.iceberg.Schema;
-import org.apache.iceberg.data.parquet.BaseParquetReaders;
 import org.apache.iceberg.parquet.ParquetValueReader;
 import org.apache.iceberg.parquet.ParquetValueReaders;
+import org.apache.iceberg.parquet.TypeWithSchemaVisitor;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
 import org.apache.iceberg.types.Types;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.io.api.Binary;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.LogicalTypeAnnotation.DecimalLogicalTypeAnnotation;
 import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
 import org.apache.parquet.schema.Type;
 
-public class FlinkParquetReaders extends BaseParquetReaders<Row> {
+class FlinkParquetReaders {
+  private FlinkParquetReaders() {
+  }
 
-  private static final FlinkParquetReaders INSTANCE = new FlinkParquetReaders();
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema, MessageType fileSchema) {
+    return buildReader(expectedSchema, fileSchema, ImmutableMap.of());
+  }
 
-  private FlinkParquetReaders() {
+  @SuppressWarnings("unchecked")
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema,
+                                                        MessageType fileSchema,
+                                                        Map<Integer, ?> idToConstant) {
+    return (ParquetValueReader<RowData>) TypeWithSchemaVisitor.visit(expectedSchema.asStruct(), fileSchema,
+        new ReadBuilder(fileSchema, idToConstant)
+    );
+  }
+
+  private static class ReadBuilder extends TypeWithSchemaVisitor<ParquetValueReader<?>> {
+    private final MessageType type;
+    private final Map<Integer, ?> idToConstant;
+
+    ReadBuilder(MessageType type, Map<Integer, ?> idToConstant) {
+      this.type = type;
+      this.idToConstant = idToConstant;
+    }
+
+    @Override
+    public ParquetValueReader<RowData> message(Types.StructType expected, MessageType message,
+                                               List<ParquetValueReader<?>> fieldReaders) {
+      return struct(expected, message.asGroupType(), fieldReaders);
+    }
+
+    @Override
+    public ParquetValueReader<RowData> struct(Types.StructType expected, GroupType struct,
+                                              List<ParquetValueReader<?>> fieldReaders) {
+      // match the expected struct's order
+      Map<Integer, ParquetValueReader<?>> readersById = Maps.newHashMap();
+      Map<Integer, Type> typesById = Maps.newHashMap();
+      List<Type> fields = struct.getFields();
+      for (int i = 0; i < fields.size(); i += 1) {
+        Type fieldType = fields.get(i);
+        int fieldD = type.getMaxDefinitionLevel(path(fieldType.getName())) - 1;
+        if (fieldType.getId() != null) {
+          int id = fieldType.getId().intValue();
+          readersById.put(id, ParquetValueReaders.option(fieldType, fieldD, fieldReaders.get(i)));
+          typesById.put(id, fieldType);
+        }
+      }
+
+      List<Types.NestedField> expectedFields = expected != null ?
+          expected.fields() : ImmutableList.of();
+      List<ParquetValueReader<?>> reorderedFields = Lists.newArrayListWithExpectedSize(
+          expectedFields.size());
+      List<Type> types = Lists.newArrayListWithExpectedSize(expectedFields.size());
+      for (Types.NestedField field : expectedFields) {
+        int id = field.fieldId();
+        if (idToConstant.containsKey(id)) {
+          // containsKey is used because the constant may be null
+          reorderedFields.add(ParquetValueReaders.constant(idToConstant.get(id)));
+          types.add(null);
+        } else {
+          ParquetValueReader<?> reader = readersById.get(id);
+          if (reader != null) {
+            reorderedFields.add(reader);
+            types.add(typesById.get(id));
+          } else {
+            reorderedFields.add(ParquetValueReaders.nulls());
+            types.add(null);
+          }
+        }
+      }
+
+      return new RowDataReader(types, reorderedFields);
+    }
+
+    @Override
+    public ParquetValueReader<?> list(Types.ListType expectedList, GroupType array,
+                                      ParquetValueReader<?> elementReader) {
+      GroupType repeated = array.getFields().get(0).asGroupType();
+      String[] repeatedPath = currentPath();
+
+      int repeatedD = type.getMaxDefinitionLevel(repeatedPath) - 1;
+      int repeatedR = type.getMaxRepetitionLevel(repeatedPath) - 1;
+
+      Type elementType = repeated.getType(0);
+      int elementD = type.getMaxDefinitionLevel(path(elementType.getName())) - 1;
+
+      return new ArrayReader<>(repeatedD, repeatedR, ParquetValueReaders.option(elementType, elementD, elementReader));
+    }
+
+    @Override
+    public ParquetValueReader<?> map(Types.MapType expectedMap, GroupType map,
+                                     ParquetValueReader<?> keyReader,
+                                     ParquetValueReader<?> valueReader) {
+      GroupType repeatedKeyValue = map.getFields().get(0).asGroupType();
+      String[] repeatedPath = currentPath();
+
+      int repeatedD = type.getMaxDefinitionLevel(repeatedPath) - 1;
+      int repeatedR = type.getMaxRepetitionLevel(repeatedPath) - 1;
+
+      Type keyType = repeatedKeyValue.getType(0);
+      int keyD = type.getMaxDefinitionLevel(path(keyType.getName())) - 1;
+      Type valueType = repeatedKeyValue.getType(1);
+      int valueD = type.getMaxDefinitionLevel(path(valueType.getName())) - 1;
+
+      return new MapReader<>(repeatedD, repeatedR,
+          ParquetValueReaders.option(keyType, keyD, keyReader),
+          ParquetValueReaders.option(valueType, valueD, valueReader));
+    }
+
+    @Override
+    public ParquetValueReader<?> primitive(org.apache.iceberg.types.Type.PrimitiveType expected,
+                                           PrimitiveType primitive) {
+      ColumnDescriptor desc = type.getColumnDescription(currentPath());
+
+      if (primitive.getOriginalType() != null) {
+        switch (primitive.getOriginalType()) {
+          case ENUM:
+          case JSON:
+          case UTF8:
+            return new StringReader(desc);
+          case INT_8:
+          case INT_16:
+          case INT_32:
+          case DATE:
+            if (expected != null && expected.typeId() == Types.LongType.get().typeId()) {
+              return new ParquetValueReaders.IntAsLongReader(desc);
+            } else {
+              return new ParquetValueReaders.UnboxedReader<>(desc);
+            }
+          case TIME_MICROS:
+            return new LossyMicrosToMillisTimeReader(desc);
+          case INT_64:
+            return new ParquetValueReaders.UnboxedReader<>(desc);
+          case TIMESTAMP_MICROS:
+            if (((Types.TimestampType) expected).shouldAdjustToUTC()) {
+              return new TimestampTzReader(desc);
+            } else {
+              return new TimestampReader(desc);
+            }
+          case DECIMAL:
+            DecimalLogicalTypeAnnotation decimal = (DecimalLogicalTypeAnnotation) primitive.getLogicalTypeAnnotation();
+            switch (primitive.getPrimitiveTypeName()) {
+              case BINARY:
+              case FIXED_LEN_BYTE_ARRAY:
+                return new BinaryDecimalReader(desc, decimal.getScale());
+              case INT64:
+                return new LongDecimalReader(desc, decimal.getPrecision(), decimal.getScale());
+              case INT32:
+                return new IntegerDecimalReader(desc, decimal.getPrecision(), decimal.getScale());
+              default:
+                throw new UnsupportedOperationException(
+                    "Unsupported base type for decimal: " + primitive.getPrimitiveTypeName());
+            }
+          case BSON:
+            return new ParquetValueReaders.ByteArrayReader(desc);
+          default:
+            throw new UnsupportedOperationException(
+                "Unsupported logical type: " + primitive.getOriginalType());
+        }
+      }
+
+      switch (primitive.getPrimitiveTypeName()) {
+        case FIXED_LEN_BYTE_ARRAY:
+        case BINARY:
+          return new ParquetValueReaders.ByteArrayReader(desc);
+        case INT32:
+          if (expected != null && expected.typeId() == org.apache.iceberg.types.Type.TypeID.LONG) {
+            return new ParquetValueReaders.IntAsLongReader(desc);
+          } else {
+            return new ParquetValueReaders.UnboxedReader<>(desc);
+          }
+        case FLOAT:
+          if (expected != null && expected.typeId() == org.apache.iceberg.types.Type.TypeID.DOUBLE) {
+            return new ParquetValueReaders.FloatAsDoubleReader(desc);
+          } else {
+            return new ParquetValueReaders.UnboxedReader<>(desc);
+          }
+        case BOOLEAN:
+        case INT64:
+        case DOUBLE:
+          return new ParquetValueReaders.UnboxedReader<>(desc);
+        default:
+          throw new UnsupportedOperationException("Unsupported type: " + primitive);
+      }
+    }
+  }
+
+  private static class BinaryDecimalReader extends ParquetValueReaders.PrimitiveReader<DecimalData> {
+    private final int scale;
+
+    BinaryDecimalReader(ColumnDescriptor desc, int scale) {
+      super(desc);
+      this.scale = scale;
+    }
+
+    @Override
+    public DecimalData read(DecimalData ignored) {
+      Binary binary = column.nextBinary();
+      BigDecimal bigDecimal = new BigDecimal(new BigInteger(binary.getBytes()), scale);
+      return DecimalData.fromBigDecimal(bigDecimal, bigDecimal.precision(), scale);

Review comment:
       BTW, is it possible that  we have few unit tests to address this ? 

##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java
##########
@@ -19,64 +19,701 @@
 
 package org.apache.iceberg.flink.data;
 
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.time.Instant;
+import java.time.ZoneOffset;
 import java.util.List;
-import org.apache.flink.types.Row;
+import java.util.Map;
+import org.apache.commons.lang3.ArrayUtils;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.MapData;
+import org.apache.flink.table.data.RawValueData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
 import org.apache.iceberg.Schema;
-import org.apache.iceberg.data.parquet.BaseParquetReaders;
 import org.apache.iceberg.parquet.ParquetValueReader;
 import org.apache.iceberg.parquet.ParquetValueReaders;
+import org.apache.iceberg.parquet.TypeWithSchemaVisitor;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
 import org.apache.iceberg.types.Types;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.io.api.Binary;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.LogicalTypeAnnotation.DecimalLogicalTypeAnnotation;
 import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
 import org.apache.parquet.schema.Type;
 
-public class FlinkParquetReaders extends BaseParquetReaders<Row> {
+class FlinkParquetReaders {
+  private FlinkParquetReaders() {
+  }
 
-  private static final FlinkParquetReaders INSTANCE = new FlinkParquetReaders();
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema, MessageType fileSchema) {
+    return buildReader(expectedSchema, fileSchema, ImmutableMap.of());
+  }
 
-  private FlinkParquetReaders() {
+  @SuppressWarnings("unchecked")
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema,
+                                                        MessageType fileSchema,
+                                                        Map<Integer, ?> idToConstant) {
+    return (ParquetValueReader<RowData>) TypeWithSchemaVisitor.visit(expectedSchema.asStruct(), fileSchema,
+        new ReadBuilder(fileSchema, idToConstant)
+    );
+  }
+
+  private static class ReadBuilder extends TypeWithSchemaVisitor<ParquetValueReader<?>> {
+    private final MessageType type;
+    private final Map<Integer, ?> idToConstant;
+
+    ReadBuilder(MessageType type, Map<Integer, ?> idToConstant) {
+      this.type = type;
+      this.idToConstant = idToConstant;
+    }
+
+    @Override
+    public ParquetValueReader<RowData> message(Types.StructType expected, MessageType message,
+                                               List<ParquetValueReader<?>> fieldReaders) {
+      return struct(expected, message.asGroupType(), fieldReaders);
+    }
+
+    @Override
+    public ParquetValueReader<RowData> struct(Types.StructType expected, GroupType struct,
+                                              List<ParquetValueReader<?>> fieldReaders) {
+      // match the expected struct's order
+      Map<Integer, ParquetValueReader<?>> readersById = Maps.newHashMap();
+      Map<Integer, Type> typesById = Maps.newHashMap();
+      List<Type> fields = struct.getFields();
+      for (int i = 0; i < fields.size(); i += 1) {
+        Type fieldType = fields.get(i);
+        int fieldD = type.getMaxDefinitionLevel(path(fieldType.getName())) - 1;
+        if (fieldType.getId() != null) {
+          int id = fieldType.getId().intValue();
+          readersById.put(id, ParquetValueReaders.option(fieldType, fieldD, fieldReaders.get(i)));
+          typesById.put(id, fieldType);
+        }
+      }
+
+      List<Types.NestedField> expectedFields = expected != null ?
+          expected.fields() : ImmutableList.of();
+      List<ParquetValueReader<?>> reorderedFields = Lists.newArrayListWithExpectedSize(
+          expectedFields.size());
+      List<Type> types = Lists.newArrayListWithExpectedSize(expectedFields.size());
+      for (Types.NestedField field : expectedFields) {
+        int id = field.fieldId();
+        if (idToConstant.containsKey(id)) {
+          // containsKey is used because the constant may be null
+          reorderedFields.add(ParquetValueReaders.constant(idToConstant.get(id)));
+          types.add(null);
+        } else {
+          ParquetValueReader<?> reader = readersById.get(id);
+          if (reader != null) {
+            reorderedFields.add(reader);
+            types.add(typesById.get(id));
+          } else {
+            reorderedFields.add(ParquetValueReaders.nulls());
+            types.add(null);
+          }
+        }
+      }
+
+      return new RowDataReader(types, reorderedFields);
+    }
+
+    @Override
+    public ParquetValueReader<?> list(Types.ListType expectedList, GroupType array,
+                                      ParquetValueReader<?> elementReader) {
+      GroupType repeated = array.getFields().get(0).asGroupType();
+      String[] repeatedPath = currentPath();
+
+      int repeatedD = type.getMaxDefinitionLevel(repeatedPath) - 1;
+      int repeatedR = type.getMaxRepetitionLevel(repeatedPath) - 1;
+
+      Type elementType = repeated.getType(0);
+      int elementD = type.getMaxDefinitionLevel(path(elementType.getName())) - 1;
+
+      return new ArrayReader<>(repeatedD, repeatedR, ParquetValueReaders.option(elementType, elementD, elementReader));
+    }
+
+    @Override
+    public ParquetValueReader<?> map(Types.MapType expectedMap, GroupType map,
+                                     ParquetValueReader<?> keyReader,
+                                     ParquetValueReader<?> valueReader) {
+      GroupType repeatedKeyValue = map.getFields().get(0).asGroupType();
+      String[] repeatedPath = currentPath();
+
+      int repeatedD = type.getMaxDefinitionLevel(repeatedPath) - 1;
+      int repeatedR = type.getMaxRepetitionLevel(repeatedPath) - 1;
+
+      Type keyType = repeatedKeyValue.getType(0);
+      int keyD = type.getMaxDefinitionLevel(path(keyType.getName())) - 1;
+      Type valueType = repeatedKeyValue.getType(1);
+      int valueD = type.getMaxDefinitionLevel(path(valueType.getName())) - 1;
+
+      return new MapReader<>(repeatedD, repeatedR,
+          ParquetValueReaders.option(keyType, keyD, keyReader),
+          ParquetValueReaders.option(valueType, valueD, valueReader));
+    }
+
+    @Override
+    public ParquetValueReader<?> primitive(org.apache.iceberg.types.Type.PrimitiveType expected,
+                                           PrimitiveType primitive) {
+      ColumnDescriptor desc = type.getColumnDescription(currentPath());
+
+      if (primitive.getOriginalType() != null) {
+        switch (primitive.getOriginalType()) {
+          case ENUM:
+          case JSON:
+          case UTF8:
+            return new StringReader(desc);
+          case INT_8:
+          case INT_16:
+          case INT_32:
+          case DATE:
+            if (expected != null && expected.typeId() == Types.LongType.get().typeId()) {
+              return new ParquetValueReaders.IntAsLongReader(desc);
+            } else {
+              return new ParquetValueReaders.UnboxedReader<>(desc);
+            }
+          case TIME_MICROS:
+            return new LossyMicrosToMillisTimeReader(desc);
+          case INT_64:
+            return new ParquetValueReaders.UnboxedReader<>(desc);
+          case TIMESTAMP_MICROS:
+            if (((Types.TimestampType) expected).shouldAdjustToUTC()) {
+              return new TimestampTzReader(desc);
+            } else {
+              return new TimestampReader(desc);
+            }
+          case DECIMAL:
+            DecimalLogicalTypeAnnotation decimal = (DecimalLogicalTypeAnnotation) primitive.getLogicalTypeAnnotation();
+            switch (primitive.getPrimitiveTypeName()) {
+              case BINARY:
+              case FIXED_LEN_BYTE_ARRAY:
+                return new BinaryDecimalReader(desc, decimal.getScale());
+              case INT64:
+                return new LongDecimalReader(desc, decimal.getPrecision(), decimal.getScale());
+              case INT32:
+                return new IntegerDecimalReader(desc, decimal.getPrecision(), decimal.getScale());
+              default:
+                throw new UnsupportedOperationException(
+                    "Unsupported base type for decimal: " + primitive.getPrimitiveTypeName());
+            }
+          case BSON:
+            return new ParquetValueReaders.ByteArrayReader(desc);
+          default:
+            throw new UnsupportedOperationException(
+                "Unsupported logical type: " + primitive.getOriginalType());
+        }
+      }
+
+      switch (primitive.getPrimitiveTypeName()) {
+        case FIXED_LEN_BYTE_ARRAY:
+        case BINARY:
+          return new ParquetValueReaders.ByteArrayReader(desc);
+        case INT32:
+          if (expected != null && expected.typeId() == org.apache.iceberg.types.Type.TypeID.LONG) {
+            return new ParquetValueReaders.IntAsLongReader(desc);
+          } else {
+            return new ParquetValueReaders.UnboxedReader<>(desc);
+          }
+        case FLOAT:
+          if (expected != null && expected.typeId() == org.apache.iceberg.types.Type.TypeID.DOUBLE) {
+            return new ParquetValueReaders.FloatAsDoubleReader(desc);
+          } else {
+            return new ParquetValueReaders.UnboxedReader<>(desc);
+          }
+        case BOOLEAN:
+        case INT64:
+        case DOUBLE:
+          return new ParquetValueReaders.UnboxedReader<>(desc);
+        default:
+          throw new UnsupportedOperationException("Unsupported type: " + primitive);
+      }
+    }
+  }
+
+  private static class BinaryDecimalReader extends ParquetValueReaders.PrimitiveReader<DecimalData> {
+    private final int scale;
+
+    BinaryDecimalReader(ColumnDescriptor desc, int scale) {
+      super(desc);
+      this.scale = scale;
+    }
+
+    @Override
+    public DecimalData read(DecimalData ignored) {
+      Binary binary = column.nextBinary();
+      BigDecimal bigDecimal = new BigDecimal(new BigInteger(binary.getBytes()), scale);
+      return DecimalData.fromBigDecimal(bigDecimal, bigDecimal.precision(), scale);
+    }
+  }
+
+  private static class IntegerDecimalReader extends ParquetValueReaders.PrimitiveReader<DecimalData> {
+    private final int precision;
+    private final int scale;
+
+    IntegerDecimalReader(ColumnDescriptor desc, int precision, int scale) {
+      super(desc);
+      this.precision = precision;
+      this.scale = scale;
+    }
+
+    @Override
+    public DecimalData read(DecimalData ignored) {
+      return DecimalData.fromUnscaledLong(column.nextInteger(), precision, scale);
+    }
+  }
+
+  private static class LongDecimalReader extends ParquetValueReaders.PrimitiveReader<DecimalData> {
+    private final int precision;
+    private final int scale;
+
+    LongDecimalReader(ColumnDescriptor desc, int precision, int scale) {
+      super(desc);
+      this.precision = precision;
+      this.scale = scale;
+    }
+
+    @Override
+    public DecimalData read(DecimalData ignored) {
+      return DecimalData.fromUnscaledLong(column.nextLong(), precision, scale);
+    }
+  }
+
+  private static class TimestampTzReader extends ParquetValueReaders.UnboxedReader<TimestampData> {
+    TimestampTzReader(ColumnDescriptor desc) {
+      super(desc);
+    }
+
+    @Override
+    public TimestampData read(TimestampData ignored) {
+      long value = readLong();
+      return TimestampData.fromLocalDateTime(Instant.ofEpochSecond(Math.floorDiv(value, 1000_000),
+          Math.floorMod(value, 1000_000) * 1000)
+          .atOffset(ZoneOffset.UTC)
+          .toLocalDateTime());

Review comment:
       Is it correct ?  For my understanding,   the timestamp with time zone don't need to convert to a `LocalDateTime` (because it has its own time zone),   while the timestamp without time zone need to.

##########
File path: data/src/test/java/org/apache/iceberg/data/RandomGenericData.java
##########
@@ -55,6 +59,14 @@ private RandomGenericData() {}
     return records;
   }
 
+  public static Iterable<Record> generateFallbackRecords(Schema schema, int numRecords, long seed, long numDictRows) {
+    return generateRecords(schema, numRecords, new FallbackGenerator(seed, numDictRows));

Review comment:
       I think here we'd better to create the record lazily, for `Iterable<Record>` result, (similar to spark [RandomData](https://github.com/apache/iceberg/blob/edb8d7759d74418daa677a8c39420f29e28edb66/spark/src/test/java/org/apache/iceberg/spark/data/RandomData.java#L68)),  because if we wanna to generate lots of records, it will be not easy to OOM . 

##########
File path: data/src/test/java/org/apache/iceberg/data/RandomGenericData.java
##########
@@ -46,7 +46,11 @@
   private RandomGenericData() {}
 
   public static List<Record> generate(Schema schema, int numRecords, long seed) {
-    RandomRecordGenerator generator = new RandomRecordGenerator(seed);
+    return generateRecords(schema, numRecords, new RandomRecordGenerator(seed));
+  }
+
+  public static List<Record> generateRecords(Schema schema, int numRecords,

Review comment:
       We usually don't expose this method to public because `RandomRecordGenerator` is a private static class and others could not access this method actually.   

##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java
##########
@@ -19,64 +19,701 @@
 
 package org.apache.iceberg.flink.data;
 
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.time.Instant;
+import java.time.ZoneOffset;
 import java.util.List;
-import org.apache.flink.types.Row;
+import java.util.Map;
+import org.apache.commons.lang3.ArrayUtils;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.MapData;
+import org.apache.flink.table.data.RawValueData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
 import org.apache.iceberg.Schema;
-import org.apache.iceberg.data.parquet.BaseParquetReaders;
 import org.apache.iceberg.parquet.ParquetValueReader;
 import org.apache.iceberg.parquet.ParquetValueReaders;
+import org.apache.iceberg.parquet.TypeWithSchemaVisitor;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
 import org.apache.iceberg.types.Types;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.io.api.Binary;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.LogicalTypeAnnotation.DecimalLogicalTypeAnnotation;
 import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
 import org.apache.parquet.schema.Type;
 
-public class FlinkParquetReaders extends BaseParquetReaders<Row> {
+class FlinkParquetReaders {
+  private FlinkParquetReaders() {
+  }
 
-  private static final FlinkParquetReaders INSTANCE = new FlinkParquetReaders();
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema, MessageType fileSchema) {
+    return buildReader(expectedSchema, fileSchema, ImmutableMap.of());
+  }
 
-  private FlinkParquetReaders() {
+  @SuppressWarnings("unchecked")
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema,
+                                                        MessageType fileSchema,
+                                                        Map<Integer, ?> idToConstant) {
+    return (ParquetValueReader<RowData>) TypeWithSchemaVisitor.visit(expectedSchema.asStruct(), fileSchema,
+        new ReadBuilder(fileSchema, idToConstant)
+    );
+  }
+
+  private static class ReadBuilder extends TypeWithSchemaVisitor<ParquetValueReader<?>> {
+    private final MessageType type;
+    private final Map<Integer, ?> idToConstant;
+
+    ReadBuilder(MessageType type, Map<Integer, ?> idToConstant) {
+      this.type = type;
+      this.idToConstant = idToConstant;
+    }
+
+    @Override
+    public ParquetValueReader<RowData> message(Types.StructType expected, MessageType message,
+                                               List<ParquetValueReader<?>> fieldReaders) {
+      return struct(expected, message.asGroupType(), fieldReaders);
+    }
+
+    @Override
+    public ParquetValueReader<RowData> struct(Types.StructType expected, GroupType struct,
+                                              List<ParquetValueReader<?>> fieldReaders) {
+      // match the expected struct's order
+      Map<Integer, ParquetValueReader<?>> readersById = Maps.newHashMap();
+      Map<Integer, Type> typesById = Maps.newHashMap();
+      List<Type> fields = struct.getFields();
+      for (int i = 0; i < fields.size(); i += 1) {
+        Type fieldType = fields.get(i);
+        int fieldD = type.getMaxDefinitionLevel(path(fieldType.getName())) - 1;
+        if (fieldType.getId() != null) {
+          int id = fieldType.getId().intValue();
+          readersById.put(id, ParquetValueReaders.option(fieldType, fieldD, fieldReaders.get(i)));
+          typesById.put(id, fieldType);
+        }
+      }
+
+      List<Types.NestedField> expectedFields = expected != null ?
+          expected.fields() : ImmutableList.of();
+      List<ParquetValueReader<?>> reorderedFields = Lists.newArrayListWithExpectedSize(
+          expectedFields.size());
+      List<Type> types = Lists.newArrayListWithExpectedSize(expectedFields.size());
+      for (Types.NestedField field : expectedFields) {
+        int id = field.fieldId();
+        if (idToConstant.containsKey(id)) {
+          // containsKey is used because the constant may be null
+          reorderedFields.add(ParquetValueReaders.constant(idToConstant.get(id)));
+          types.add(null);
+        } else {
+          ParquetValueReader<?> reader = readersById.get(id);
+          if (reader != null) {
+            reorderedFields.add(reader);
+            types.add(typesById.get(id));
+          } else {
+            reorderedFields.add(ParquetValueReaders.nulls());
+            types.add(null);
+          }
+        }
+      }
+
+      return new RowDataReader(types, reorderedFields);
+    }
+
+    @Override
+    public ParquetValueReader<?> list(Types.ListType expectedList, GroupType array,
+                                      ParquetValueReader<?> elementReader) {
+      GroupType repeated = array.getFields().get(0).asGroupType();
+      String[] repeatedPath = currentPath();
+
+      int repeatedD = type.getMaxDefinitionLevel(repeatedPath) - 1;
+      int repeatedR = type.getMaxRepetitionLevel(repeatedPath) - 1;
+
+      Type elementType = repeated.getType(0);
+      int elementD = type.getMaxDefinitionLevel(path(elementType.getName())) - 1;
+
+      return new ArrayReader<>(repeatedD, repeatedR, ParquetValueReaders.option(elementType, elementD, elementReader));
+    }
+
+    @Override
+    public ParquetValueReader<?> map(Types.MapType expectedMap, GroupType map,
+                                     ParquetValueReader<?> keyReader,
+                                     ParquetValueReader<?> valueReader) {
+      GroupType repeatedKeyValue = map.getFields().get(0).asGroupType();
+      String[] repeatedPath = currentPath();
+
+      int repeatedD = type.getMaxDefinitionLevel(repeatedPath) - 1;
+      int repeatedR = type.getMaxRepetitionLevel(repeatedPath) - 1;
+
+      Type keyType = repeatedKeyValue.getType(0);
+      int keyD = type.getMaxDefinitionLevel(path(keyType.getName())) - 1;
+      Type valueType = repeatedKeyValue.getType(1);
+      int valueD = type.getMaxDefinitionLevel(path(valueType.getName())) - 1;
+
+      return new MapReader<>(repeatedD, repeatedR,
+          ParquetValueReaders.option(keyType, keyD, keyReader),
+          ParquetValueReaders.option(valueType, valueD, valueReader));
+    }
+
+    @Override
+    public ParquetValueReader<?> primitive(org.apache.iceberg.types.Type.PrimitiveType expected,
+                                           PrimitiveType primitive) {
+      ColumnDescriptor desc = type.getColumnDescription(currentPath());
+
+      if (primitive.getOriginalType() != null) {
+        switch (primitive.getOriginalType()) {
+          case ENUM:
+          case JSON:
+          case UTF8:
+            return new StringReader(desc);
+          case INT_8:
+          case INT_16:
+          case INT_32:
+          case DATE:

Review comment:
       The `DATE` type should always be an integer ?   For me,  seems more reasonable to move the `DATE` case to the place where `INT_64` is, because they are surely to use `UnboxedReader` . 

##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java
##########
@@ -19,64 +19,701 @@
 
 package org.apache.iceberg.flink.data;
 
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.time.Instant;
+import java.time.ZoneOffset;
 import java.util.List;
-import org.apache.flink.types.Row;
+import java.util.Map;
+import org.apache.commons.lang3.ArrayUtils;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.MapData;
+import org.apache.flink.table.data.RawValueData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
 import org.apache.iceberg.Schema;
-import org.apache.iceberg.data.parquet.BaseParquetReaders;
 import org.apache.iceberg.parquet.ParquetValueReader;
 import org.apache.iceberg.parquet.ParquetValueReaders;
+import org.apache.iceberg.parquet.TypeWithSchemaVisitor;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
 import org.apache.iceberg.types.Types;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.io.api.Binary;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.LogicalTypeAnnotation.DecimalLogicalTypeAnnotation;
 import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
 import org.apache.parquet.schema.Type;
 
-public class FlinkParquetReaders extends BaseParquetReaders<Row> {
+class FlinkParquetReaders {
+  private FlinkParquetReaders() {
+  }
 
-  private static final FlinkParquetReaders INSTANCE = new FlinkParquetReaders();
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema, MessageType fileSchema) {
+    return buildReader(expectedSchema, fileSchema, ImmutableMap.of());
+  }
 
-  private FlinkParquetReaders() {
+  @SuppressWarnings("unchecked")
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema,
+                                                        MessageType fileSchema,
+                                                        Map<Integer, ?> idToConstant) {
+    return (ParquetValueReader<RowData>) TypeWithSchemaVisitor.visit(expectedSchema.asStruct(), fileSchema,
+        new ReadBuilder(fileSchema, idToConstant)
+    );
+  }
+
+  private static class ReadBuilder extends TypeWithSchemaVisitor<ParquetValueReader<?>> {
+    private final MessageType type;
+    private final Map<Integer, ?> idToConstant;
+
+    ReadBuilder(MessageType type, Map<Integer, ?> idToConstant) {
+      this.type = type;
+      this.idToConstant = idToConstant;
+    }
+
+    @Override
+    public ParquetValueReader<RowData> message(Types.StructType expected, MessageType message,
+                                               List<ParquetValueReader<?>> fieldReaders) {
+      return struct(expected, message.asGroupType(), fieldReaders);
+    }
+
+    @Override
+    public ParquetValueReader<RowData> struct(Types.StructType expected, GroupType struct,
+                                              List<ParquetValueReader<?>> fieldReaders) {
+      // match the expected struct's order
+      Map<Integer, ParquetValueReader<?>> readersById = Maps.newHashMap();
+      Map<Integer, Type> typesById = Maps.newHashMap();
+      List<Type> fields = struct.getFields();
+      for (int i = 0; i < fields.size(); i += 1) {
+        Type fieldType = fields.get(i);
+        int fieldD = type.getMaxDefinitionLevel(path(fieldType.getName())) - 1;
+        if (fieldType.getId() != null) {
+          int id = fieldType.getId().intValue();
+          readersById.put(id, ParquetValueReaders.option(fieldType, fieldD, fieldReaders.get(i)));
+          typesById.put(id, fieldType);
+        }
+      }
+
+      List<Types.NestedField> expectedFields = expected != null ?
+          expected.fields() : ImmutableList.of();
+      List<ParquetValueReader<?>> reorderedFields = Lists.newArrayListWithExpectedSize(
+          expectedFields.size());
+      List<Type> types = Lists.newArrayListWithExpectedSize(expectedFields.size());
+      for (Types.NestedField field : expectedFields) {
+        int id = field.fieldId();
+        if (idToConstant.containsKey(id)) {
+          // containsKey is used because the constant may be null
+          reorderedFields.add(ParquetValueReaders.constant(idToConstant.get(id)));
+          types.add(null);
+        } else {
+          ParquetValueReader<?> reader = readersById.get(id);
+          if (reader != null) {
+            reorderedFields.add(reader);
+            types.add(typesById.get(id));
+          } else {
+            reorderedFields.add(ParquetValueReaders.nulls());
+            types.add(null);
+          }
+        }
+      }
+
+      return new RowDataReader(types, reorderedFields);
+    }
+
+    @Override
+    public ParquetValueReader<?> list(Types.ListType expectedList, GroupType array,
+                                      ParquetValueReader<?> elementReader) {
+      GroupType repeated = array.getFields().get(0).asGroupType();
+      String[] repeatedPath = currentPath();
+
+      int repeatedD = type.getMaxDefinitionLevel(repeatedPath) - 1;
+      int repeatedR = type.getMaxRepetitionLevel(repeatedPath) - 1;
+
+      Type elementType = repeated.getType(0);
+      int elementD = type.getMaxDefinitionLevel(path(elementType.getName())) - 1;
+
+      return new ArrayReader<>(repeatedD, repeatedR, ParquetValueReaders.option(elementType, elementD, elementReader));
+    }
+
+    @Override
+    public ParquetValueReader<?> map(Types.MapType expectedMap, GroupType map,
+                                     ParquetValueReader<?> keyReader,
+                                     ParquetValueReader<?> valueReader) {
+      GroupType repeatedKeyValue = map.getFields().get(0).asGroupType();
+      String[] repeatedPath = currentPath();
+
+      int repeatedD = type.getMaxDefinitionLevel(repeatedPath) - 1;
+      int repeatedR = type.getMaxRepetitionLevel(repeatedPath) - 1;
+
+      Type keyType = repeatedKeyValue.getType(0);
+      int keyD = type.getMaxDefinitionLevel(path(keyType.getName())) - 1;
+      Type valueType = repeatedKeyValue.getType(1);
+      int valueD = type.getMaxDefinitionLevel(path(valueType.getName())) - 1;
+
+      return new MapReader<>(repeatedD, repeatedR,
+          ParquetValueReaders.option(keyType, keyD, keyReader),
+          ParquetValueReaders.option(valueType, valueD, valueReader));
+    }
+
+    @Override
+    public ParquetValueReader<?> primitive(org.apache.iceberg.types.Type.PrimitiveType expected,
+                                           PrimitiveType primitive) {
+      ColumnDescriptor desc = type.getColumnDescription(currentPath());
+
+      if (primitive.getOriginalType() != null) {
+        switch (primitive.getOriginalType()) {
+          case ENUM:
+          case JSON:
+          case UTF8:
+            return new StringReader(desc);
+          case INT_8:
+          case INT_16:
+          case INT_32:
+          case DATE:
+            if (expected != null && expected.typeId() == Types.LongType.get().typeId()) {
+              return new ParquetValueReaders.IntAsLongReader(desc);
+            } else {
+              return new ParquetValueReaders.UnboxedReader<>(desc);
+            }
+          case TIME_MICROS:
+            return new LossyMicrosToMillisTimeReader(desc);
+          case INT_64:
+            return new ParquetValueReaders.UnboxedReader<>(desc);
+          case TIMESTAMP_MICROS:
+            if (((Types.TimestampType) expected).shouldAdjustToUTC()) {
+              return new TimestampTzReader(desc);
+            } else {
+              return new TimestampReader(desc);
+            }
+          case DECIMAL:
+            DecimalLogicalTypeAnnotation decimal = (DecimalLogicalTypeAnnotation) primitive.getLogicalTypeAnnotation();
+            switch (primitive.getPrimitiveTypeName()) {
+              case BINARY:
+              case FIXED_LEN_BYTE_ARRAY:
+                return new BinaryDecimalReader(desc, decimal.getScale());
+              case INT64:
+                return new LongDecimalReader(desc, decimal.getPrecision(), decimal.getScale());
+              case INT32:
+                return new IntegerDecimalReader(desc, decimal.getPrecision(), decimal.getScale());
+              default:
+                throw new UnsupportedOperationException(
+                    "Unsupported base type for decimal: " + primitive.getPrimitiveTypeName());
+            }
+          case BSON:
+            return new ParquetValueReaders.ByteArrayReader(desc);
+          default:
+            throw new UnsupportedOperationException(
+                "Unsupported logical type: " + primitive.getOriginalType());
+        }
+      }
+
+      switch (primitive.getPrimitiveTypeName()) {
+        case FIXED_LEN_BYTE_ARRAY:
+        case BINARY:
+          return new ParquetValueReaders.ByteArrayReader(desc);
+        case INT32:
+          if (expected != null && expected.typeId() == org.apache.iceberg.types.Type.TypeID.LONG) {
+            return new ParquetValueReaders.IntAsLongReader(desc);
+          } else {
+            return new ParquetValueReaders.UnboxedReader<>(desc);
+          }
+        case FLOAT:
+          if (expected != null && expected.typeId() == org.apache.iceberg.types.Type.TypeID.DOUBLE) {
+            return new ParquetValueReaders.FloatAsDoubleReader(desc);
+          } else {
+            return new ParquetValueReaders.UnboxedReader<>(desc);
+          }
+        case BOOLEAN:
+        case INT64:
+        case DOUBLE:
+          return new ParquetValueReaders.UnboxedReader<>(desc);
+        default:
+          throw new UnsupportedOperationException("Unsupported type: " + primitive);
+      }
+    }
+  }
+
+  private static class BinaryDecimalReader extends ParquetValueReaders.PrimitiveReader<DecimalData> {
+    private final int scale;
+
+    BinaryDecimalReader(ColumnDescriptor desc, int scale) {
+      super(desc);
+      this.scale = scale;
+    }
+
+    @Override
+    public DecimalData read(DecimalData ignored) {
+      Binary binary = column.nextBinary();
+      BigDecimal bigDecimal = new BigDecimal(new BigInteger(binary.getBytes()), scale);
+      return DecimalData.fromBigDecimal(bigDecimal, bigDecimal.precision(), scale);
+    }
+  }
+
+  private static class IntegerDecimalReader extends ParquetValueReaders.PrimitiveReader<DecimalData> {
+    private final int precision;
+    private final int scale;
+
+    IntegerDecimalReader(ColumnDescriptor desc, int precision, int scale) {
+      super(desc);
+      this.precision = precision;
+      this.scale = scale;
+    }
+
+    @Override
+    public DecimalData read(DecimalData ignored) {
+      return DecimalData.fromUnscaledLong(column.nextInteger(), precision, scale);
+    }
+  }
+
+  private static class LongDecimalReader extends ParquetValueReaders.PrimitiveReader<DecimalData> {
+    private final int precision;
+    private final int scale;
+
+    LongDecimalReader(ColumnDescriptor desc, int precision, int scale) {
+      super(desc);
+      this.precision = precision;
+      this.scale = scale;
+    }
+
+    @Override
+    public DecimalData read(DecimalData ignored) {
+      return DecimalData.fromUnscaledLong(column.nextLong(), precision, scale);
+    }
+  }
+
+  private static class TimestampTzReader extends ParquetValueReaders.UnboxedReader<TimestampData> {
+    TimestampTzReader(ColumnDescriptor desc) {
+      super(desc);
+    }
+
+    @Override
+    public TimestampData read(TimestampData ignored) {
+      long value = readLong();
+      return TimestampData.fromLocalDateTime(Instant.ofEpochSecond(Math.floorDiv(value, 1000_000),
+          Math.floorMod(value, 1000_000) * 1000)
+          .atOffset(ZoneOffset.UTC)
+          .toLocalDateTime());
+    }
+
+    @Override
+    public long readLong() {
+      return column.nextLong();
+    }
   }
 
-  public static ParquetValueReader<Row> buildReader(Schema expectedSchema, MessageType fileSchema) {
-    return INSTANCE.createReader(expectedSchema, fileSchema);
+  private static class TimestampReader extends ParquetValueReaders.UnboxedReader<TimestampData> {
+    TimestampReader(ColumnDescriptor desc) {
+      super(desc);
+    }
+
+    @Override
+    public TimestampData read(TimestampData ignored) {
+      long value = readLong();
+      return TimestampData.fromInstant(Instant.ofEpochSecond(Math.floorDiv(value, 1000_000),
+          Math.floorMod(value, 1000_000) * 1000));
+    }
+
+    @Override
+    public long readLong() {
+      return column.nextLong();
+    }
   }
 
-  @Override
-  protected ParquetValueReader<Row> createStructReader(List<Type> types,
-                                                       List<ParquetValueReader<?>> fieldReaders,
-                                                       Types.StructType structType) {
-    return new RowReader(types, fieldReaders, structType);
+  private static class StringReader extends ParquetValueReaders.PrimitiveReader<StringData> {
+    StringReader(ColumnDescriptor desc) {
+      super(desc);
+    }
+
+    @Override
+    public StringData read(StringData ignored) {
+      Binary binary = column.nextBinary();
+      ByteBuffer buffer = binary.toByteBuffer();
+      if (buffer.hasArray()) {
+        return StringData.fromBytes(
+            buffer.array(), buffer.arrayOffset() + buffer.position(), buffer.remaining());
+      } else {
+        return StringData.fromBytes(binary.getBytes());
+      }
+    }
+  }
+
+  private static class LossyMicrosToMillisTimeReader extends ParquetValueReaders.PrimitiveReader<Integer> {
+    LossyMicrosToMillisTimeReader(ColumnDescriptor desc) {
+      super(desc);
+    }
+
+    @Override
+    public Integer read(Integer reuse) {
+      // Discard microseconds since Flink uses millisecond unit for TIME type.
+      return (int) Math.floorDiv(column.nextLong(), 1000);
+    }
+  }
+
+  private static class ArrayReader<E> extends ParquetValueReaders.RepeatedReader<ArrayData, ReusableArrayData, E> {
+    private int readPos = 0;
+    private int writePos = 0;
+
+    ArrayReader(int definitionLevel, int repetitionLevel, ParquetValueReader<E> reader) {
+      super(definitionLevel, repetitionLevel, reader);
+    }
+
+    @Override
+    protected ReusableArrayData newListData(ArrayData reuse) {
+      this.readPos = 0;
+      this.writePos = 0;
+
+      if (reuse instanceof ReusableArrayData) {
+        return (ReusableArrayData) reuse;
+      } else {
+        return new ReusableArrayData();
+      }
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    protected E getElement(ReusableArrayData list) {
+      E value = null;
+      if (readPos < list.capacity()) {
+        value = (E) list.values[readPos];
+      }
+
+      readPos += 1;
+
+      return value;
+    }
+
+    @Override
+    protected void addElement(ReusableArrayData reused, E element) {
+      if (writePos >= reused.capacity()) {
+        reused.grow();
+      }
+
+      reused.values[writePos] = element;
+
+      writePos += 1;
+    }
+
+    @Override
+    protected ArrayData buildList(ReusableArrayData list) {
+      list.setNumElements(writePos);
+      return list;
+    }
   }
 
-  private static class RowReader extends ParquetValueReaders.StructReader<Row, Row> {
-    private final Types.StructType structType;
+  private static class MapReader<K, V> extends
+      ParquetValueReaders.RepeatedKeyValueReader<MapData, ReusableMapData, K, V> {
+    private int readPos = 0;
+    private int writePos = 0;
+
+    private final ParquetValueReaders.ReusableEntry<K, V> entry = new ParquetValueReaders.ReusableEntry<>();
+    private final ParquetValueReaders.ReusableEntry<K, V> nullEntry = new ParquetValueReaders.ReusableEntry<>();
+
+    MapReader(int definitionLevel, int repetitionLevel,
+              ParquetValueReader<K> keyReader, ParquetValueReader<V> valueReader) {
+      super(definitionLevel, repetitionLevel, keyReader, valueReader);
+    }
+
+    @Override
+    protected ReusableMapData newMapData(MapData reuse) {
+      this.readPos = 0;
+      this.writePos = 0;
+
+      if (reuse instanceof ReusableMapData) {
+        return (ReusableMapData) reuse;
+      } else {
+        return new ReusableMapData();
+      }
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    protected Map.Entry<K, V> getPair(ReusableMapData map) {
+      Map.Entry<K, V> kv = nullEntry;
+      if (readPos < map.capacity()) {
+        entry.set((K) map.keys.values[readPos], (V) map.values.values[readPos]);
+        kv = entry;
+      }
+
+      readPos += 1;

Review comment:
       Same question .

##########
File path: flink/src/test/java/org/apache/iceberg/flink/data/TestHelpers.java
##########
@@ -0,0 +1,269 @@
+/*
+ * 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.iceberg.flink.data;
+
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.OffsetDateTime;
+import java.time.ZoneOffset;
+import java.time.temporal.ChronoUnit;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.RowData;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+
+public class TestHelpers {
+  private TestHelpers() {}
+
+  private static final OffsetDateTime EPOCH = Instant.ofEpochMilli(0L).atOffset(ZoneOffset.UTC);
+  private static final LocalDate EPOCH_DAY = EPOCH.toLocalDate();
+
+  public static void assertRowData(Type type, Record expectedRecord, RowData actualRowData) {
+    if (expectedRecord == null && actualRowData == null) {
+      return;
+    }
+
+    Assert.assertTrue("expected Record and actual RowData should be both null or not null",
+        expectedRecord != null && actualRowData != null);
+
+    List<Type> types = new ArrayList<>();

Review comment:
       nit:  Lists.newArrayList.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1266: Flink: update parquet reader with schema visitor

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1266:
URL: https://github.com/apache/iceberg/pull/1266#discussion_r463887115



##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java
##########
@@ -19,64 +19,719 @@
 
 package org.apache.iceberg.flink.data;
 
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.time.Instant;
 import java.util.List;
-import org.apache.flink.types.Row;
+import java.util.Map;
+import org.apache.commons.lang3.ArrayUtils;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.MapData;
+import org.apache.flink.table.data.RawValueData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
 import org.apache.iceberg.Schema;
-import org.apache.iceberg.data.parquet.BaseParquetReaders;
+import org.apache.iceberg.parquet.ParquetSchemaUtil;
 import org.apache.iceberg.parquet.ParquetValueReader;
 import org.apache.iceberg.parquet.ParquetValueReaders;
+import org.apache.iceberg.parquet.TypeWithSchemaVisitor;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
 import org.apache.iceberg.types.Types;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.io.api.Binary;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.LogicalTypeAnnotation.DecimalLogicalTypeAnnotation;
 import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
 import org.apache.parquet.schema.Type;
 
-public class FlinkParquetReaders extends BaseParquetReaders<Row> {
+class FlinkParquetReaders {
+  private FlinkParquetReaders() {
+  }
 
-  private static final FlinkParquetReaders INSTANCE = new FlinkParquetReaders();
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema, MessageType fileSchema) {
+    return buildReader(expectedSchema, fileSchema, ImmutableMap.of());
+  }
 
-  private FlinkParquetReaders() {
+  @SuppressWarnings("unchecked")
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema,
+                                                        MessageType fileSchema,
+                                                        Map<Integer, ?> idToConstant) {
+    ReadBuilder builder = new ReadBuilder(fileSchema, idToConstant);
+    if (ParquetSchemaUtil.hasIds(fileSchema)) {
+      return (ParquetValueReader<RowData>)
+          TypeWithSchemaVisitor.visit(expectedSchema.asStruct(), fileSchema, builder);
+    } else {
+      return (ParquetValueReader<RowData>)
+          TypeWithSchemaVisitor.visit(expectedSchema.asStruct(), fileSchema,
+              new FallbackReadBuilder(builder));
+    }
+  }
+
+  private static class FallbackReadBuilder extends TypeWithSchemaVisitor<ParquetValueReader<?>> {
+    private MessageType type;
+    private final TypeWithSchemaVisitor<ParquetValueReader<?>> builder;

Review comment:
       I think this refactor should be done in a separate commit. Let's not over-complicated this one. We can judge the value of the change better when it is isolated to its own PR.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] chenjunjiedada commented on a change in pull request #1266: Flink: update parquet reader with schema visitor

Posted by GitBox <gi...@apache.org>.
chenjunjiedada commented on a change in pull request #1266:
URL: https://github.com/apache/iceberg/pull/1266#discussion_r469690153



##########
File path: flink/src/test/java/org/apache/iceberg/flink/data/TestHelpers.java
##########
@@ -0,0 +1,198 @@
+/*
+ * 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.iceberg.flink.data;
+
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.OffsetDateTime;
+import java.time.ZoneOffset;
+import java.time.temporal.ChronoUnit;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import java.util.function.Supplier;
+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.MapType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+
+public class TestHelpers {
+  private TestHelpers() {}
+
+  private static final OffsetDateTime EPOCH = Instant.ofEpochMilli(0L).atOffset(ZoneOffset.UTC);
+  private static final LocalDate EPOCH_DAY = EPOCH.toLocalDate();
+
+  public static void assertRowData(Type type, LogicalType rowType, Record expectedRecord, RowData actualRowData) {

Review comment:
       Done.

##########
File path: flink/src/test/java/org/apache/iceberg/flink/data/TestHelpers.java
##########
@@ -0,0 +1,198 @@
+/*
+ * 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.iceberg.flink.data;
+
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.OffsetDateTime;
+import java.time.ZoneOffset;
+import java.time.temporal.ChronoUnit;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import java.util.function.Supplier;
+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.MapType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+
+public class TestHelpers {
+  private TestHelpers() {}
+
+  private static final OffsetDateTime EPOCH = Instant.ofEpochMilli(0L).atOffset(ZoneOffset.UTC);
+  private static final LocalDate EPOCH_DAY = EPOCH.toLocalDate();
+
+  public static void assertRowData(Type type, LogicalType rowType, Record expectedRecord, RowData actualRowData) {
+    if (expectedRecord == null && actualRowData == null) {
+      return;
+    }
+
+    Assert.assertTrue("expected Record and actual RowData should be both null or not null",
+        expectedRecord != null && actualRowData != null);
+
+    List<Type> types = Lists.newArrayList();
+    for (Types.NestedField field : type.asStructType().fields()) {
+      types.add(field.type());
+    }
+
+    for (int i = 0; i < types.size(); i += 1) {
+      if (expectedRecord.get(i) == null) {
+        Assert.assertTrue(actualRowData.isNullAt(i));
+        continue;
+      }
+
+      Object expected = expectedRecord.get(i);
+      LogicalType logicalType = ((RowType) rowType).getTypeAt(i);
+
+      final int fieldPos = i;
+      assertEquals(types.get(i), logicalType, expected,
+          () -> RowData.createFieldGetter(logicalType, fieldPos).getFieldOrNull(actualRowData));

Review comment:
       Done.

##########
File path: flink/src/test/java/org/apache/iceberg/flink/data/TestHelpers.java
##########
@@ -0,0 +1,198 @@
+/*
+ * 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.iceberg.flink.data;
+
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.OffsetDateTime;
+import java.time.ZoneOffset;
+import java.time.temporal.ChronoUnit;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import java.util.function.Supplier;
+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.MapType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+
+public class TestHelpers {
+  private TestHelpers() {}
+
+  private static final OffsetDateTime EPOCH = Instant.ofEpochMilli(0L).atOffset(ZoneOffset.UTC);
+  private static final LocalDate EPOCH_DAY = EPOCH.toLocalDate();
+
+  public static void assertRowData(Type type, LogicalType rowType, Record expectedRecord, RowData actualRowData) {
+    if (expectedRecord == null && actualRowData == null) {
+      return;
+    }
+
+    Assert.assertTrue("expected Record and actual RowData should be both null or not null",
+        expectedRecord != null && actualRowData != null);
+
+    List<Type> types = Lists.newArrayList();
+    for (Types.NestedField field : type.asStructType().fields()) {
+      types.add(field.type());
+    }
+
+    for (int i = 0; i < types.size(); i += 1) {
+      if (expectedRecord.get(i) == null) {
+        Assert.assertTrue(actualRowData.isNullAt(i));
+        continue;
+      }
+
+      Object expected = expectedRecord.get(i);
+      LogicalType logicalType = ((RowType) rowType).getTypeAt(i);
+
+      final int fieldPos = i;
+      assertEquals(types.get(i), logicalType, expected,
+          () -> RowData.createFieldGetter(logicalType, fieldPos).getFieldOrNull(actualRowData));
+    }
+  }
+
+  private static void assertEquals(Type type, LogicalType logicalType, Object expected, Supplier<Object> supplier) {

Review comment:
       Fixed.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] chenjunjiedada commented on a change in pull request #1266: Flink: update parquet reader with schema visitor

Posted by GitBox <gi...@apache.org>.
chenjunjiedada commented on a change in pull request #1266:
URL: https://github.com/apache/iceberg/pull/1266#discussion_r464997844



##########
File path: flink/src/test/java/org/apache/iceberg/flink/data/TestHelpers.java
##########
@@ -0,0 +1,271 @@
+/*
+ * 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.iceberg.flink.data;
+
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.OffsetDateTime;
+import java.time.ZoneOffset;
+import java.time.temporal.ChronoUnit;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.RowData;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+
+public class TestHelpers {
+  private TestHelpers() {}
+
+  private static final OffsetDateTime EPOCH = Instant.ofEpochMilli(0L).atOffset(ZoneOffset.UTC);
+  private static final LocalDate EPOCH_DAY = EPOCH.toLocalDate();
+
+  public static void assertRowData(Type type, Record expected, RowData actual) {
+    if (expected == null && actual == null) {
+      return;
+    }
+
+    List<Type> types = new ArrayList<>();
+    for (Types.NestedField field : type.asStructType().fields()) {
+      types.add(field.type());
+    }
+
+    for (int i = 0; i < types.size(); i += 1) {
+      if (expected.get(i) == null) {
+        Assert.assertTrue(actual.isNullAt(i));
+        continue;
+      }
+      Type.TypeID typeId = types.get(i).typeId();
+      Object value = expected.get(i);
+      switch (typeId) {
+        case BOOLEAN:
+          Assert.assertEquals("boolean value should be equal", value, actual.getBoolean(i));
+          break;
+        case INTEGER:
+          Assert.assertEquals("int value should be equal", value, actual.getInt(i));
+          break;
+        case LONG:
+          Assert.assertEquals("long value should be equal", value, actual.getLong(i));
+          break;
+        case FLOAT:
+          Assert.assertEquals("float value should be equal", value, actual.getFloat(i));
+          break;
+        case DOUBLE:
+          Assert.assertEquals("double should be equal", value, actual.getDouble(i));
+          break;
+        case STRING:
+          Assert.assertTrue("Should expect a CharSequence", value instanceof CharSequence);
+          Assert.assertEquals("string should be equal", String.valueOf(value), actual.getString(i).toString());
+          break;
+        case DATE:
+          Assert.assertTrue("Should expect a Date", value instanceof LocalDate);
+          LocalDate date = ChronoUnit.DAYS.addTo(EPOCH_DAY, actual.getInt(i));
+          Assert.assertEquals("date should be equal", value, date);
+          break;
+        case TIME:
+          Assert.assertTrue("Should expect a LocalTime", value instanceof LocalTime);
+          int milliseconds = (int) (((LocalTime) value).toNanoOfDay() / 1000_000);
+          Assert.assertEquals("time millis should be equal", milliseconds, actual.getInt(i));
+          break;
+        case TIMESTAMP:
+          if (((Types.TimestampType) type.asPrimitiveType()).shouldAdjustToUTC()) {
+            Assert.assertTrue("Should expect a OffsetDataTime", value instanceof OffsetDateTime);
+            OffsetDateTime ts = (OffsetDateTime) value;
+            Assert.assertEquals("OffsetDataTime should be equal", ts.toLocalDateTime(),
+                actual.getTimestamp(i, 6).toLocalDateTime());
+          } else {
+            Assert.assertTrue("Should expect a LocalDataTime", value instanceof LocalDateTime);
+            LocalDateTime ts = (LocalDateTime) value;
+            Assert.assertEquals("LocalDataTime should be equal", ts,
+                actual.getTimestamp(i, 6).toLocalDateTime());
+          }
+          break;
+        case FIXED:
+          Assert.assertTrue("Should expect byte[]", value instanceof byte[]);
+          Assert.assertArrayEquals("binary should be equal", (byte[]) value, actual.getBinary(i));
+          break;
+        case BINARY:
+          Assert.assertTrue("Should expect a ByteBuffer", value instanceof ByteBuffer);
+          Assert.assertArrayEquals("binary should be equal", ((ByteBuffer) value).array(), actual.getBinary(i));
+          break;
+        case DECIMAL:

Review comment:
       I tried to extract to one common method with an interface like `assertData(Type type, Record expected, Object actual` so that we can determine them via `instanceof`, but in the function, you have to call the getter with ((RowData)actual).getXXX and ((ArrayData)actual).getXXX.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] chenjunjiedada commented on a change in pull request #1266: Flink: update parquet reader with schema visitor

Posted by GitBox <gi...@apache.org>.
chenjunjiedada commented on a change in pull request #1266:
URL: https://github.com/apache/iceberg/pull/1266#discussion_r466906927



##########
File path: data/src/test/java/org/apache/iceberg/data/RandomGenericData.java
##########
@@ -46,7 +46,11 @@
   private RandomGenericData() {}
 
   public static List<Record> generate(Schema schema, int numRecords, long seed) {
-    RandomRecordGenerator generator = new RandomRecordGenerator(seed);
+    return generateRecords(schema, numRecords, new RandomRecordGenerator(seed));
+  }
+
+  public static List<Record> generateRecords(Schema schema, int numRecords,

Review comment:
       Fixed.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] chenjunjiedada commented on a change in pull request #1266: Flink: update parquet reader with schema visitor

Posted by GitBox <gi...@apache.org>.
chenjunjiedada commented on a change in pull request #1266:
URL: https://github.com/apache/iceberg/pull/1266#discussion_r463628416



##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java
##########
@@ -19,64 +19,719 @@
 
 package org.apache.iceberg.flink.data;
 
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.time.Instant;
 import java.util.List;
-import org.apache.flink.types.Row;
+import java.util.Map;
+import org.apache.commons.lang3.ArrayUtils;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.MapData;
+import org.apache.flink.table.data.RawValueData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
 import org.apache.iceberg.Schema;
-import org.apache.iceberg.data.parquet.BaseParquetReaders;
+import org.apache.iceberg.parquet.ParquetSchemaUtil;
 import org.apache.iceberg.parquet.ParquetValueReader;
 import org.apache.iceberg.parquet.ParquetValueReaders;
+import org.apache.iceberg.parquet.TypeWithSchemaVisitor;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
 import org.apache.iceberg.types.Types;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.io.api.Binary;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.LogicalTypeAnnotation.DecimalLogicalTypeAnnotation;
 import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
 import org.apache.parquet.schema.Type;
 
-public class FlinkParquetReaders extends BaseParquetReaders<Row> {
+class FlinkParquetReaders {
+  private FlinkParquetReaders() {
+  }
 
-  private static final FlinkParquetReaders INSTANCE = new FlinkParquetReaders();
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema, MessageType fileSchema) {
+    return buildReader(expectedSchema, fileSchema, ImmutableMap.of());
+  }
 
-  private FlinkParquetReaders() {
+  @SuppressWarnings("unchecked")
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema,
+                                                        MessageType fileSchema,
+                                                        Map<Integer, ?> idToConstant) {
+    ReadBuilder builder = new ReadBuilder(fileSchema, idToConstant);
+    if (ParquetSchemaUtil.hasIds(fileSchema)) {
+      return (ParquetValueReader<RowData>)
+          TypeWithSchemaVisitor.visit(expectedSchema.asStruct(), fileSchema, builder);
+    } else {
+      return (ParquetValueReader<RowData>)
+          TypeWithSchemaVisitor.visit(expectedSchema.asStruct(), fileSchema,
+              new FallbackReadBuilder(builder));
+    }
+  }
+
+  private static class FallbackReadBuilder extends TypeWithSchemaVisitor<ParquetValueReader<?>> {
+    private MessageType type;
+    private final TypeWithSchemaVisitor<ParquetValueReader<?>> builder;
+
+    FallbackReadBuilder(TypeWithSchemaVisitor<ParquetValueReader<?>> builder) {
+      this.builder = builder;
+    }
+
+    @Override
+    public ParquetValueReader<?> message(Types.StructType expected, MessageType message,
+                                         List<ParquetValueReader<?>> fieldReaders) {
+      // the top level matches by ID, but the remaining IDs are missing
+      this.type = message;
+      return builder.struct(expected, message, fieldReaders);
+    }
+
+    @Override
+    public ParquetValueReader<?> struct(Types.StructType ignored, GroupType struct,
+                                        List<ParquetValueReader<?>> fieldReaders) {
+      // the expected struct is ignored because nested fields are never found when the IDs are missing
+      List<ParquetValueReader<?>> newFields = Lists.newArrayListWithExpectedSize(
+          fieldReaders.size());
+      List<Type> types = Lists.newArrayListWithExpectedSize(fieldReaders.size());
+      List<Type> fields = struct.getFields();
+      for (int i = 0; i < fields.size(); i += 1) {
+        Type fieldType = fields.get(i);
+        int fieldD = type.getMaxDefinitionLevel(path(fieldType.getName())) - 1;
+        newFields.add(ParquetValueReaders.option(fieldType, fieldD, fieldReaders.get(i)));
+        types.add(fieldType);
+      }
+
+      return new RowDataReader(types, newFields);
+    }
+  }
+
+  private static class ReadBuilder extends TypeWithSchemaVisitor<ParquetValueReader<?>> {
+    private final MessageType type;
+    private final Map<Integer, ?> idToConstant;
+
+    ReadBuilder(MessageType type, Map<Integer, ?> idToConstant) {
+      this.type = type;
+      this.idToConstant = idToConstant;
+    }
+
+    @Override
+    public ParquetValueReader<?> message(Types.StructType expected, MessageType message,
+                                         List<ParquetValueReader<?>> fieldReaders) {
+      return struct(expected, message.asGroupType(), fieldReaders);
+    }
+
+    @Override
+    public ParquetValueReader<?> struct(Types.StructType expected, GroupType struct,
+                                        List<ParquetValueReader<?>> fieldReaders) {
+      // match the expected struct's order
+      Map<Integer, ParquetValueReader<?>> readersById = Maps.newHashMap();
+      Map<Integer, Type> typesById = Maps.newHashMap();
+      List<Type> fields = struct.getFields();
+      for (int i = 0; i < fields.size(); i += 1) {
+        Type fieldType = fields.get(i);
+        int fieldD = type.getMaxDefinitionLevel(path(fieldType.getName())) - 1;
+        if (fieldType.getId() != null) {
+          int id = fieldType.getId().intValue();
+          readersById.put(id, ParquetValueReaders.option(fieldType, fieldD, fieldReaders.get(i)));
+          typesById.put(id, fieldType);
+        }
+      }
+
+      List<Types.NestedField> expectedFields = expected != null ?
+          expected.fields() : ImmutableList.of();
+      List<ParquetValueReader<?>> reorderedFields = Lists.newArrayListWithExpectedSize(
+          expectedFields.size());
+      List<Type> types = Lists.newArrayListWithExpectedSize(expectedFields.size());
+      for (Types.NestedField field : expectedFields) {
+        int id = field.fieldId();
+        if (idToConstant.containsKey(id)) {
+          // containsKey is used because the constant may be null
+          reorderedFields.add(ParquetValueReaders.constant(idToConstant.get(id)));
+          types.add(null);
+        } else {
+          ParquetValueReader<?> reader = readersById.get(id);
+          if (reader != null) {
+            reorderedFields.add(reader);
+            types.add(typesById.get(id));
+          } else {
+            reorderedFields.add(ParquetValueReaders.nulls());
+            types.add(null);
+          }
+        }
+      }
+
+      return new RowDataReader(types, reorderedFields);
+    }
+
+    @Override
+    public ParquetValueReader<?> list(Types.ListType expectedList, GroupType array,
+                                      ParquetValueReader<?> elementReader) {
+      GroupType repeated = array.getFields().get(0).asGroupType();
+      String[] repeatedPath = currentPath();
+
+      int repeatedD = type.getMaxDefinitionLevel(repeatedPath) - 1;
+      int repeatedR = type.getMaxRepetitionLevel(repeatedPath) - 1;
+
+      Type elementType = repeated.getType(0);
+      int elementD = type.getMaxDefinitionLevel(path(elementType.getName())) - 1;
+
+      return new ArrayReader<>(repeatedD, repeatedR, ParquetValueReaders.option(elementType, elementD, elementReader));
+    }
+
+    @Override
+    public ParquetValueReader<?> map(Types.MapType expectedMap, GroupType map,
+                                     ParquetValueReader<?> keyReader,
+                                     ParquetValueReader<?> valueReader) {
+      GroupType repeatedKeyValue = map.getFields().get(0).asGroupType();
+      String[] repeatedPath = currentPath();
+
+      int repeatedD = type.getMaxDefinitionLevel(repeatedPath) - 1;
+      int repeatedR = type.getMaxRepetitionLevel(repeatedPath) - 1;
+
+      Type keyType = repeatedKeyValue.getType(0);
+      int keyD = type.getMaxDefinitionLevel(path(keyType.getName())) - 1;
+      Type valueType = repeatedKeyValue.getType(1);
+      int valueD = type.getMaxDefinitionLevel(path(valueType.getName())) - 1;
+
+      return new MapReader<>(repeatedD, repeatedR,
+          ParquetValueReaders.option(keyType, keyD, keyReader),
+          ParquetValueReaders.option(valueType, valueD, valueReader));
+    }
+
+    @Override
+    public ParquetValueReader<?> primitive(org.apache.iceberg.types.Type.PrimitiveType expected,
+                                           PrimitiveType primitive) {
+      ColumnDescriptor desc = type.getColumnDescription(currentPath());
+
+      if (primitive.getOriginalType() != null) {
+        switch (primitive.getOriginalType()) {
+          case ENUM:
+          case JSON:
+          case UTF8:
+            return new StringReader(desc);
+          case INT_8:
+          case INT_16:
+          case INT_32:
+          case DATE:
+            if (expected != null && expected.typeId() == Types.LongType.get().typeId()) {
+              return new ParquetValueReaders.IntAsLongReader(desc);
+            } else {
+              return new ParquetValueReaders.UnboxedReader<>(desc);
+            }
+          case TIME_MICROS:
+            return new TimeMillisReader(desc);
+          case INT_64:
+            return new ParquetValueReaders.UnboxedReader<>(desc);
+          case TIMESTAMP_MICROS:
+            return new TimestampMicroReader(desc);
+          case DECIMAL:
+            DecimalLogicalTypeAnnotation decimal = (DecimalLogicalTypeAnnotation) primitive.getLogicalTypeAnnotation();
+            switch (primitive.getPrimitiveTypeName()) {
+              case BINARY:
+              case FIXED_LEN_BYTE_ARRAY:
+                return new BinaryDecimalReader(desc, decimal.getScale());
+              case INT64:
+                return new LongDecimalReader(desc, decimal.getPrecision(), decimal.getScale());
+              case INT32:
+                return new IntegerDecimalReader(desc, decimal.getPrecision(), decimal.getScale());
+              default:
+                throw new UnsupportedOperationException(
+                    "Unsupported base type for decimal: " + primitive.getPrimitiveTypeName());
+            }
+          case BSON:
+            return new ParquetValueReaders.ByteArrayReader(desc);
+          default:
+            throw new UnsupportedOperationException(
+                "Unsupported logical type: " + primitive.getOriginalType());
+        }
+      }
+
+      switch (primitive.getPrimitiveTypeName()) {
+        case FIXED_LEN_BYTE_ARRAY:
+        case BINARY:
+          return new ParquetValueReaders.ByteArrayReader(desc);
+        case INT32:
+          if (expected != null && expected.typeId() == org.apache.iceberg.types.Type.TypeID.LONG) {
+            return new ParquetValueReaders.IntAsLongReader(desc);
+          } else {
+            return new ParquetValueReaders.UnboxedReader<>(desc);
+          }
+        case FLOAT:
+          if (expected != null && expected.typeId() == org.apache.iceberg.types.Type.TypeID.DOUBLE) {
+            return new ParquetValueReaders.FloatAsDoubleReader(desc);
+          } else {
+            return new ParquetValueReaders.UnboxedReader<>(desc);
+          }
+        case BOOLEAN:
+        case INT64:
+        case DOUBLE:
+          return new ParquetValueReaders.UnboxedReader<>(desc);
+        default:
+          throw new UnsupportedOperationException("Unsupported type: " + primitive);
+      }
+    }
+  }
+
+  private static class BinaryDecimalReader extends ParquetValueReaders.PrimitiveReader<DecimalData> {
+    private final int scale;
+
+    BinaryDecimalReader(ColumnDescriptor desc, int scale) {
+      super(desc);
+      this.scale = scale;
+    }
+
+    @Override
+    public DecimalData read(DecimalData ignored) {
+      Binary binary = column.nextBinary();
+      BigDecimal bigDecimal = new BigDecimal(new BigInteger(binary.getBytes()), scale);
+      return DecimalData.fromBigDecimal(bigDecimal, bigDecimal.precision(), scale);
+    }
+  }
+
+  private static class IntegerDecimalReader extends ParquetValueReaders.PrimitiveReader<DecimalData> {
+    private final int precision;
+    private final int scale;
+
+    IntegerDecimalReader(ColumnDescriptor desc, int precision, int scale) {
+      super(desc);
+      this.precision = precision;
+      this.scale = scale;
+    }
+
+    @Override
+    public DecimalData read(DecimalData ignored) {
+      return DecimalData.fromUnscaledLong(column.nextInteger(), precision, scale);
+    }
+  }
+
+  private static class LongDecimalReader extends ParquetValueReaders.PrimitiveReader<DecimalData> {
+    private final int precision;
+    private final int scale;
+
+    LongDecimalReader(ColumnDescriptor desc, int precision, int scale) {
+      super(desc);
+      this.precision = precision;
+      this.scale = scale;
+    }
+
+    @Override
+    public DecimalData read(DecimalData ignored) {
+      return DecimalData.fromUnscaledLong(column.nextLong(), precision, scale);
+    }
   }
 
-  public static ParquetValueReader<Row> buildReader(Schema expectedSchema, MessageType fileSchema) {
-    return INSTANCE.createReader(expectedSchema, fileSchema);
+  private static class TimestampMicroReader extends ParquetValueReaders.UnboxedReader<TimestampData> {
+    TimestampMicroReader(ColumnDescriptor desc) {
+      super(desc);
+    }
+
+    @Override
+    public TimestampData read(TimestampData ignored) {
+      long value = readLong();
+      return TimestampData.fromInstant(Instant.ofEpochSecond(value / 1000_000, (value % 1000_000) * 1000));
+    }
+
+    @Override
+    public long readLong() {
+      return column.nextLong();
+    }
+  }
+
+  private static class StringReader extends ParquetValueReaders.PrimitiveReader<StringData> {
+    StringReader(ColumnDescriptor desc) {
+      super(desc);
+    }
+
+    @Override
+    public StringData read(StringData ignored) {
+      Binary binary = column.nextBinary();
+      ByteBuffer buffer = binary.toByteBuffer();
+      if (buffer.hasArray()) {
+        return StringData.fromBytes(
+            buffer.array(), buffer.arrayOffset() + buffer.position(), buffer.remaining());
+      } else {
+        return StringData.fromBytes(binary.getBytes());
+      }
+    }
   }
 
-  @Override
-  protected ParquetValueReader<Row> createStructReader(List<Type> types,
-                                                       List<ParquetValueReader<?>> fieldReaders,
-                                                       Types.StructType structType) {
-    return new RowReader(types, fieldReaders, structType);
+  private static class TimeMillisReader extends ParquetValueReaders.PrimitiveReader<Integer> {
+    TimeMillisReader(ColumnDescriptor desc) {
+      super(desc);
+    }
+
+    @Override
+    public Integer read(Integer reuse) {
+      // Flink only supports millisecond, so we discard microseconds in millisecond
+      return (int) column.nextLong() / 1000;

Review comment:
       Looks like Iceberg doesn't have time zone attribute for TimeType.  Do we need to handle timezone logic, here?
   
   I updated this to use `Math.floorDiv` anyway.

##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java
##########
@@ -19,64 +19,719 @@
 
 package org.apache.iceberg.flink.data;
 
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.time.Instant;
 import java.util.List;
-import org.apache.flink.types.Row;
+import java.util.Map;
+import org.apache.commons.lang3.ArrayUtils;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.MapData;
+import org.apache.flink.table.data.RawValueData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
 import org.apache.iceberg.Schema;
-import org.apache.iceberg.data.parquet.BaseParquetReaders;
+import org.apache.iceberg.parquet.ParquetSchemaUtil;
 import org.apache.iceberg.parquet.ParquetValueReader;
 import org.apache.iceberg.parquet.ParquetValueReaders;
+import org.apache.iceberg.parquet.TypeWithSchemaVisitor;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
 import org.apache.iceberg.types.Types;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.io.api.Binary;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.LogicalTypeAnnotation.DecimalLogicalTypeAnnotation;
 import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
 import org.apache.parquet.schema.Type;
 
-public class FlinkParquetReaders extends BaseParquetReaders<Row> {
+class FlinkParquetReaders {
+  private FlinkParquetReaders() {
+  }
 
-  private static final FlinkParquetReaders INSTANCE = new FlinkParquetReaders();
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema, MessageType fileSchema) {
+    return buildReader(expectedSchema, fileSchema, ImmutableMap.of());
+  }
 
-  private FlinkParquetReaders() {
+  @SuppressWarnings("unchecked")
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema,
+                                                        MessageType fileSchema,
+                                                        Map<Integer, ?> idToConstant) {
+    ReadBuilder builder = new ReadBuilder(fileSchema, idToConstant);
+    if (ParquetSchemaUtil.hasIds(fileSchema)) {
+      return (ParquetValueReader<RowData>)
+          TypeWithSchemaVisitor.visit(expectedSchema.asStruct(), fileSchema, builder);
+    } else {
+      return (ParquetValueReader<RowData>)
+          TypeWithSchemaVisitor.visit(expectedSchema.asStruct(), fileSchema,
+              new FallbackReadBuilder(builder));
+    }
+  }
+
+  private static class FallbackReadBuilder extends TypeWithSchemaVisitor<ParquetValueReader<?>> {
+    private MessageType type;
+    private final TypeWithSchemaVisitor<ParquetValueReader<?>> builder;
+
+    FallbackReadBuilder(TypeWithSchemaVisitor<ParquetValueReader<?>> builder) {
+      this.builder = builder;
+    }
+
+    @Override
+    public ParquetValueReader<?> message(Types.StructType expected, MessageType message,
+                                         List<ParquetValueReader<?>> fieldReaders) {
+      // the top level matches by ID, but the remaining IDs are missing
+      this.type = message;
+      return builder.struct(expected, message, fieldReaders);
+    }
+
+    @Override
+    public ParquetValueReader<?> struct(Types.StructType ignored, GroupType struct,
+                                        List<ParquetValueReader<?>> fieldReaders) {
+      // the expected struct is ignored because nested fields are never found when the IDs are missing
+      List<ParquetValueReader<?>> newFields = Lists.newArrayListWithExpectedSize(
+          fieldReaders.size());
+      List<Type> types = Lists.newArrayListWithExpectedSize(fieldReaders.size());
+      List<Type> fields = struct.getFields();
+      for (int i = 0; i < fields.size(); i += 1) {
+        Type fieldType = fields.get(i);
+        int fieldD = type.getMaxDefinitionLevel(path(fieldType.getName())) - 1;
+        newFields.add(ParquetValueReaders.option(fieldType, fieldD, fieldReaders.get(i)));
+        types.add(fieldType);
+      }
+
+      return new RowDataReader(types, newFields);
+    }
+  }
+
+  private static class ReadBuilder extends TypeWithSchemaVisitor<ParquetValueReader<?>> {
+    private final MessageType type;
+    private final Map<Integer, ?> idToConstant;
+
+    ReadBuilder(MessageType type, Map<Integer, ?> idToConstant) {
+      this.type = type;
+      this.idToConstant = idToConstant;
+    }
+
+    @Override
+    public ParquetValueReader<?> message(Types.StructType expected, MessageType message,
+                                         List<ParquetValueReader<?>> fieldReaders) {
+      return struct(expected, message.asGroupType(), fieldReaders);
+    }
+
+    @Override
+    public ParquetValueReader<?> struct(Types.StructType expected, GroupType struct,
+                                        List<ParquetValueReader<?>> fieldReaders) {
+      // match the expected struct's order
+      Map<Integer, ParquetValueReader<?>> readersById = Maps.newHashMap();
+      Map<Integer, Type> typesById = Maps.newHashMap();
+      List<Type> fields = struct.getFields();
+      for (int i = 0; i < fields.size(); i += 1) {
+        Type fieldType = fields.get(i);
+        int fieldD = type.getMaxDefinitionLevel(path(fieldType.getName())) - 1;
+        if (fieldType.getId() != null) {
+          int id = fieldType.getId().intValue();
+          readersById.put(id, ParquetValueReaders.option(fieldType, fieldD, fieldReaders.get(i)));
+          typesById.put(id, fieldType);
+        }
+      }
+
+      List<Types.NestedField> expectedFields = expected != null ?
+          expected.fields() : ImmutableList.of();
+      List<ParquetValueReader<?>> reorderedFields = Lists.newArrayListWithExpectedSize(
+          expectedFields.size());
+      List<Type> types = Lists.newArrayListWithExpectedSize(expectedFields.size());
+      for (Types.NestedField field : expectedFields) {
+        int id = field.fieldId();
+        if (idToConstant.containsKey(id)) {
+          // containsKey is used because the constant may be null
+          reorderedFields.add(ParquetValueReaders.constant(idToConstant.get(id)));
+          types.add(null);
+        } else {
+          ParquetValueReader<?> reader = readersById.get(id);
+          if (reader != null) {
+            reorderedFields.add(reader);
+            types.add(typesById.get(id));
+          } else {
+            reorderedFields.add(ParquetValueReaders.nulls());
+            types.add(null);
+          }
+        }
+      }
+
+      return new RowDataReader(types, reorderedFields);
+    }
+
+    @Override
+    public ParquetValueReader<?> list(Types.ListType expectedList, GroupType array,
+                                      ParquetValueReader<?> elementReader) {
+      GroupType repeated = array.getFields().get(0).asGroupType();
+      String[] repeatedPath = currentPath();
+
+      int repeatedD = type.getMaxDefinitionLevel(repeatedPath) - 1;
+      int repeatedR = type.getMaxRepetitionLevel(repeatedPath) - 1;
+
+      Type elementType = repeated.getType(0);
+      int elementD = type.getMaxDefinitionLevel(path(elementType.getName())) - 1;
+
+      return new ArrayReader<>(repeatedD, repeatedR, ParquetValueReaders.option(elementType, elementD, elementReader));
+    }
+
+    @Override
+    public ParquetValueReader<?> map(Types.MapType expectedMap, GroupType map,
+                                     ParquetValueReader<?> keyReader,
+                                     ParquetValueReader<?> valueReader) {
+      GroupType repeatedKeyValue = map.getFields().get(0).asGroupType();
+      String[] repeatedPath = currentPath();
+
+      int repeatedD = type.getMaxDefinitionLevel(repeatedPath) - 1;
+      int repeatedR = type.getMaxRepetitionLevel(repeatedPath) - 1;
+
+      Type keyType = repeatedKeyValue.getType(0);
+      int keyD = type.getMaxDefinitionLevel(path(keyType.getName())) - 1;
+      Type valueType = repeatedKeyValue.getType(1);
+      int valueD = type.getMaxDefinitionLevel(path(valueType.getName())) - 1;
+
+      return new MapReader<>(repeatedD, repeatedR,
+          ParquetValueReaders.option(keyType, keyD, keyReader),
+          ParquetValueReaders.option(valueType, valueD, valueReader));
+    }
+
+    @Override
+    public ParquetValueReader<?> primitive(org.apache.iceberg.types.Type.PrimitiveType expected,
+                                           PrimitiveType primitive) {
+      ColumnDescriptor desc = type.getColumnDescription(currentPath());
+
+      if (primitive.getOriginalType() != null) {
+        switch (primitive.getOriginalType()) {
+          case ENUM:
+          case JSON:
+          case UTF8:
+            return new StringReader(desc);
+          case INT_8:
+          case INT_16:
+          case INT_32:
+          case DATE:
+            if (expected != null && expected.typeId() == Types.LongType.get().typeId()) {
+              return new ParquetValueReaders.IntAsLongReader(desc);
+            } else {
+              return new ParquetValueReaders.UnboxedReader<>(desc);
+            }
+          case TIME_MICROS:
+            return new TimeMillisReader(desc);
+          case INT_64:
+            return new ParquetValueReaders.UnboxedReader<>(desc);
+          case TIMESTAMP_MICROS:
+            return new TimestampMicroReader(desc);
+          case DECIMAL:
+            DecimalLogicalTypeAnnotation decimal = (DecimalLogicalTypeAnnotation) primitive.getLogicalTypeAnnotation();
+            switch (primitive.getPrimitiveTypeName()) {
+              case BINARY:
+              case FIXED_LEN_BYTE_ARRAY:
+                return new BinaryDecimalReader(desc, decimal.getScale());
+              case INT64:
+                return new LongDecimalReader(desc, decimal.getPrecision(), decimal.getScale());
+              case INT32:
+                return new IntegerDecimalReader(desc, decimal.getPrecision(), decimal.getScale());
+              default:
+                throw new UnsupportedOperationException(
+                    "Unsupported base type for decimal: " + primitive.getPrimitiveTypeName());
+            }
+          case BSON:
+            return new ParquetValueReaders.ByteArrayReader(desc);
+          default:
+            throw new UnsupportedOperationException(
+                "Unsupported logical type: " + primitive.getOriginalType());
+        }
+      }
+
+      switch (primitive.getPrimitiveTypeName()) {
+        case FIXED_LEN_BYTE_ARRAY:
+        case BINARY:
+          return new ParquetValueReaders.ByteArrayReader(desc);
+        case INT32:
+          if (expected != null && expected.typeId() == org.apache.iceberg.types.Type.TypeID.LONG) {
+            return new ParquetValueReaders.IntAsLongReader(desc);
+          } else {
+            return new ParquetValueReaders.UnboxedReader<>(desc);
+          }
+        case FLOAT:
+          if (expected != null && expected.typeId() == org.apache.iceberg.types.Type.TypeID.DOUBLE) {
+            return new ParquetValueReaders.FloatAsDoubleReader(desc);
+          } else {
+            return new ParquetValueReaders.UnboxedReader<>(desc);
+          }
+        case BOOLEAN:
+        case INT64:
+        case DOUBLE:
+          return new ParquetValueReaders.UnboxedReader<>(desc);
+        default:
+          throw new UnsupportedOperationException("Unsupported type: " + primitive);
+      }
+    }
+  }
+
+  private static class BinaryDecimalReader extends ParquetValueReaders.PrimitiveReader<DecimalData> {
+    private final int scale;
+
+    BinaryDecimalReader(ColumnDescriptor desc, int scale) {
+      super(desc);
+      this.scale = scale;
+    }
+
+    @Override
+    public DecimalData read(DecimalData ignored) {
+      Binary binary = column.nextBinary();
+      BigDecimal bigDecimal = new BigDecimal(new BigInteger(binary.getBytes()), scale);
+      return DecimalData.fromBigDecimal(bigDecimal, bigDecimal.precision(), scale);
+    }
+  }
+
+  private static class IntegerDecimalReader extends ParquetValueReaders.PrimitiveReader<DecimalData> {
+    private final int precision;
+    private final int scale;
+
+    IntegerDecimalReader(ColumnDescriptor desc, int precision, int scale) {
+      super(desc);
+      this.precision = precision;
+      this.scale = scale;
+    }
+
+    @Override
+    public DecimalData read(DecimalData ignored) {
+      return DecimalData.fromUnscaledLong(column.nextInteger(), precision, scale);
+    }
+  }
+
+  private static class LongDecimalReader extends ParquetValueReaders.PrimitiveReader<DecimalData> {
+    private final int precision;
+    private final int scale;
+
+    LongDecimalReader(ColumnDescriptor desc, int precision, int scale) {
+      super(desc);
+      this.precision = precision;
+      this.scale = scale;
+    }
+
+    @Override
+    public DecimalData read(DecimalData ignored) {
+      return DecimalData.fromUnscaledLong(column.nextLong(), precision, scale);
+    }
   }
 
-  public static ParquetValueReader<Row> buildReader(Schema expectedSchema, MessageType fileSchema) {
-    return INSTANCE.createReader(expectedSchema, fileSchema);
+  private static class TimestampMicroReader extends ParquetValueReaders.UnboxedReader<TimestampData> {
+    TimestampMicroReader(ColumnDescriptor desc) {
+      super(desc);
+    }
+
+    @Override
+    public TimestampData read(TimestampData ignored) {
+      long value = readLong();
+      return TimestampData.fromInstant(Instant.ofEpochSecond(value / 1000_000, (value % 1000_000) * 1000));
+    }
+
+    @Override
+    public long readLong() {
+      return column.nextLong();
+    }
+  }
+
+  private static class StringReader extends ParquetValueReaders.PrimitiveReader<StringData> {
+    StringReader(ColumnDescriptor desc) {
+      super(desc);
+    }
+
+    @Override
+    public StringData read(StringData ignored) {
+      Binary binary = column.nextBinary();
+      ByteBuffer buffer = binary.toByteBuffer();
+      if (buffer.hasArray()) {
+        return StringData.fromBytes(
+            buffer.array(), buffer.arrayOffset() + buffer.position(), buffer.remaining());
+      } else {
+        return StringData.fromBytes(binary.getBytes());
+      }
+    }
   }
 
-  @Override
-  protected ParquetValueReader<Row> createStructReader(List<Type> types,
-                                                       List<ParquetValueReader<?>> fieldReaders,
-                                                       Types.StructType structType) {
-    return new RowReader(types, fieldReaders, structType);
+  private static class TimeMillisReader extends ParquetValueReaders.PrimitiveReader<Integer> {
+    TimeMillisReader(ColumnDescriptor desc) {
+      super(desc);
+    }
+
+    @Override
+    public Integer read(Integer reuse) {
+      // Flink only supports millisecond, so we discard microseconds in millisecond
+      return (int) column.nextLong() / 1000;
+    }
   }
 
-  private static class RowReader extends ParquetValueReaders.StructReader<Row, Row> {
-    private final Types.StructType structType;
+  private static class ArrayReader<E> extends ParquetValueReaders.RepeatedReader<ArrayData, ReusableArrayData, E> {
+    private int readPos = 0;
+    private int writePos = 0;
+
+    ArrayReader(int definitionLevel, int repetitionLevel, ParquetValueReader<E> reader) {
+      super(definitionLevel, repetitionLevel, reader);
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")

Review comment:
       Done.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] chenjunjiedada commented on pull request #1266: Flink: update parquet reader with schema visitor

Posted by GitBox <gi...@apache.org>.
chenjunjiedada commented on pull request #1266:
URL: https://github.com/apache/iceberg/pull/1266#issuecomment-670508553


   @openinx, Addressed your lastest comments. Thanks for your detailed review!


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] chenjunjiedada commented on a change in pull request #1266: Flink: update parquet reader with schema visitor

Posted by GitBox <gi...@apache.org>.
chenjunjiedada commented on a change in pull request #1266:
URL: https://github.com/apache/iceberg/pull/1266#discussion_r466907036



##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java
##########
@@ -19,64 +19,701 @@
 
 package org.apache.iceberg.flink.data;
 
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.time.Instant;
+import java.time.ZoneOffset;
 import java.util.List;
-import org.apache.flink.types.Row;
+import java.util.Map;
+import org.apache.commons.lang3.ArrayUtils;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.MapData;
+import org.apache.flink.table.data.RawValueData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
 import org.apache.iceberg.Schema;
-import org.apache.iceberg.data.parquet.BaseParquetReaders;
 import org.apache.iceberg.parquet.ParquetValueReader;
 import org.apache.iceberg.parquet.ParquetValueReaders;
+import org.apache.iceberg.parquet.TypeWithSchemaVisitor;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
 import org.apache.iceberg.types.Types;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.io.api.Binary;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.LogicalTypeAnnotation.DecimalLogicalTypeAnnotation;
 import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
 import org.apache.parquet.schema.Type;
 
-public class FlinkParquetReaders extends BaseParquetReaders<Row> {
+class FlinkParquetReaders {
+  private FlinkParquetReaders() {
+  }
 
-  private static final FlinkParquetReaders INSTANCE = new FlinkParquetReaders();
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema, MessageType fileSchema) {
+    return buildReader(expectedSchema, fileSchema, ImmutableMap.of());
+  }
 
-  private FlinkParquetReaders() {
+  @SuppressWarnings("unchecked")
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema,
+                                                        MessageType fileSchema,
+                                                        Map<Integer, ?> idToConstant) {
+    return (ParquetValueReader<RowData>) TypeWithSchemaVisitor.visit(expectedSchema.asStruct(), fileSchema,
+        new ReadBuilder(fileSchema, idToConstant)
+    );
+  }
+
+  private static class ReadBuilder extends TypeWithSchemaVisitor<ParquetValueReader<?>> {
+    private final MessageType type;
+    private final Map<Integer, ?> idToConstant;
+
+    ReadBuilder(MessageType type, Map<Integer, ?> idToConstant) {
+      this.type = type;
+      this.idToConstant = idToConstant;
+    }
+
+    @Override
+    public ParquetValueReader<RowData> message(Types.StructType expected, MessageType message,
+                                               List<ParquetValueReader<?>> fieldReaders) {
+      return struct(expected, message.asGroupType(), fieldReaders);
+    }
+
+    @Override
+    public ParquetValueReader<RowData> struct(Types.StructType expected, GroupType struct,
+                                              List<ParquetValueReader<?>> fieldReaders) {
+      // match the expected struct's order
+      Map<Integer, ParquetValueReader<?>> readersById = Maps.newHashMap();
+      Map<Integer, Type> typesById = Maps.newHashMap();
+      List<Type> fields = struct.getFields();
+      for (int i = 0; i < fields.size(); i += 1) {
+        Type fieldType = fields.get(i);
+        int fieldD = type.getMaxDefinitionLevel(path(fieldType.getName())) - 1;
+        if (fieldType.getId() != null) {
+          int id = fieldType.getId().intValue();
+          readersById.put(id, ParquetValueReaders.option(fieldType, fieldD, fieldReaders.get(i)));
+          typesById.put(id, fieldType);
+        }
+      }
+
+      List<Types.NestedField> expectedFields = expected != null ?
+          expected.fields() : ImmutableList.of();
+      List<ParquetValueReader<?>> reorderedFields = Lists.newArrayListWithExpectedSize(
+          expectedFields.size());
+      List<Type> types = Lists.newArrayListWithExpectedSize(expectedFields.size());
+      for (Types.NestedField field : expectedFields) {
+        int id = field.fieldId();
+        if (idToConstant.containsKey(id)) {
+          // containsKey is used because the constant may be null
+          reorderedFields.add(ParquetValueReaders.constant(idToConstant.get(id)));
+          types.add(null);
+        } else {
+          ParquetValueReader<?> reader = readersById.get(id);
+          if (reader != null) {
+            reorderedFields.add(reader);
+            types.add(typesById.get(id));
+          } else {
+            reorderedFields.add(ParquetValueReaders.nulls());
+            types.add(null);
+          }
+        }
+      }
+
+      return new RowDataReader(types, reorderedFields);
+    }
+
+    @Override
+    public ParquetValueReader<?> list(Types.ListType expectedList, GroupType array,
+                                      ParquetValueReader<?> elementReader) {
+      GroupType repeated = array.getFields().get(0).asGroupType();
+      String[] repeatedPath = currentPath();
+
+      int repeatedD = type.getMaxDefinitionLevel(repeatedPath) - 1;
+      int repeatedR = type.getMaxRepetitionLevel(repeatedPath) - 1;
+
+      Type elementType = repeated.getType(0);
+      int elementD = type.getMaxDefinitionLevel(path(elementType.getName())) - 1;
+
+      return new ArrayReader<>(repeatedD, repeatedR, ParquetValueReaders.option(elementType, elementD, elementReader));
+    }
+
+    @Override
+    public ParquetValueReader<?> map(Types.MapType expectedMap, GroupType map,
+                                     ParquetValueReader<?> keyReader,
+                                     ParquetValueReader<?> valueReader) {
+      GroupType repeatedKeyValue = map.getFields().get(0).asGroupType();
+      String[] repeatedPath = currentPath();
+
+      int repeatedD = type.getMaxDefinitionLevel(repeatedPath) - 1;
+      int repeatedR = type.getMaxRepetitionLevel(repeatedPath) - 1;
+
+      Type keyType = repeatedKeyValue.getType(0);
+      int keyD = type.getMaxDefinitionLevel(path(keyType.getName())) - 1;
+      Type valueType = repeatedKeyValue.getType(1);
+      int valueD = type.getMaxDefinitionLevel(path(valueType.getName())) - 1;
+
+      return new MapReader<>(repeatedD, repeatedR,
+          ParquetValueReaders.option(keyType, keyD, keyReader),
+          ParquetValueReaders.option(valueType, valueD, valueReader));
+    }
+
+    @Override
+    public ParquetValueReader<?> primitive(org.apache.iceberg.types.Type.PrimitiveType expected,
+                                           PrimitiveType primitive) {
+      ColumnDescriptor desc = type.getColumnDescription(currentPath());
+
+      if (primitive.getOriginalType() != null) {
+        switch (primitive.getOriginalType()) {
+          case ENUM:
+          case JSON:
+          case UTF8:
+            return new StringReader(desc);
+          case INT_8:
+          case INT_16:
+          case INT_32:
+          case DATE:

Review comment:
       Agreed, updated.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1266: Flink: update parquet reader with schema visitor

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1266:
URL: https://github.com/apache/iceberg/pull/1266#discussion_r469426103



##########
File path: flink/src/test/java/org/apache/iceberg/flink/data/TestHelpers.java
##########
@@ -0,0 +1,198 @@
+/*
+ * 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.iceberg.flink.data;
+
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.OffsetDateTime;
+import java.time.ZoneOffset;
+import java.time.temporal.ChronoUnit;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import java.util.function.Supplier;
+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.MapType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+
+public class TestHelpers {
+  private TestHelpers() {}
+
+  private static final OffsetDateTime EPOCH = Instant.ofEpochMilli(0L).atOffset(ZoneOffset.UTC);
+  private static final LocalDate EPOCH_DAY = EPOCH.toLocalDate();
+
+  public static void assertRowData(Type type, LogicalType rowType, Record expectedRecord, RowData actualRowData) {
+    if (expectedRecord == null && actualRowData == null) {
+      return;
+    }
+
+    Assert.assertTrue("expected Record and actual RowData should be both null or not null",
+        expectedRecord != null && actualRowData != null);
+
+    List<Type> types = Lists.newArrayList();
+    for (Types.NestedField field : type.asStructType().fields()) {
+      types.add(field.type());
+    }
+
+    for (int i = 0; i < types.size(); i += 1) {
+      if (expectedRecord.get(i) == null) {
+        Assert.assertTrue(actualRowData.isNullAt(i));
+        continue;
+      }
+
+      Object expected = expectedRecord.get(i);
+      LogicalType logicalType = ((RowType) rowType).getTypeAt(i);
+
+      final int fieldPos = i;
+      assertEquals(types.get(i), logicalType, expected,
+          () -> RowData.createFieldGetter(logicalType, fieldPos).getFieldOrNull(actualRowData));
+    }
+  }
+
+  private static void assertEquals(Type type, LogicalType logicalType, Object expected, Supplier<Object> supplier) {
+    switch (type.typeId()) {
+      case BOOLEAN:
+        Assert.assertEquals("boolean value should be equal", expected, supplier.get());
+        break;
+      case INTEGER:
+        Assert.assertEquals("int value should be equal", expected, supplier.get());
+        break;
+      case LONG:
+        Assert.assertEquals("long value should be equal", expected, supplier.get());
+        break;
+      case FLOAT:
+        Assert.assertEquals("float value should be equal", expected, supplier.get());
+        break;
+      case DOUBLE:
+        Assert.assertEquals("double value should be equal", expected, supplier.get());
+        break;
+      case STRING:
+        Assert.assertTrue("Should expect a CharSequence", expected instanceof CharSequence);
+        Assert.assertEquals("string should be equal",
+            String.valueOf(expected), supplier.get().toString());
+        break;
+      case DATE:
+        Assert.assertTrue("Should expect a Date", expected instanceof LocalDate);
+        LocalDate date = ChronoUnit.DAYS.addTo(EPOCH_DAY, (int) supplier.get());
+        Assert.assertEquals("date should be equal", expected, date);
+        break;
+      case TIME:
+        Assert.assertTrue("Should expect a LocalTime", expected instanceof LocalTime);
+        int milliseconds = (int) (((LocalTime) expected).toNanoOfDay() / 1000_000);
+        Assert.assertEquals("time millis should be equal", milliseconds, supplier.get());
+        break;
+      case TIMESTAMP:
+        if (((Types.TimestampType) type).shouldAdjustToUTC()) {
+          Assert.assertTrue("Should expect a OffsetDataTime", expected instanceof OffsetDateTime);
+          OffsetDateTime ts = (OffsetDateTime) expected;
+          Assert.assertEquals("OffsetDataTime should be equal", ts.toLocalDateTime(),
+              ((TimestampData) supplier.get()).toLocalDateTime());
+        } else {
+          Assert.assertTrue("Should expect a LocalDataTime", expected instanceof LocalDateTime);
+          LocalDateTime ts = (LocalDateTime) expected;
+          Assert.assertEquals("LocalDataTime should be equal", ts,
+              ((TimestampData) supplier.get()).toLocalDateTime());
+        }
+        break;
+      case BINARY:
+        Assert.assertTrue("Should expect a ByteBuffer", expected instanceof ByteBuffer);
+        Assert.assertArrayEquals("binary should be equal", ((ByteBuffer) expected).array(), (byte[]) supplier.get());
+        break;
+      case DECIMAL:
+        Assert.assertTrue("Should expect a BigDecimal", expected instanceof BigDecimal);
+        BigDecimal bd = (BigDecimal) expected;
+        Assert.assertEquals("decimal value should be equal", bd,
+            ((DecimalData) supplier.get()).toBigDecimal());
+        break;
+      case LIST:
+        Assert.assertTrue("Should expect a Collection", expected instanceof Collection);
+        Collection<?> expectedArrayData = (Collection<?>) expected;
+        ArrayData actualArrayData = (ArrayData) supplier.get();
+        LogicalType elementType = ((ArrayType) logicalType).getElementType();
+        Assert.assertEquals("array length should be equal", expectedArrayData.size(), actualArrayData.size());
+        assertArrayValues(type.asListType().elementType(), elementType, expectedArrayData, actualArrayData);
+        break;
+      case MAP:
+        Assert.assertTrue("Should expect a Map", expected instanceof Map);
+        MapData actual = (MapData) supplier.get();
+        Assert.assertEquals("map size should be equal",
+            ((Map<?, ?>) expected).size(), actual.size());
+        Collection<?> expectedKeyArrayData = ((Map<?, ?>) expected).keySet();

Review comment:
       `keySet` isn't required to be an ordered collection, so this could easily break if keys are not returned in the same order as they are in `MapData`. This also allows maps to be equal that actually are not, like `Map('b' -> 1, 'a' -> 2)` and `Map('a' -> 1, 'b' -> 2)` if the first map's keys are reordered.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] chenjunjiedada commented on a change in pull request #1266: Flink: update parquet reader with schema visitor

Posted by GitBox <gi...@apache.org>.
chenjunjiedada commented on a change in pull request #1266:
URL: https://github.com/apache/iceberg/pull/1266#discussion_r467028680



##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java
##########
@@ -19,64 +19,701 @@
 
 package org.apache.iceberg.flink.data;
 
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.time.Instant;
+import java.time.ZoneOffset;
 import java.util.List;
-import org.apache.flink.types.Row;
+import java.util.Map;
+import org.apache.commons.lang3.ArrayUtils;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.MapData;
+import org.apache.flink.table.data.RawValueData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
 import org.apache.iceberg.Schema;
-import org.apache.iceberg.data.parquet.BaseParquetReaders;
 import org.apache.iceberg.parquet.ParquetValueReader;
 import org.apache.iceberg.parquet.ParquetValueReaders;
+import org.apache.iceberg.parquet.TypeWithSchemaVisitor;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
 import org.apache.iceberg.types.Types;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.io.api.Binary;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.LogicalTypeAnnotation.DecimalLogicalTypeAnnotation;
 import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
 import org.apache.parquet.schema.Type;
 
-public class FlinkParquetReaders extends BaseParquetReaders<Row> {
+class FlinkParquetReaders {
+  private FlinkParquetReaders() {
+  }
 
-  private static final FlinkParquetReaders INSTANCE = new FlinkParquetReaders();
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema, MessageType fileSchema) {
+    return buildReader(expectedSchema, fileSchema, ImmutableMap.of());
+  }
 
-  private FlinkParquetReaders() {
+  @SuppressWarnings("unchecked")
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema,
+                                                        MessageType fileSchema,
+                                                        Map<Integer, ?> idToConstant) {
+    return (ParquetValueReader<RowData>) TypeWithSchemaVisitor.visit(expectedSchema.asStruct(), fileSchema,
+        new ReadBuilder(fileSchema, idToConstant)
+    );
+  }
+
+  private static class ReadBuilder extends TypeWithSchemaVisitor<ParquetValueReader<?>> {
+    private final MessageType type;
+    private final Map<Integer, ?> idToConstant;
+
+    ReadBuilder(MessageType type, Map<Integer, ?> idToConstant) {
+      this.type = type;
+      this.idToConstant = idToConstant;
+    }
+
+    @Override
+    public ParquetValueReader<RowData> message(Types.StructType expected, MessageType message,
+                                               List<ParquetValueReader<?>> fieldReaders) {
+      return struct(expected, message.asGroupType(), fieldReaders);
+    }
+
+    @Override
+    public ParquetValueReader<RowData> struct(Types.StructType expected, GroupType struct,
+                                              List<ParquetValueReader<?>> fieldReaders) {
+      // match the expected struct's order
+      Map<Integer, ParquetValueReader<?>> readersById = Maps.newHashMap();
+      Map<Integer, Type> typesById = Maps.newHashMap();
+      List<Type> fields = struct.getFields();
+      for (int i = 0; i < fields.size(); i += 1) {
+        Type fieldType = fields.get(i);
+        int fieldD = type.getMaxDefinitionLevel(path(fieldType.getName())) - 1;
+        if (fieldType.getId() != null) {
+          int id = fieldType.getId().intValue();
+          readersById.put(id, ParquetValueReaders.option(fieldType, fieldD, fieldReaders.get(i)));
+          typesById.put(id, fieldType);
+        }
+      }
+
+      List<Types.NestedField> expectedFields = expected != null ?
+          expected.fields() : ImmutableList.of();
+      List<ParquetValueReader<?>> reorderedFields = Lists.newArrayListWithExpectedSize(
+          expectedFields.size());
+      List<Type> types = Lists.newArrayListWithExpectedSize(expectedFields.size());
+      for (Types.NestedField field : expectedFields) {
+        int id = field.fieldId();
+        if (idToConstant.containsKey(id)) {
+          // containsKey is used because the constant may be null
+          reorderedFields.add(ParquetValueReaders.constant(idToConstant.get(id)));
+          types.add(null);
+        } else {
+          ParquetValueReader<?> reader = readersById.get(id);
+          if (reader != null) {
+            reorderedFields.add(reader);
+            types.add(typesById.get(id));
+          } else {
+            reorderedFields.add(ParquetValueReaders.nulls());
+            types.add(null);
+          }
+        }
+      }
+
+      return new RowDataReader(types, reorderedFields);
+    }
+
+    @Override
+    public ParquetValueReader<?> list(Types.ListType expectedList, GroupType array,
+                                      ParquetValueReader<?> elementReader) {
+      GroupType repeated = array.getFields().get(0).asGroupType();
+      String[] repeatedPath = currentPath();
+
+      int repeatedD = type.getMaxDefinitionLevel(repeatedPath) - 1;
+      int repeatedR = type.getMaxRepetitionLevel(repeatedPath) - 1;
+
+      Type elementType = repeated.getType(0);
+      int elementD = type.getMaxDefinitionLevel(path(elementType.getName())) - 1;
+
+      return new ArrayReader<>(repeatedD, repeatedR, ParquetValueReaders.option(elementType, elementD, elementReader));
+    }
+
+    @Override
+    public ParquetValueReader<?> map(Types.MapType expectedMap, GroupType map,
+                                     ParquetValueReader<?> keyReader,
+                                     ParquetValueReader<?> valueReader) {
+      GroupType repeatedKeyValue = map.getFields().get(0).asGroupType();
+      String[] repeatedPath = currentPath();
+
+      int repeatedD = type.getMaxDefinitionLevel(repeatedPath) - 1;
+      int repeatedR = type.getMaxRepetitionLevel(repeatedPath) - 1;
+
+      Type keyType = repeatedKeyValue.getType(0);
+      int keyD = type.getMaxDefinitionLevel(path(keyType.getName())) - 1;
+      Type valueType = repeatedKeyValue.getType(1);
+      int valueD = type.getMaxDefinitionLevel(path(valueType.getName())) - 1;
+
+      return new MapReader<>(repeatedD, repeatedR,
+          ParquetValueReaders.option(keyType, keyD, keyReader),
+          ParquetValueReaders.option(valueType, valueD, valueReader));
+    }
+
+    @Override
+    public ParquetValueReader<?> primitive(org.apache.iceberg.types.Type.PrimitiveType expected,
+                                           PrimitiveType primitive) {
+      ColumnDescriptor desc = type.getColumnDescription(currentPath());
+
+      if (primitive.getOriginalType() != null) {
+        switch (primitive.getOriginalType()) {
+          case ENUM:
+          case JSON:
+          case UTF8:
+            return new StringReader(desc);
+          case INT_8:
+          case INT_16:
+          case INT_32:
+          case DATE:
+            if (expected != null && expected.typeId() == Types.LongType.get().typeId()) {
+              return new ParquetValueReaders.IntAsLongReader(desc);
+            } else {
+              return new ParquetValueReaders.UnboxedReader<>(desc);
+            }
+          case TIME_MICROS:
+            return new LossyMicrosToMillisTimeReader(desc);
+          case INT_64:
+            return new ParquetValueReaders.UnboxedReader<>(desc);
+          case TIMESTAMP_MICROS:
+            if (((Types.TimestampType) expected).shouldAdjustToUTC()) {
+              return new TimestampTzReader(desc);
+            } else {
+              return new TimestampReader(desc);
+            }
+          case DECIMAL:
+            DecimalLogicalTypeAnnotation decimal = (DecimalLogicalTypeAnnotation) primitive.getLogicalTypeAnnotation();
+            switch (primitive.getPrimitiveTypeName()) {
+              case BINARY:
+              case FIXED_LEN_BYTE_ARRAY:
+                return new BinaryDecimalReader(desc, decimal.getScale());
+              case INT64:
+                return new LongDecimalReader(desc, decimal.getPrecision(), decimal.getScale());
+              case INT32:
+                return new IntegerDecimalReader(desc, decimal.getPrecision(), decimal.getScale());
+              default:
+                throw new UnsupportedOperationException(
+                    "Unsupported base type for decimal: " + primitive.getPrimitiveTypeName());
+            }
+          case BSON:
+            return new ParquetValueReaders.ByteArrayReader(desc);
+          default:
+            throw new UnsupportedOperationException(
+                "Unsupported logical type: " + primitive.getOriginalType());
+        }
+      }
+
+      switch (primitive.getPrimitiveTypeName()) {
+        case FIXED_LEN_BYTE_ARRAY:
+        case BINARY:
+          return new ParquetValueReaders.ByteArrayReader(desc);
+        case INT32:
+          if (expected != null && expected.typeId() == org.apache.iceberg.types.Type.TypeID.LONG) {
+            return new ParquetValueReaders.IntAsLongReader(desc);
+          } else {
+            return new ParquetValueReaders.UnboxedReader<>(desc);
+          }
+        case FLOAT:
+          if (expected != null && expected.typeId() == org.apache.iceberg.types.Type.TypeID.DOUBLE) {
+            return new ParquetValueReaders.FloatAsDoubleReader(desc);
+          } else {
+            return new ParquetValueReaders.UnboxedReader<>(desc);
+          }
+        case BOOLEAN:
+        case INT64:
+        case DOUBLE:
+          return new ParquetValueReaders.UnboxedReader<>(desc);
+        default:
+          throw new UnsupportedOperationException("Unsupported type: " + primitive);
+      }
+    }
+  }
+
+  private static class BinaryDecimalReader extends ParquetValueReaders.PrimitiveReader<DecimalData> {
+    private final int scale;
+
+    BinaryDecimalReader(ColumnDescriptor desc, int scale) {
+      super(desc);
+      this.scale = scale;
+    }
+
+    @Override
+    public DecimalData read(DecimalData ignored) {
+      Binary binary = column.nextBinary();
+      BigDecimal bigDecimal = new BigDecimal(new BigInteger(binary.getBytes()), scale);
+      return DecimalData.fromBigDecimal(bigDecimal, bigDecimal.precision(), scale);
+    }
+  }
+
+  private static class IntegerDecimalReader extends ParquetValueReaders.PrimitiveReader<DecimalData> {
+    private final int precision;
+    private final int scale;
+
+    IntegerDecimalReader(ColumnDescriptor desc, int precision, int scale) {
+      super(desc);
+      this.precision = precision;
+      this.scale = scale;
+    }
+
+    @Override
+    public DecimalData read(DecimalData ignored) {
+      return DecimalData.fromUnscaledLong(column.nextInteger(), precision, scale);
+    }
+  }
+
+  private static class LongDecimalReader extends ParquetValueReaders.PrimitiveReader<DecimalData> {
+    private final int precision;
+    private final int scale;
+
+    LongDecimalReader(ColumnDescriptor desc, int precision, int scale) {
+      super(desc);
+      this.precision = precision;
+      this.scale = scale;
+    }
+
+    @Override
+    public DecimalData read(DecimalData ignored) {
+      return DecimalData.fromUnscaledLong(column.nextLong(), precision, scale);
+    }
+  }
+
+  private static class TimestampTzReader extends ParquetValueReaders.UnboxedReader<TimestampData> {
+    TimestampTzReader(ColumnDescriptor desc) {
+      super(desc);
+    }
+
+    @Override
+    public TimestampData read(TimestampData ignored) {
+      long value = readLong();
+      return TimestampData.fromLocalDateTime(Instant.ofEpochSecond(Math.floorDiv(value, 1000_000),
+          Math.floorMod(value, 1000_000) * 1000)
+          .atOffset(ZoneOffset.UTC)
+          .toLocalDateTime());

Review comment:
       In the previous version, I used this way and change to this because I want to make it symmetrical. It looks not necessary now.   Just changed back. 




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #1266: Flink: update parquet reader with schema visitor

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1266:
URL: https://github.com/apache/iceberg/pull/1266#discussion_r466929987



##########
File path: flink/src/test/java/org/apache/iceberg/flink/data/TestHelpers.java
##########
@@ -0,0 +1,199 @@
+/*
+ * 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.iceberg.flink.data;
+
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.OffsetDateTime;
+import java.time.ZoneOffset;
+import java.time.temporal.ChronoUnit;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import java.util.function.Supplier;
+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.MapType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+
+public class TestHelpers {
+  private TestHelpers() {}
+
+  private static final OffsetDateTime EPOCH = Instant.ofEpochMilli(0L).atOffset(ZoneOffset.UTC);
+  private static final LocalDate EPOCH_DAY = EPOCH.toLocalDate();
+
+  public static void assertRowData(Type type, LogicalType rowType, Record expectedRecord, RowData actualRowData) {
+    if (expectedRecord == null && actualRowData == null) {
+      return;
+    }
+
+    Assert.assertTrue("expected Record and actual RowData should be both null or not null",
+        expectedRecord != null && actualRowData != null);
+
+    List<Type> types = Lists.newArrayList();
+    for (Types.NestedField field : type.asStructType().fields()) {
+      types.add(field.type());
+    }
+
+    for (int i = 0; i < types.size(); i += 1) {
+      if (expectedRecord.get(i) == null) {
+        Assert.assertTrue(actualRowData.isNullAt(i));
+        continue;
+      }
+
+      Object expected = expectedRecord.get(i);
+      LogicalType logicalType = ((RowType) rowType).getTypeAt(i);
+
+      final int fieldPos = i;
+      assertEquals(types.get(i), logicalType, expected,
+          () -> RowData.createFieldGetter(logicalType, fieldPos).getFieldOrNull(actualRowData));
+    }
+
+  }
+
+  private static void assertEquals(Type type, LogicalType logicalType, Object expected, Supplier<Object> supplier) {
+    switch (type.typeId()) {
+      case BOOLEAN:
+        Assert.assertEquals("boolean value should be equal", expected, supplier.get());
+        break;
+      case INTEGER:
+        Assert.assertEquals("int value should be equal", expected, supplier.get());
+        break;
+      case LONG:
+        Assert.assertEquals("long value should be equal", expected, supplier.get());
+        break;
+      case FLOAT:
+        Assert.assertEquals("float value should be equal", expected, supplier.get());
+        break;
+      case DOUBLE:
+        Assert.assertEquals("double value should be equal", expected, supplier.get());
+        break;
+      case STRING:
+        Assert.assertTrue("Should expect a CharSequence", expected instanceof CharSequence);
+        Assert.assertEquals("string should be equal",
+            String.valueOf(expected), supplier.get().toString());
+        break;
+      case DATE:
+        Assert.assertTrue("Should expect a Date", expected instanceof LocalDate);
+        LocalDate date = ChronoUnit.DAYS.addTo(EPOCH_DAY, (int) supplier.get());
+        Assert.assertEquals("date should be equal", expected, date);
+        break;
+      case TIME:
+        Assert.assertTrue("Should expect a LocalTime", expected instanceof LocalTime);
+        int milliseconds = (int) (((LocalTime) expected).toNanoOfDay() / 1000_000);
+        Assert.assertEquals("time millis should be equal", milliseconds, supplier.get());
+        break;
+      case TIMESTAMP:
+        if (((Types.TimestampType) type.asPrimitiveType()).shouldAdjustToUTC()) {

Review comment:
       nit:  use `((Types.TimestampType) type).shouldAdjustToUTC()`. 




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on pull request #1266: Flink: update parquet reader with schema visitor

Posted by GitBox <gi...@apache.org>.
rdblue commented on pull request #1266:
URL: https://github.com/apache/iceberg/pull/1266#issuecomment-676829884


   The only remaining issue that I see is that this uses `Supplier` when I don't think it needs to, but we can follow up. I'll merge this to unblock the next steps.


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1266: Flink: update parquet reader with schema visitor

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1266:
URL: https://github.com/apache/iceberg/pull/1266#discussion_r469415261



##########
File path: flink/src/test/java/org/apache/iceberg/flink/data/TestHelpers.java
##########
@@ -0,0 +1,198 @@
+/*
+ * 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.iceberg.flink.data;
+
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.OffsetDateTime;
+import java.time.ZoneOffset;
+import java.time.temporal.ChronoUnit;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import java.util.function.Supplier;
+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.MapType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+
+public class TestHelpers {
+  private TestHelpers() {}
+
+  private static final OffsetDateTime EPOCH = Instant.ofEpochMilli(0L).atOffset(ZoneOffset.UTC);
+  private static final LocalDate EPOCH_DAY = EPOCH.toLocalDate();
+
+  public static void assertRowData(Type type, LogicalType rowType, Record expectedRecord, RowData actualRowData) {

Review comment:
       If the objects to validate are `Record` and `RowData`, then `type` should be `StructType`.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #1266: Flink: update parquet reader with schema visitor

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1266:
URL: https://github.com/apache/iceberg/pull/1266#discussion_r464871472



##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java
##########
@@ -19,64 +19,701 @@
 
 package org.apache.iceberg.flink.data;
 
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.time.Instant;
+import java.time.ZoneOffset;
 import java.util.List;
-import org.apache.flink.types.Row;
+import java.util.Map;
+import org.apache.commons.lang3.ArrayUtils;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.MapData;
+import org.apache.flink.table.data.RawValueData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
 import org.apache.iceberg.Schema;
-import org.apache.iceberg.data.parquet.BaseParquetReaders;
 import org.apache.iceberg.parquet.ParquetValueReader;
 import org.apache.iceberg.parquet.ParquetValueReaders;
+import org.apache.iceberg.parquet.TypeWithSchemaVisitor;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
 import org.apache.iceberg.types.Types;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.io.api.Binary;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.LogicalTypeAnnotation.DecimalLogicalTypeAnnotation;
 import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
 import org.apache.parquet.schema.Type;
 
-public class FlinkParquetReaders extends BaseParquetReaders<Row> {
+class FlinkParquetReaders {
+  private FlinkParquetReaders() {
+  }
 
-  private static final FlinkParquetReaders INSTANCE = new FlinkParquetReaders();
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema, MessageType fileSchema) {
+    return buildReader(expectedSchema, fileSchema, ImmutableMap.of());
+  }
 
-  private FlinkParquetReaders() {
+  @SuppressWarnings("unchecked")
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema,
+                                                        MessageType fileSchema,
+                                                        Map<Integer, ?> idToConstant) {
+    return (ParquetValueReader<RowData>) TypeWithSchemaVisitor.visit(expectedSchema.asStruct(), fileSchema,
+        new ReadBuilder(fileSchema, idToConstant)
+    );
+  }
+
+  private static class ReadBuilder extends TypeWithSchemaVisitor<ParquetValueReader<?>> {
+    private final MessageType type;
+    private final Map<Integer, ?> idToConstant;
+
+    ReadBuilder(MessageType type, Map<Integer, ?> idToConstant) {
+      this.type = type;
+      this.idToConstant = idToConstant;
+    }
+
+    @Override
+    public ParquetValueReader<RowData> message(Types.StructType expected, MessageType message,
+                                         List<ParquetValueReader<?>> fieldReaders) {

Review comment:
       nit: could code format it, seems we missing the indent.

##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java
##########
@@ -19,64 +19,701 @@
 
 package org.apache.iceberg.flink.data;
 
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.time.Instant;
+import java.time.ZoneOffset;
 import java.util.List;
-import org.apache.flink.types.Row;
+import java.util.Map;
+import org.apache.commons.lang3.ArrayUtils;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.MapData;
+import org.apache.flink.table.data.RawValueData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
 import org.apache.iceberg.Schema;
-import org.apache.iceberg.data.parquet.BaseParquetReaders;
 import org.apache.iceberg.parquet.ParquetValueReader;
 import org.apache.iceberg.parquet.ParquetValueReaders;
+import org.apache.iceberg.parquet.TypeWithSchemaVisitor;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
 import org.apache.iceberg.types.Types;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.io.api.Binary;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.LogicalTypeAnnotation.DecimalLogicalTypeAnnotation;
 import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
 import org.apache.parquet.schema.Type;
 
-public class FlinkParquetReaders extends BaseParquetReaders<Row> {
+class FlinkParquetReaders {
+  private FlinkParquetReaders() {
+  }
 
-  private static final FlinkParquetReaders INSTANCE = new FlinkParquetReaders();
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema, MessageType fileSchema) {
+    return buildReader(expectedSchema, fileSchema, ImmutableMap.of());
+  }
 
-  private FlinkParquetReaders() {
+  @SuppressWarnings("unchecked")
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema,
+                                                        MessageType fileSchema,
+                                                        Map<Integer, ?> idToConstant) {
+    return (ParquetValueReader<RowData>) TypeWithSchemaVisitor.visit(expectedSchema.asStruct(), fileSchema,
+        new ReadBuilder(fileSchema, idToConstant)
+    );
+  }
+
+  private static class ReadBuilder extends TypeWithSchemaVisitor<ParquetValueReader<?>> {
+    private final MessageType type;
+    private final Map<Integer, ?> idToConstant;
+
+    ReadBuilder(MessageType type, Map<Integer, ?> idToConstant) {
+      this.type = type;
+      this.idToConstant = idToConstant;
+    }
+
+    @Override
+    public ParquetValueReader<RowData> message(Types.StructType expected, MessageType message,
+                                         List<ParquetValueReader<?>> fieldReaders) {
+      return struct(expected, message.asGroupType(), fieldReaders);
+    }
+
+    @Override
+    public ParquetValueReader<RowData> struct(Types.StructType expected, GroupType struct,
+                                        List<ParquetValueReader<?>> fieldReaders) {

Review comment:
       nit: indent ? 

##########
File path: flink/src/test/java/org/apache/iceberg/flink/data/RandomData.java
##########
@@ -88,20 +93,46 @@ public Row next() {
     };
   }
 
+  private static Iterable<Record> generateIcebergGenerics(Schema schema, int numRecords,
+                                                   Supplier<RandomGenericData.RandomDataGenerator<Record>> supplier) {
+    return () -> new Iterator<Record>() {
+      private final RandomGenericData.RandomDataGenerator<Record> generator = supplier.get();
+      private int count = 0;
+
+      @Override
+      public boolean hasNext() {
+        return count < numRecords;
+      }
+
+      @Override
+      public Record next() {
+        if (!hasNext()) {
+          throw new NoSuchElementException();
+        }
+        ++count;
+        return (Record) TypeUtil.visit(schema, generator);
+      }
+    };
+  }
+
+
+  public static Iterable<Record> generateRecords(Schema schema, int numRecords, long seed) {
+    return RandomGenericData.generate(schema, numRecords, seed);
+  }
+
   public static Iterable<Row> generate(Schema schema, int numRecords, long seed) {
     return generateData(schema, numRecords, () -> new RandomRowGenerator(seed));
   }
 
-  public static Iterable<Row> generateFallbackData(Schema schema, int numRecords, long seed, long numDictRows) {
-    return generateData(schema, numRecords, () -> new FallbackGenerator(seed, numDictRows));
+  public static Iterable<Record> generateFallbackRecords(Schema schema, int numRecords, long seed, long numDictRows) {
+    return generateIcebergGenerics(schema, numRecords, () -> new FallbackGenerator(seed, numDictRows));
   }
 
-  public static Iterable<Row> generateDictionaryEncodableData(Schema schema, int numRecords, long seed) {
-    return generateData(schema, numRecords, () -> new DictionaryEncodedGenerator(seed));
+  public static Iterable<Record> generateDictionaryEncodableRecords(Schema schema, int numRecords, long seed) {

Review comment:
       ditto.

##########
File path: flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetReader.java
##########
@@ -35,51 +36,49 @@
 
 import static org.apache.iceberg.flink.data.RandomData.COMPLEX_SCHEMA;
 
-public class TestFlinkParquetReaderWriter {
+public class TestFlinkParquetReader {

Review comment:
       Better to extend the `org.apache.iceberg.data.DataTest` because it provides more test cases, which `COMPLEX_SCHEMA`  did not cover.   ( I used `COMPLEX_SCHEMA` before,  because I don't know there's a better testing method). 

##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java
##########
@@ -19,64 +19,719 @@
 
 package org.apache.iceberg.flink.data;
 
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.time.Instant;
 import java.util.List;
-import org.apache.flink.types.Row;
+import java.util.Map;
+import org.apache.commons.lang3.ArrayUtils;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.MapData;
+import org.apache.flink.table.data.RawValueData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
 import org.apache.iceberg.Schema;
-import org.apache.iceberg.data.parquet.BaseParquetReaders;
+import org.apache.iceberg.parquet.ParquetSchemaUtil;
 import org.apache.iceberg.parquet.ParquetValueReader;
 import org.apache.iceberg.parquet.ParquetValueReaders;
+import org.apache.iceberg.parquet.TypeWithSchemaVisitor;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
 import org.apache.iceberg.types.Types;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.io.api.Binary;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.LogicalTypeAnnotation.DecimalLogicalTypeAnnotation;
 import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
 import org.apache.parquet.schema.Type;
 
-public class FlinkParquetReaders extends BaseParquetReaders<Row> {
+class FlinkParquetReaders {
+  private FlinkParquetReaders() {
+  }
 
-  private static final FlinkParquetReaders INSTANCE = new FlinkParquetReaders();
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema, MessageType fileSchema) {
+    return buildReader(expectedSchema, fileSchema, ImmutableMap.of());
+  }
 
-  private FlinkParquetReaders() {
+  @SuppressWarnings("unchecked")
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema,
+                                                        MessageType fileSchema,
+                                                        Map<Integer, ?> idToConstant) {
+    ReadBuilder builder = new ReadBuilder(fileSchema, idToConstant);
+    if (ParquetSchemaUtil.hasIds(fileSchema)) {
+      return (ParquetValueReader<RowData>)
+          TypeWithSchemaVisitor.visit(expectedSchema.asStruct(), fileSchema, builder);
+    } else {
+      return (ParquetValueReader<RowData>)
+          TypeWithSchemaVisitor.visit(expectedSchema.asStruct(), fileSchema,
+              new FallbackReadBuilder(builder));
+    }
+  }
+
+  private static class FallbackReadBuilder extends TypeWithSchemaVisitor<ParquetValueReader<?>> {
+    private MessageType type;
+    private final TypeWithSchemaVisitor<ParquetValueReader<?>> builder;
+
+    FallbackReadBuilder(TypeWithSchemaVisitor<ParquetValueReader<?>> builder) {
+      this.builder = builder;
+    }
+
+    @Override
+    public ParquetValueReader<?> message(Types.StructType expected, MessageType message,
+                                         List<ParquetValueReader<?>> fieldReaders) {
+      // the top level matches by ID, but the remaining IDs are missing
+      this.type = message;
+      return builder.struct(expected, message, fieldReaders);
+    }
+
+    @Override
+    public ParquetValueReader<?> struct(Types.StructType ignored, GroupType struct,
+                                        List<ParquetValueReader<?>> fieldReaders) {
+      // the expected struct is ignored because nested fields are never found when the IDs are missing
+      List<ParquetValueReader<?>> newFields = Lists.newArrayListWithExpectedSize(
+          fieldReaders.size());
+      List<Type> types = Lists.newArrayListWithExpectedSize(fieldReaders.size());
+      List<Type> fields = struct.getFields();
+      for (int i = 0; i < fields.size(); i += 1) {
+        Type fieldType = fields.get(i);
+        int fieldD = type.getMaxDefinitionLevel(path(fieldType.getName())) - 1;
+        newFields.add(ParquetValueReaders.option(fieldType, fieldD, fieldReaders.get(i)));
+        types.add(fieldType);
+      }
+
+      return new RowDataReader(types, newFields);
+    }
+  }
+
+  private static class ReadBuilder extends TypeWithSchemaVisitor<ParquetValueReader<?>> {
+    private final MessageType type;
+    private final Map<Integer, ?> idToConstant;
+
+    ReadBuilder(MessageType type, Map<Integer, ?> idToConstant) {
+      this.type = type;
+      this.idToConstant = idToConstant;
+    }
+
+    @Override
+    public ParquetValueReader<?> message(Types.StructType expected, MessageType message,
+                                         List<ParquetValueReader<?>> fieldReaders) {
+      return struct(expected, message.asGroupType(), fieldReaders);
+    }
+
+    @Override
+    public ParquetValueReader<?> struct(Types.StructType expected, GroupType struct,
+                                        List<ParquetValueReader<?>> fieldReaders) {
+      // match the expected struct's order
+      Map<Integer, ParquetValueReader<?>> readersById = Maps.newHashMap();
+      Map<Integer, Type> typesById = Maps.newHashMap();
+      List<Type> fields = struct.getFields();
+      for (int i = 0; i < fields.size(); i += 1) {
+        Type fieldType = fields.get(i);
+        int fieldD = type.getMaxDefinitionLevel(path(fieldType.getName())) - 1;
+        if (fieldType.getId() != null) {
+          int id = fieldType.getId().intValue();
+          readersById.put(id, ParquetValueReaders.option(fieldType, fieldD, fieldReaders.get(i)));
+          typesById.put(id, fieldType);
+        }
+      }
+
+      List<Types.NestedField> expectedFields = expected != null ?
+          expected.fields() : ImmutableList.of();
+      List<ParquetValueReader<?>> reorderedFields = Lists.newArrayListWithExpectedSize(
+          expectedFields.size());
+      List<Type> types = Lists.newArrayListWithExpectedSize(expectedFields.size());
+      for (Types.NestedField field : expectedFields) {
+        int id = field.fieldId();
+        if (idToConstant.containsKey(id)) {
+          // containsKey is used because the constant may be null
+          reorderedFields.add(ParquetValueReaders.constant(idToConstant.get(id)));
+          types.add(null);
+        } else {
+          ParquetValueReader<?> reader = readersById.get(id);
+          if (reader != null) {
+            reorderedFields.add(reader);
+            types.add(typesById.get(id));
+          } else {
+            reorderedFields.add(ParquetValueReaders.nulls());
+            types.add(null);
+          }
+        }
+      }
+
+      return new RowDataReader(types, reorderedFields);
+    }
+
+    @Override
+    public ParquetValueReader<?> list(Types.ListType expectedList, GroupType array,
+                                      ParquetValueReader<?> elementReader) {
+      GroupType repeated = array.getFields().get(0).asGroupType();
+      String[] repeatedPath = currentPath();
+
+      int repeatedD = type.getMaxDefinitionLevel(repeatedPath) - 1;
+      int repeatedR = type.getMaxRepetitionLevel(repeatedPath) - 1;
+
+      Type elementType = repeated.getType(0);
+      int elementD = type.getMaxDefinitionLevel(path(elementType.getName())) - 1;
+
+      return new ArrayReader<>(repeatedD, repeatedR, ParquetValueReaders.option(elementType, elementD, elementReader));
+    }
+
+    @Override
+    public ParquetValueReader<?> map(Types.MapType expectedMap, GroupType map,
+                                     ParquetValueReader<?> keyReader,
+                                     ParquetValueReader<?> valueReader) {
+      GroupType repeatedKeyValue = map.getFields().get(0).asGroupType();
+      String[] repeatedPath = currentPath();
+
+      int repeatedD = type.getMaxDefinitionLevel(repeatedPath) - 1;
+      int repeatedR = type.getMaxRepetitionLevel(repeatedPath) - 1;
+
+      Type keyType = repeatedKeyValue.getType(0);
+      int keyD = type.getMaxDefinitionLevel(path(keyType.getName())) - 1;
+      Type valueType = repeatedKeyValue.getType(1);
+      int valueD = type.getMaxDefinitionLevel(path(valueType.getName())) - 1;
+
+      return new MapReader<>(repeatedD, repeatedR,
+          ParquetValueReaders.option(keyType, keyD, keyReader),
+          ParquetValueReaders.option(valueType, valueD, valueReader));
+    }
+
+    @Override
+    public ParquetValueReader<?> primitive(org.apache.iceberg.types.Type.PrimitiveType expected,
+                                           PrimitiveType primitive) {
+      ColumnDescriptor desc = type.getColumnDescription(currentPath());
+
+      if (primitive.getOriginalType() != null) {
+        switch (primitive.getOriginalType()) {
+          case ENUM:
+          case JSON:
+          case UTF8:
+            return new StringReader(desc);
+          case INT_8:
+          case INT_16:
+          case INT_32:
+          case DATE:
+            if (expected != null && expected.typeId() == Types.LongType.get().typeId()) {
+              return new ParquetValueReaders.IntAsLongReader(desc);
+            } else {
+              return new ParquetValueReaders.UnboxedReader<>(desc);
+            }
+          case TIME_MICROS:
+            return new TimeMillisReader(desc);
+          case INT_64:
+            return new ParquetValueReaders.UnboxedReader<>(desc);
+          case TIMESTAMP_MICROS:
+            return new TimestampMicroReader(desc);
+          case DECIMAL:
+            DecimalLogicalTypeAnnotation decimal = (DecimalLogicalTypeAnnotation) primitive.getLogicalTypeAnnotation();
+            switch (primitive.getPrimitiveTypeName()) {
+              case BINARY:
+              case FIXED_LEN_BYTE_ARRAY:
+                return new BinaryDecimalReader(desc, decimal.getScale());
+              case INT64:
+                return new LongDecimalReader(desc, decimal.getPrecision(), decimal.getScale());
+              case INT32:
+                return new IntegerDecimalReader(desc, decimal.getPrecision(), decimal.getScale());
+              default:
+                throw new UnsupportedOperationException(
+                    "Unsupported base type for decimal: " + primitive.getPrimitiveTypeName());
+            }
+          case BSON:
+            return new ParquetValueReaders.ByteArrayReader(desc);
+          default:
+            throw new UnsupportedOperationException(
+                "Unsupported logical type: " + primitive.getOriginalType());
+        }
+      }
+
+      switch (primitive.getPrimitiveTypeName()) {
+        case FIXED_LEN_BYTE_ARRAY:
+        case BINARY:
+          return new ParquetValueReaders.ByteArrayReader(desc);
+        case INT32:
+          if (expected != null && expected.typeId() == org.apache.iceberg.types.Type.TypeID.LONG) {
+            return new ParquetValueReaders.IntAsLongReader(desc);
+          } else {
+            return new ParquetValueReaders.UnboxedReader<>(desc);
+          }
+        case FLOAT:
+          if (expected != null && expected.typeId() == org.apache.iceberg.types.Type.TypeID.DOUBLE) {
+            return new ParquetValueReaders.FloatAsDoubleReader(desc);
+          } else {
+            return new ParquetValueReaders.UnboxedReader<>(desc);
+          }
+        case BOOLEAN:
+        case INT64:
+        case DOUBLE:
+          return new ParquetValueReaders.UnboxedReader<>(desc);
+        default:
+          throw new UnsupportedOperationException("Unsupported type: " + primitive);
+      }
+    }
+  }
+
+  private static class BinaryDecimalReader extends ParquetValueReaders.PrimitiveReader<DecimalData> {
+    private final int scale;
+
+    BinaryDecimalReader(ColumnDescriptor desc, int scale) {
+      super(desc);
+      this.scale = scale;
+    }
+
+    @Override
+    public DecimalData read(DecimalData ignored) {
+      Binary binary = column.nextBinary();
+      BigDecimal bigDecimal = new BigDecimal(new BigInteger(binary.getBytes()), scale);
+      return DecimalData.fromBigDecimal(bigDecimal, bigDecimal.precision(), scale);
+    }
+  }
+
+  private static class IntegerDecimalReader extends ParquetValueReaders.PrimitiveReader<DecimalData> {
+    private final int precision;
+    private final int scale;
+
+    IntegerDecimalReader(ColumnDescriptor desc, int precision, int scale) {
+      super(desc);
+      this.precision = precision;
+      this.scale = scale;
+    }
+
+    @Override
+    public DecimalData read(DecimalData ignored) {
+      return DecimalData.fromUnscaledLong(column.nextInteger(), precision, scale);
+    }
+  }
+
+  private static class LongDecimalReader extends ParquetValueReaders.PrimitiveReader<DecimalData> {
+    private final int precision;
+    private final int scale;
+
+    LongDecimalReader(ColumnDescriptor desc, int precision, int scale) {
+      super(desc);
+      this.precision = precision;
+      this.scale = scale;
+    }
+
+    @Override
+    public DecimalData read(DecimalData ignored) {
+      return DecimalData.fromUnscaledLong(column.nextLong(), precision, scale);
+    }
   }
 
-  public static ParquetValueReader<Row> buildReader(Schema expectedSchema, MessageType fileSchema) {
-    return INSTANCE.createReader(expectedSchema, fileSchema);
+  private static class TimestampMicroReader extends ParquetValueReaders.UnboxedReader<TimestampData> {
+    TimestampMicroReader(ColumnDescriptor desc) {
+      super(desc);
+    }
+
+    @Override
+    public TimestampData read(TimestampData ignored) {
+      long value = readLong();
+      return TimestampData.fromInstant(Instant.ofEpochSecond(value / 1000_000, (value % 1000_000) * 1000));
+    }
+
+    @Override
+    public long readLong() {
+      return column.nextLong();
+    }
+  }
+
+  private static class StringReader extends ParquetValueReaders.PrimitiveReader<StringData> {
+    StringReader(ColumnDescriptor desc) {
+      super(desc);
+    }
+
+    @Override
+    public StringData read(StringData ignored) {
+      Binary binary = column.nextBinary();
+      ByteBuffer buffer = binary.toByteBuffer();
+      if (buffer.hasArray()) {
+        return StringData.fromBytes(
+            buffer.array(), buffer.arrayOffset() + buffer.position(), buffer.remaining());
+      } else {
+        return StringData.fromBytes(binary.getBytes());
+      }
+    }
   }
 
-  @Override
-  protected ParquetValueReader<Row> createStructReader(List<Type> types,
-                                                       List<ParquetValueReader<?>> fieldReaders,
-                                                       Types.StructType structType) {
-    return new RowReader(types, fieldReaders, structType);
+  private static class TimeMillisReader extends ParquetValueReaders.PrimitiveReader<Integer> {
+    TimeMillisReader(ColumnDescriptor desc) {
+      super(desc);
+    }
+
+    @Override
+    public Integer read(Integer reuse) {
+      // Flink only supports millisecond, so we discard microseconds in millisecond
+      return (int) column.nextLong() / 1000;
+    }
   }
 
-  private static class RowReader extends ParquetValueReaders.StructReader<Row, Row> {
-    private final Types.StructType structType;
+  private static class ArrayReader<E> extends ParquetValueReaders.RepeatedReader<ArrayData, ReusableArrayData, E> {
+    private int readPos = 0;
+    private int writePos = 0;
+
+    ArrayReader(int definitionLevel, int repetitionLevel, ParquetValueReader<E> reader) {
+      super(definitionLevel, repetitionLevel, reader);
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    protected ReusableArrayData newListData(ArrayData reuse) {
+      this.readPos = 0;
+      this.writePos = 0;
+
+      if (reuse instanceof ReusableArrayData) {
+        return (ReusableArrayData) reuse;
+      } else {
+        return new ReusableArrayData();
+      }
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    protected E getElement(ReusableArrayData list) {
+      E value = null;
+      if (readPos < list.capacity()) {
+        value = (E) list.values[readPos];
+      }
+
+      readPos += 1;
+
+      return value;
+    }
+
+    @Override
+    protected void addElement(ReusableArrayData reused, E element) {
+      if (writePos >= reused.capacity()) {
+        reused.grow();
+      }
+
+      reused.values[writePos] = element;
+
+      writePos += 1;
+    }
+
+    @Override
+    protected ArrayData buildList(ReusableArrayData list) {
+      list.setNumElements(writePos);
+      return list;
+    }
+  }
+
+  private static class MapReader<K, V> extends
+      ParquetValueReaders.RepeatedKeyValueReader<MapData, ReusableMapData, K, V> {
+    private int readPos = 0;
+    private int writePos = 0;
+
+    private final ParquetValueReaders.ReusableEntry<K, V> entry = new ParquetValueReaders.ReusableEntry<>();
+    private final ParquetValueReaders.ReusableEntry<K, V> nullEntry = new ParquetValueReaders.ReusableEntry<>();
+
+    MapReader(int definitionLevel, int repetitionLevel,
+              ParquetValueReader<K> keyReader, ParquetValueReader<V> valueReader) {
+      super(definitionLevel, repetitionLevel, keyReader, valueReader);
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    protected ReusableMapData newMapData(MapData reuse) {
+      this.readPos = 0;
+      this.writePos = 0;
+
+      if (reuse instanceof ReusableMapData) {
+        return (ReusableMapData) reuse;
+      } else {
+        return new ReusableMapData();
+      }
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    protected Map.Entry<K, V> getPair(ReusableMapData map) {
+      Map.Entry<K, V> kv = nullEntry;
+      if (readPos < map.capacity()) {
+        entry.set((K) map.keys.values[readPos], (V) map.values.values[readPos]);
+        kv = entry;
+      }
+
+      readPos += 1;
+
+      return kv;
+    }
+
+    @Override
+    protected void addPair(ReusableMapData map, K key, V value) {
+      if (writePos >= map.capacity()) {
+        map.grow();
+      }
+
+      map.keys.values[writePos] = key;
+      map.values.values[writePos] = value;
 
-    RowReader(List<Type> types, List<ParquetValueReader<?>> readers, Types.StructType struct) {
+      writePos += 1;
+    }
+
+    @Override
+    protected MapData buildMap(ReusableMapData map) {
+      map.setNumElements(writePos);
+      return map;
+    }
+  }
+
+  private static class RowDataReader extends ParquetValueReaders.StructReader<RowData, GenericRowData> {
+    private final int numFields;
+
+    RowDataReader(List<Type> types, List<ParquetValueReader<?>> readers) {
       super(types, readers);
-      this.structType = struct;
+      this.numFields = readers.size();
     }
 
     @Override
-    protected Row newStructData(Row reuse) {
-      if (reuse != null) {
-        return reuse;
+    protected GenericRowData newStructData(RowData reuse) {
+      if (reuse instanceof GenericRowData) {
+        return (GenericRowData) reuse;
       } else {
-        return new Row(structType.fields().size());
+        return new GenericRowData(numFields);
       }
     }
 
     @Override
-    protected Object getField(Row row, int pos) {
-      return row.getField(pos);
+    protected Object getField(GenericRowData intermediate, int pos) {
+      return intermediate.getField(pos);
+    }
+
+    @Override
+    protected RowData buildStruct(GenericRowData struct) {
+      return struct;
+    }
+
+    @Override
+    protected void set(GenericRowData row, int pos, Object value) {
+      row.setField(pos, value);
+    }
+
+    @Override
+    protected void setNull(GenericRowData row, int pos) {
+      row.setField(pos, null);
     }
 
     @Override
-    protected Row buildStruct(Row row) {
-      return row;
+    protected void setBoolean(GenericRowData row, int pos, boolean value) {
+      row.setField(pos, value);
+    }
+
+    @Override
+    protected void setInteger(GenericRowData row, int pos, int value) {
+      row.setField(pos, value);
+    }
+
+    @Override
+    protected void setLong(GenericRowData row, int pos, long value) {
+      row.setField(pos, value);
     }
 
     @Override
-    protected void set(Row row, int pos, Object value) {
+    protected void setFloat(GenericRowData row, int pos, float value) {
       row.setField(pos, value);
     }
+
+    @Override
+    protected void setDouble(GenericRowData row, int pos, double value) {
+      row.setField(pos, value);
+    }
+  }
+
+  private static class ReusableMapData implements MapData {
+    private final ReusableArrayData keys;
+    private final ReusableArrayData values;
+
+    private int numElements;
+
+    private ReusableMapData() {
+      this.keys = new ReusableArrayData();
+      this.values = new ReusableArrayData();
+    }
+
+    private void grow() {
+      keys.grow();
+      values.grow();
+    }
+
+    private int capacity() {
+      return keys.capacity();
+    }
+
+    public void setNumElements(int numElements) {
+      this.numElements = numElements;
+      keys.setNumElements(numElements);
+      values.setNumElements(numElements);
+    }
+
+    @Override
+    public int size() {
+      return numElements;
+    }
+
+    @Override
+    public ReusableArrayData keyArray() {
+      return keys;
+    }
+
+    @Override
+    public ReusableArrayData valueArray() {
+      return values;
+    }
+  }
+
+  private static class ReusableArrayData implements ArrayData {
+    private static final Object[] EMPTY = new Object[0];
+
+    private Object[] values = EMPTY;
+    private int numElements = 0;
+
+    private void grow() {
+      if (values.length == 0) {
+        this.values = new Object[20];
+      } else {
+        Object[] old = values;
+        this.values = new Object[old.length << 2];

Review comment:
       I still think we don't need to grow four times space,  actually double array space is enough. 

##########
File path: flink/src/test/java/org/apache/iceberg/flink/data/RandomData.java
##########
@@ -88,20 +93,46 @@ public Row next() {
     };
   }
 
+  private static Iterable<Record> generateIcebergGenerics(Schema schema, int numRecords,
+                                                   Supplier<RandomGenericData.RandomDataGenerator<Record>> supplier) {
+    return () -> new Iterator<Record>() {
+      private final RandomGenericData.RandomDataGenerator<Record> generator = supplier.get();
+      private int count = 0;
+
+      @Override
+      public boolean hasNext() {
+        return count < numRecords;
+      }
+
+      @Override
+      public Record next() {
+        if (!hasNext()) {
+          throw new NoSuchElementException();
+        }
+        ++count;
+        return (Record) TypeUtil.visit(schema, generator);
+      }
+    };
+  }
+
+
+  public static Iterable<Record> generateRecords(Schema schema, int numRecords, long seed) {

Review comment:
       ditto

##########
File path: flink/src/test/java/org/apache/iceberg/flink/data/RandomData.java
##########
@@ -88,20 +93,46 @@ public Row next() {
     };
   }
 
+  private static Iterable<Record> generateIcebergGenerics(Schema schema, int numRecords,
+                                                   Supplier<RandomGenericData.RandomDataGenerator<Record>> supplier) {
+    return () -> new Iterator<Record>() {
+      private final RandomGenericData.RandomDataGenerator<Record> generator = supplier.get();
+      private int count = 0;
+
+      @Override
+      public boolean hasNext() {
+        return count < numRecords;
+      }
+
+      @Override
+      public Record next() {
+        if (!hasNext()) {
+          throw new NoSuchElementException();
+        }
+        ++count;
+        return (Record) TypeUtil.visit(schema, generator);
+      }
+    };
+  }
+
+
+  public static Iterable<Record> generateRecords(Schema schema, int numRecords, long seed) {
+    return RandomGenericData.generate(schema, numRecords, seed);
+  }
+
   public static Iterable<Row> generate(Schema schema, int numRecords, long seed) {
     return generateData(schema, numRecords, () -> new RandomRowGenerator(seed));
   }
 
-  public static Iterable<Row> generateFallbackData(Schema schema, int numRecords, long seed, long numDictRows) {
-    return generateData(schema, numRecords, () -> new FallbackGenerator(seed, numDictRows));
+  public static Iterable<Record> generateFallbackRecords(Schema schema, int numRecords, long seed, long numDictRows) {

Review comment:
       ditto.

##########
File path: flink/src/test/java/org/apache/iceberg/flink/data/RandomData.java
##########
@@ -88,20 +93,46 @@ public Row next() {
     };
   }
 
+  private static Iterable<Record> generateIcebergGenerics(Schema schema, int numRecords,

Review comment:
       How about moving this method into `org.apache.iceberg.data.RandomGenericData`,  then we don't need to expose the RandomRecordGenerator to public ? 
   
   btw,  we could  integrate the `genericIcebergGenerics` method with `RandomGenericData#generate` . 

##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/ParquetWithFlinkSchemaVisitor.java
##########
@@ -0,0 +1,199 @@
+/*
+ * 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.iceberg.flink.data;
+
+import java.util.Deque;
+import java.util.List;
+import org.apache.flink.table.types.logical.ArrayType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.MapType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.table.types.logical.RowType.RowField;
+import org.apache.iceberg.avro.AvroSchemaUtil;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.OriginalType;
+import org.apache.parquet.schema.PrimitiveType;
+import org.apache.parquet.schema.Type;
+
+public class ParquetWithFlinkSchemaVisitor<T> {

Review comment:
       For this PR ( flink parquet reader) , seems we don't need this writer visitor .

##########
File path: flink/src/test/java/org/apache/iceberg/flink/data/TestHelpers.java
##########
@@ -0,0 +1,271 @@
+/*
+ * 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.iceberg.flink.data;
+
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.OffsetDateTime;
+import java.time.ZoneOffset;
+import java.time.temporal.ChronoUnit;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.RowData;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+
+public class TestHelpers {
+  private TestHelpers() {}
+
+  private static final OffsetDateTime EPOCH = Instant.ofEpochMilli(0L).atOffset(ZoneOffset.UTC);
+  private static final LocalDate EPOCH_DAY = EPOCH.toLocalDate();
+
+  public static void assertRowData(Type type, Record expected, RowData actual) {
+    if (expected == null && actual == null) {
+      return;
+    }
+
+    List<Type> types = new ArrayList<>();
+    for (Types.NestedField field : type.asStructType().fields()) {
+      types.add(field.type());
+    }
+
+    for (int i = 0; i < types.size(); i += 1) {
+      if (expected.get(i) == null) {
+        Assert.assertTrue(actual.isNullAt(i));
+        continue;
+      }
+      Type.TypeID typeId = types.get(i).typeId();
+      Object value = expected.get(i);
+      switch (typeId) {
+        case BOOLEAN:
+          Assert.assertEquals("boolean value should be equal", value, actual.getBoolean(i));
+          break;
+        case INTEGER:
+          Assert.assertEquals("int value should be equal", value, actual.getInt(i));
+          break;
+        case LONG:
+          Assert.assertEquals("long value should be equal", value, actual.getLong(i));
+          break;
+        case FLOAT:
+          Assert.assertEquals("float value should be equal", value, actual.getFloat(i));
+          break;
+        case DOUBLE:
+          Assert.assertEquals("double should be equal", value, actual.getDouble(i));
+          break;
+        case STRING:
+          Assert.assertTrue("Should expect a CharSequence", value instanceof CharSequence);
+          Assert.assertEquals("string should be equal", String.valueOf(value), actual.getString(i).toString());
+          break;
+        case DATE:
+          Assert.assertTrue("Should expect a Date", value instanceof LocalDate);
+          LocalDate date = ChronoUnit.DAYS.addTo(EPOCH_DAY, actual.getInt(i));
+          Assert.assertEquals("date should be equal", value, date);
+          break;
+        case TIME:
+          Assert.assertTrue("Should expect a LocalTime", value instanceof LocalTime);
+          int milliseconds = (int) (((LocalTime) value).toNanoOfDay() / 1000_000);
+          Assert.assertEquals("time millis should be equal", milliseconds, actual.getInt(i));
+          break;
+        case TIMESTAMP:
+          if (((Types.TimestampType) type.asPrimitiveType()).shouldAdjustToUTC()) {
+            Assert.assertTrue("Should expect a OffsetDataTime", value instanceof OffsetDateTime);
+            OffsetDateTime ts = (OffsetDateTime) value;
+            Assert.assertEquals("OffsetDataTime should be equal", ts.toLocalDateTime(),
+                actual.getTimestamp(i, 6).toLocalDateTime());
+          } else {
+            Assert.assertTrue("Should expect a LocalDataTime", value instanceof LocalDateTime);
+            LocalDateTime ts = (LocalDateTime) value;
+            Assert.assertEquals("LocalDataTime should be equal", ts,
+                actual.getTimestamp(i, 6).toLocalDateTime());
+          }
+          break;
+        case FIXED:
+          Assert.assertTrue("Should expect byte[]", value instanceof byte[]);
+          Assert.assertArrayEquals("binary should be equal", (byte[]) value, actual.getBinary(i));
+          break;
+        case BINARY:
+          Assert.assertTrue("Should expect a ByteBuffer", value instanceof ByteBuffer);
+          Assert.assertArrayEquals("binary should be equal", ((ByteBuffer) value).array(), actual.getBinary(i));
+          break;
+        case DECIMAL:

Review comment:
       Seems we share the same primitive value assertion between `assertRowData` and `assertArrayValues`, could it be a common method ? 

##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java
##########
@@ -19,64 +19,719 @@
 
 package org.apache.iceberg.flink.data;
 
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.time.Instant;
 import java.util.List;
-import org.apache.flink.types.Row;
+import java.util.Map;
+import org.apache.commons.lang3.ArrayUtils;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.MapData;
+import org.apache.flink.table.data.RawValueData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
 import org.apache.iceberg.Schema;
-import org.apache.iceberg.data.parquet.BaseParquetReaders;
+import org.apache.iceberg.parquet.ParquetSchemaUtil;
 import org.apache.iceberg.parquet.ParquetValueReader;
 import org.apache.iceberg.parquet.ParquetValueReaders;
+import org.apache.iceberg.parquet.TypeWithSchemaVisitor;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
 import org.apache.iceberg.types.Types;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.io.api.Binary;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.LogicalTypeAnnotation.DecimalLogicalTypeAnnotation;
 import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
 import org.apache.parquet.schema.Type;
 
-public class FlinkParquetReaders extends BaseParquetReaders<Row> {
+class FlinkParquetReaders {
+  private FlinkParquetReaders() {
+  }
 
-  private static final FlinkParquetReaders INSTANCE = new FlinkParquetReaders();
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema, MessageType fileSchema) {
+    return buildReader(expectedSchema, fileSchema, ImmutableMap.of());
+  }
 
-  private FlinkParquetReaders() {
+  @SuppressWarnings("unchecked")
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema,
+                                                        MessageType fileSchema,
+                                                        Map<Integer, ?> idToConstant) {
+    ReadBuilder builder = new ReadBuilder(fileSchema, idToConstant);
+    if (ParquetSchemaUtil.hasIds(fileSchema)) {
+      return (ParquetValueReader<RowData>)
+          TypeWithSchemaVisitor.visit(expectedSchema.asStruct(), fileSchema, builder);
+    } else {
+      return (ParquetValueReader<RowData>)
+          TypeWithSchemaVisitor.visit(expectedSchema.asStruct(), fileSchema,
+              new FallbackReadBuilder(builder));
+    }
+  }
+
+  private static class FallbackReadBuilder extends TypeWithSchemaVisitor<ParquetValueReader<?>> {
+    private MessageType type;
+    private final TypeWithSchemaVisitor<ParquetValueReader<?>> builder;
+
+    FallbackReadBuilder(TypeWithSchemaVisitor<ParquetValueReader<?>> builder) {
+      this.builder = builder;
+    }
+
+    @Override
+    public ParquetValueReader<?> message(Types.StructType expected, MessageType message,
+                                         List<ParquetValueReader<?>> fieldReaders) {
+      // the top level matches by ID, but the remaining IDs are missing
+      this.type = message;
+      return builder.struct(expected, message, fieldReaders);
+    }
+
+    @Override
+    public ParquetValueReader<?> struct(Types.StructType ignored, GroupType struct,
+                                        List<ParquetValueReader<?>> fieldReaders) {
+      // the expected struct is ignored because nested fields are never found when the IDs are missing
+      List<ParquetValueReader<?>> newFields = Lists.newArrayListWithExpectedSize(
+          fieldReaders.size());
+      List<Type> types = Lists.newArrayListWithExpectedSize(fieldReaders.size());
+      List<Type> fields = struct.getFields();
+      for (int i = 0; i < fields.size(); i += 1) {
+        Type fieldType = fields.get(i);
+        int fieldD = type.getMaxDefinitionLevel(path(fieldType.getName())) - 1;
+        newFields.add(ParquetValueReaders.option(fieldType, fieldD, fieldReaders.get(i)));
+        types.add(fieldType);
+      }
+
+      return new RowDataReader(types, newFields);
+    }
+  }
+
+  private static class ReadBuilder extends TypeWithSchemaVisitor<ParquetValueReader<?>> {

Review comment:
       Better to create an issue to address this thing,  once the readers and writers get in,  we could do the refactor in that issue .




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] chenjunjiedada commented on pull request #1266: Flink: update parquet reader with schema visitor

Posted by GitBox <gi...@apache.org>.
chenjunjiedada commented on pull request #1266:
URL: https://github.com/apache/iceberg/pull/1266#issuecomment-667134966


   @openinx, Thanks for your comments. I just addressed them, would you please help to take another if you have time? 


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on pull request #1266: Flink: update parquet reader with schema visitor

Posted by GitBox <gi...@apache.org>.
rdblue commented on pull request #1266:
URL: https://github.com/apache/iceberg/pull/1266#issuecomment-673022067


   Thanks, @chenjunjiedada. I think the implementation and tests are correct, but I'd like to fix the data assertions before merging.


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1266: Flink: update parquet reader with schema visitor

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1266:
URL: https://github.com/apache/iceberg/pull/1266#discussion_r469424526



##########
File path: flink/src/test/java/org/apache/iceberg/flink/data/TestHelpers.java
##########
@@ -0,0 +1,198 @@
+/*
+ * 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.iceberg.flink.data;
+
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.OffsetDateTime;
+import java.time.ZoneOffset;
+import java.time.temporal.ChronoUnit;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import java.util.function.Supplier;
+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.MapType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+
+public class TestHelpers {
+  private TestHelpers() {}
+
+  private static final OffsetDateTime EPOCH = Instant.ofEpochMilli(0L).atOffset(ZoneOffset.UTC);
+  private static final LocalDate EPOCH_DAY = EPOCH.toLocalDate();
+
+  public static void assertRowData(Type type, LogicalType rowType, Record expectedRecord, RowData actualRowData) {
+    if (expectedRecord == null && actualRowData == null) {
+      return;
+    }
+
+    Assert.assertTrue("expected Record and actual RowData should be both null or not null",
+        expectedRecord != null && actualRowData != null);
+
+    List<Type> types = Lists.newArrayList();
+    for (Types.NestedField field : type.asStructType().fields()) {
+      types.add(field.type());
+    }
+
+    for (int i = 0; i < types.size(); i += 1) {
+      if (expectedRecord.get(i) == null) {
+        Assert.assertTrue(actualRowData.isNullAt(i));
+        continue;
+      }
+
+      Object expected = expectedRecord.get(i);
+      LogicalType logicalType = ((RowType) rowType).getTypeAt(i);
+
+      final int fieldPos = i;
+      assertEquals(types.get(i), logicalType, expected,
+          () -> RowData.createFieldGetter(logicalType, fieldPos).getFieldOrNull(actualRowData));
+    }
+  }
+
+  private static void assertEquals(Type type, LogicalType logicalType, Object expected, Supplier<Object> supplier) {
+    switch (type.typeId()) {
+      case BOOLEAN:
+        Assert.assertEquals("boolean value should be equal", expected, supplier.get());
+        break;
+      case INTEGER:
+        Assert.assertEquals("int value should be equal", expected, supplier.get());
+        break;
+      case LONG:
+        Assert.assertEquals("long value should be equal", expected, supplier.get());
+        break;
+      case FLOAT:
+        Assert.assertEquals("float value should be equal", expected, supplier.get());
+        break;
+      case DOUBLE:
+        Assert.assertEquals("double value should be equal", expected, supplier.get());
+        break;
+      case STRING:
+        Assert.assertTrue("Should expect a CharSequence", expected instanceof CharSequence);
+        Assert.assertEquals("string should be equal",
+            String.valueOf(expected), supplier.get().toString());
+        break;
+      case DATE:
+        Assert.assertTrue("Should expect a Date", expected instanceof LocalDate);
+        LocalDate date = ChronoUnit.DAYS.addTo(EPOCH_DAY, (int) supplier.get());
+        Assert.assertEquals("date should be equal", expected, date);
+        break;
+      case TIME:
+        Assert.assertTrue("Should expect a LocalTime", expected instanceof LocalTime);
+        int milliseconds = (int) (((LocalTime) expected).toNanoOfDay() / 1000_000);
+        Assert.assertEquals("time millis should be equal", milliseconds, supplier.get());
+        break;
+      case TIMESTAMP:
+        if (((Types.TimestampType) type).shouldAdjustToUTC()) {
+          Assert.assertTrue("Should expect a OffsetDataTime", expected instanceof OffsetDateTime);
+          OffsetDateTime ts = (OffsetDateTime) expected;
+          Assert.assertEquals("OffsetDataTime should be equal", ts.toLocalDateTime(),
+              ((TimestampData) supplier.get()).toLocalDateTime());
+        } else {
+          Assert.assertTrue("Should expect a LocalDataTime", expected instanceof LocalDateTime);
+          LocalDateTime ts = (LocalDateTime) expected;
+          Assert.assertEquals("LocalDataTime should be equal", ts,
+              ((TimestampData) supplier.get()).toLocalDateTime());
+        }
+        break;
+      case BINARY:
+        Assert.assertTrue("Should expect a ByteBuffer", expected instanceof ByteBuffer);
+        Assert.assertArrayEquals("binary should be equal", ((ByteBuffer) expected).array(), (byte[]) supplier.get());
+        break;
+      case DECIMAL:
+        Assert.assertTrue("Should expect a BigDecimal", expected instanceof BigDecimal);
+        BigDecimal bd = (BigDecimal) expected;
+        Assert.assertEquals("decimal value should be equal", bd,
+            ((DecimalData) supplier.get()).toBigDecimal());
+        break;
+      case LIST:
+        Assert.assertTrue("Should expect a Collection", expected instanceof Collection);
+        Collection<?> expectedArrayData = (Collection<?>) expected;
+        ArrayData actualArrayData = (ArrayData) supplier.get();
+        LogicalType elementType = ((ArrayType) logicalType).getElementType();
+        Assert.assertEquals("array length should be equal", expectedArrayData.size(), actualArrayData.size());
+        assertArrayValues(type.asListType().elementType(), elementType, expectedArrayData, actualArrayData);

Review comment:
       I think it would be better to keep this already large method a bit smaller and add a method to assert array equality, not just one for values. Same with maps. An `assertEquals(MapType, LogicalType, Map, MapData)` method would be helpful.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] chenjunjiedada commented on a change in pull request #1266: Flink: update parquet reader with schema visitor

Posted by GitBox <gi...@apache.org>.
chenjunjiedada commented on a change in pull request #1266:
URL: https://github.com/apache/iceberg/pull/1266#discussion_r463626895



##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java
##########
@@ -19,64 +19,719 @@
 
 package org.apache.iceberg.flink.data;
 
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.time.Instant;
 import java.util.List;
-import org.apache.flink.types.Row;
+import java.util.Map;
+import org.apache.commons.lang3.ArrayUtils;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.MapData;
+import org.apache.flink.table.data.RawValueData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
 import org.apache.iceberg.Schema;
-import org.apache.iceberg.data.parquet.BaseParquetReaders;
+import org.apache.iceberg.parquet.ParquetSchemaUtil;
 import org.apache.iceberg.parquet.ParquetValueReader;
 import org.apache.iceberg.parquet.ParquetValueReaders;
+import org.apache.iceberg.parquet.TypeWithSchemaVisitor;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
 import org.apache.iceberg.types.Types;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.io.api.Binary;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.LogicalTypeAnnotation.DecimalLogicalTypeAnnotation;
 import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
 import org.apache.parquet.schema.Type;
 
-public class FlinkParquetReaders extends BaseParquetReaders<Row> {
+class FlinkParquetReaders {
+  private FlinkParquetReaders() {
+  }
 
-  private static final FlinkParquetReaders INSTANCE = new FlinkParquetReaders();
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema, MessageType fileSchema) {
+    return buildReader(expectedSchema, fileSchema, ImmutableMap.of());
+  }
 
-  private FlinkParquetReaders() {
+  @SuppressWarnings("unchecked")
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema,
+                                                        MessageType fileSchema,
+                                                        Map<Integer, ?> idToConstant) {
+    ReadBuilder builder = new ReadBuilder(fileSchema, idToConstant);
+    if (ParquetSchemaUtil.hasIds(fileSchema)) {
+      return (ParquetValueReader<RowData>)
+          TypeWithSchemaVisitor.visit(expectedSchema.asStruct(), fileSchema, builder);
+    } else {
+      return (ParquetValueReader<RowData>)
+          TypeWithSchemaVisitor.visit(expectedSchema.asStruct(), fileSchema,
+              new FallbackReadBuilder(builder));
+    }
+  }
+
+  private static class FallbackReadBuilder extends TypeWithSchemaVisitor<ParquetValueReader<?>> {
+    private MessageType type;
+    private final TypeWithSchemaVisitor<ParquetValueReader<?>> builder;

Review comment:
       It is. I forgot that it doesn't extend the `ReaderBuilder` now but its parent class. This is a bit complex than directly extend the `ReaderBuilder` and override necessary methods. @rdblue, what do you think?




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] chenjunjiedada commented on a change in pull request #1266: Flink: update parquet reader with schema visitor

Posted by GitBox <gi...@apache.org>.
chenjunjiedada commented on a change in pull request #1266:
URL: https://github.com/apache/iceberg/pull/1266#discussion_r463624525



##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java
##########
@@ -19,64 +19,719 @@
 
 package org.apache.iceberg.flink.data;
 
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.time.Instant;
 import java.util.List;
-import org.apache.flink.types.Row;
+import java.util.Map;
+import org.apache.commons.lang3.ArrayUtils;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.MapData;
+import org.apache.flink.table.data.RawValueData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
 import org.apache.iceberg.Schema;
-import org.apache.iceberg.data.parquet.BaseParquetReaders;
+import org.apache.iceberg.parquet.ParquetSchemaUtil;
 import org.apache.iceberg.parquet.ParquetValueReader;
 import org.apache.iceberg.parquet.ParquetValueReaders;
+import org.apache.iceberg.parquet.TypeWithSchemaVisitor;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
 import org.apache.iceberg.types.Types;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.io.api.Binary;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.LogicalTypeAnnotation.DecimalLogicalTypeAnnotation;
 import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
 import org.apache.parquet.schema.Type;
 
-public class FlinkParquetReaders extends BaseParquetReaders<Row> {
+class FlinkParquetReaders {
+  private FlinkParquetReaders() {
+  }
 
-  private static final FlinkParquetReaders INSTANCE = new FlinkParquetReaders();
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema, MessageType fileSchema) {
+    return buildReader(expectedSchema, fileSchema, ImmutableMap.of());
+  }
 
-  private FlinkParquetReaders() {
+  @SuppressWarnings("unchecked")
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema,
+                                                        MessageType fileSchema,
+                                                        Map<Integer, ?> idToConstant) {
+    ReadBuilder builder = new ReadBuilder(fileSchema, idToConstant);
+    if (ParquetSchemaUtil.hasIds(fileSchema)) {
+      return (ParquetValueReader<RowData>)
+          TypeWithSchemaVisitor.visit(expectedSchema.asStruct(), fileSchema, builder);
+    } else {
+      return (ParquetValueReader<RowData>)
+          TypeWithSchemaVisitor.visit(expectedSchema.asStruct(), fileSchema,
+              new FallbackReadBuilder(builder));
+    }
+  }
+
+  private static class FallbackReadBuilder extends TypeWithSchemaVisitor<ParquetValueReader<?>> {
+    private MessageType type;
+    private final TypeWithSchemaVisitor<ParquetValueReader<?>> builder;
+
+    FallbackReadBuilder(TypeWithSchemaVisitor<ParquetValueReader<?>> builder) {
+      this.builder = builder;
+    }
+
+    @Override
+    public ParquetValueReader<?> message(Types.StructType expected, MessageType message,
+                                         List<ParquetValueReader<?>> fieldReaders) {
+      // the top level matches by ID, but the remaining IDs are missing
+      this.type = message;
+      return builder.struct(expected, message, fieldReaders);
+    }
+
+    @Override
+    public ParquetValueReader<?> struct(Types.StructType ignored, GroupType struct,
+                                        List<ParquetValueReader<?>> fieldReaders) {
+      // the expected struct is ignored because nested fields are never found when the IDs are missing
+      List<ParquetValueReader<?>> newFields = Lists.newArrayListWithExpectedSize(
+          fieldReaders.size());
+      List<Type> types = Lists.newArrayListWithExpectedSize(fieldReaders.size());
+      List<Type> fields = struct.getFields();
+      for (int i = 0; i < fields.size(); i += 1) {
+        Type fieldType = fields.get(i);
+        int fieldD = type.getMaxDefinitionLevel(path(fieldType.getName())) - 1;
+        newFields.add(ParquetValueReaders.option(fieldType, fieldD, fieldReaders.get(i)));
+        types.add(fieldType);
+      }
+
+      return new RowDataReader(types, newFields);
+    }
+  }
+
+  private static class ReadBuilder extends TypeWithSchemaVisitor<ParquetValueReader<?>> {

Review comment:
       +1 to refactor this. The struct/list/map methods are mostly the same, only primitive methods are different from each other due to differences in data models. This should also apply to Avro and Orc `ReadBuilder`.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1266: Flink: update parquet reader with schema visitor

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1266:
URL: https://github.com/apache/iceberg/pull/1266#discussion_r469416739



##########
File path: flink/src/test/java/org/apache/iceberg/flink/data/TestHelpers.java
##########
@@ -0,0 +1,198 @@
+/*
+ * 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.iceberg.flink.data;
+
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.OffsetDateTime;
+import java.time.ZoneOffset;
+import java.time.temporal.ChronoUnit;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import java.util.function.Supplier;
+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.MapType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+
+public class TestHelpers {
+  private TestHelpers() {}
+
+  private static final OffsetDateTime EPOCH = Instant.ofEpochMilli(0L).atOffset(ZoneOffset.UTC);
+  private static final LocalDate EPOCH_DAY = EPOCH.toLocalDate();
+
+  public static void assertRowData(Type type, LogicalType rowType, Record expectedRecord, RowData actualRowData) {
+    if (expectedRecord == null && actualRowData == null) {
+      return;
+    }
+
+    Assert.assertTrue("expected Record and actual RowData should be both null or not null",
+        expectedRecord != null && actualRowData != null);
+
+    List<Type> types = Lists.newArrayList();
+    for (Types.NestedField field : type.asStructType().fields()) {
+      types.add(field.type());
+    }
+
+    for (int i = 0; i < types.size(); i += 1) {
+      if (expectedRecord.get(i) == null) {
+        Assert.assertTrue(actualRowData.isNullAt(i));
+        continue;
+      }
+
+      Object expected = expectedRecord.get(i);
+      LogicalType logicalType = ((RowType) rowType).getTypeAt(i);
+
+      final int fieldPos = i;
+      assertEquals(types.get(i), logicalType, expected,
+          () -> RowData.createFieldGetter(logicalType, fieldPos).getFieldOrNull(actualRowData));

Review comment:
       If this uses `getFieldOrNull` then there is no need to do a `null` check at the top of this loop. It can be done in `assertEquals` called here.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1266: Flink: update parquet reader with schema visitor

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1266:
URL: https://github.com/apache/iceberg/pull/1266#discussion_r469419599



##########
File path: flink/src/test/java/org/apache/iceberg/flink/data/TestHelpers.java
##########
@@ -0,0 +1,198 @@
+/*
+ * 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.iceberg.flink.data;
+
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.OffsetDateTime;
+import java.time.ZoneOffset;
+import java.time.temporal.ChronoUnit;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import java.util.function.Supplier;
+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.MapType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+
+public class TestHelpers {
+  private TestHelpers() {}
+
+  private static final OffsetDateTime EPOCH = Instant.ofEpochMilli(0L).atOffset(ZoneOffset.UTC);
+  private static final LocalDate EPOCH_DAY = EPOCH.toLocalDate();
+
+  public static void assertRowData(Type type, LogicalType rowType, Record expectedRecord, RowData actualRowData) {
+    if (expectedRecord == null && actualRowData == null) {
+      return;
+    }
+
+    Assert.assertTrue("expected Record and actual RowData should be both null or not null",
+        expectedRecord != null && actualRowData != null);
+
+    List<Type> types = Lists.newArrayList();
+    for (Types.NestedField field : type.asStructType().fields()) {
+      types.add(field.type());
+    }
+
+    for (int i = 0; i < types.size(); i += 1) {
+      if (expectedRecord.get(i) == null) {
+        Assert.assertTrue(actualRowData.isNullAt(i));
+        continue;
+      }
+
+      Object expected = expectedRecord.get(i);
+      LogicalType logicalType = ((RowType) rowType).getTypeAt(i);
+
+      final int fieldPos = i;
+      assertEquals(types.get(i), logicalType, expected,
+          () -> RowData.createFieldGetter(logicalType, fieldPos).getFieldOrNull(actualRowData));
+    }
+  }
+
+  private static void assertEquals(Type type, LogicalType logicalType, Object expected, Supplier<Object> supplier) {
+    switch (type.typeId()) {
+      case BOOLEAN:
+        Assert.assertEquals("boolean value should be equal", expected, supplier.get());
+        break;
+      case INTEGER:
+        Assert.assertEquals("int value should be equal", expected, supplier.get());
+        break;
+      case LONG:
+        Assert.assertEquals("long value should be equal", expected, supplier.get());
+        break;
+      case FLOAT:
+        Assert.assertEquals("float value should be equal", expected, supplier.get());
+        break;
+      case DOUBLE:
+        Assert.assertEquals("double value should be equal", expected, supplier.get());
+        break;
+      case STRING:
+        Assert.assertTrue("Should expect a CharSequence", expected instanceof CharSequence);
+        Assert.assertEquals("string should be equal",
+            String.valueOf(expected), supplier.get().toString());
+        break;
+      case DATE:
+        Assert.assertTrue("Should expect a Date", expected instanceof LocalDate);
+        LocalDate date = ChronoUnit.DAYS.addTo(EPOCH_DAY, (int) supplier.get());
+        Assert.assertEquals("date should be equal", expected, date);
+        break;
+      case TIME:
+        Assert.assertTrue("Should expect a LocalTime", expected instanceof LocalTime);
+        int milliseconds = (int) (((LocalTime) expected).toNanoOfDay() / 1000_000);
+        Assert.assertEquals("time millis should be equal", milliseconds, supplier.get());
+        break;
+      case TIMESTAMP:
+        if (((Types.TimestampType) type).shouldAdjustToUTC()) {
+          Assert.assertTrue("Should expect a OffsetDataTime", expected instanceof OffsetDateTime);
+          OffsetDateTime ts = (OffsetDateTime) expected;
+          Assert.assertEquals("OffsetDataTime should be equal", ts.toLocalDateTime(),
+              ((TimestampData) supplier.get()).toLocalDateTime());

Review comment:
       Why does this convert to `LocalDateTime` before validating? Can `TimestampData` be converted to `OffsetDateTime`?




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1266: Flink: update parquet reader with schema visitor

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1266:
URL: https://github.com/apache/iceberg/pull/1266#discussion_r469431025



##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java
##########
@@ -19,64 +19,678 @@
 
 package org.apache.iceberg.flink.data;
 
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
 import java.util.List;
-import org.apache.flink.types.Row;
+import java.util.Map;
+import org.apache.commons.lang3.ArrayUtils;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.MapData;
+import org.apache.flink.table.data.RawValueData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
 import org.apache.iceberg.Schema;
-import org.apache.iceberg.data.parquet.BaseParquetReaders;
 import org.apache.iceberg.parquet.ParquetValueReader;
 import org.apache.iceberg.parquet.ParquetValueReaders;
+import org.apache.iceberg.parquet.TypeWithSchemaVisitor;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
 import org.apache.iceberg.types.Types;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.io.api.Binary;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.LogicalTypeAnnotation.DecimalLogicalTypeAnnotation;
 import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
 import org.apache.parquet.schema.Type;
 
-public class FlinkParquetReaders extends BaseParquetReaders<Row> {
+class FlinkParquetReaders {
+  private FlinkParquetReaders() {
+  }
 
-  private static final FlinkParquetReaders INSTANCE = new FlinkParquetReaders();
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema, MessageType fileSchema) {
+    return buildReader(expectedSchema, fileSchema, ImmutableMap.of());
+  }
 
-  private FlinkParquetReaders() {
+  @SuppressWarnings("unchecked")
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema,
+                                                        MessageType fileSchema,
+                                                        Map<Integer, ?> idToConstant) {
+    return (ParquetValueReader<RowData>) TypeWithSchemaVisitor.visit(expectedSchema.asStruct(), fileSchema,
+        new ReadBuilder(fileSchema, idToConstant)
+    );
+  }
+
+  private static class ReadBuilder extends TypeWithSchemaVisitor<ParquetValueReader<?>> {
+    private final MessageType type;
+    private final Map<Integer, ?> idToConstant;
+
+    ReadBuilder(MessageType type, Map<Integer, ?> idToConstant) {
+      this.type = type;
+      this.idToConstant = idToConstant;
+    }
+
+    @Override
+    public ParquetValueReader<RowData> message(Types.StructType expected, MessageType message,
+                                               List<ParquetValueReader<?>> fieldReaders) {
+      return struct(expected, message.asGroupType(), fieldReaders);
+    }
+
+    @Override
+    public ParquetValueReader<RowData> struct(Types.StructType expected, GroupType struct,
+                                              List<ParquetValueReader<?>> fieldReaders) {
+      // match the expected struct's order
+      Map<Integer, ParquetValueReader<?>> readersById = Maps.newHashMap();
+      Map<Integer, Type> typesById = Maps.newHashMap();
+      List<Type> fields = struct.getFields();
+      for (int i = 0; i < fields.size(); i += 1) {
+        Type fieldType = fields.get(i);
+        int fieldD = type.getMaxDefinitionLevel(path(fieldType.getName())) - 1;
+        if (fieldType.getId() != null) {
+          int id = fieldType.getId().intValue();
+          readersById.put(id, ParquetValueReaders.option(fieldType, fieldD, fieldReaders.get(i)));
+          typesById.put(id, fieldType);
+        }
+      }
+
+      List<Types.NestedField> expectedFields = expected != null ?
+          expected.fields() : ImmutableList.of();
+      List<ParquetValueReader<?>> reorderedFields = Lists.newArrayListWithExpectedSize(
+          expectedFields.size());
+      List<Type> types = Lists.newArrayListWithExpectedSize(expectedFields.size());
+      for (Types.NestedField field : expectedFields) {
+        int id = field.fieldId();
+        if (idToConstant.containsKey(id)) {
+          // containsKey is used because the constant may be null
+          reorderedFields.add(ParquetValueReaders.constant(idToConstant.get(id)));
+          types.add(null);
+        } else {
+          ParquetValueReader<?> reader = readersById.get(id);
+          if (reader != null) {
+            reorderedFields.add(reader);
+            types.add(typesById.get(id));
+          } else {
+            reorderedFields.add(ParquetValueReaders.nulls());
+            types.add(null);
+          }
+        }
+      }
+
+      return new RowDataReader(types, reorderedFields);
+    }
+
+    @Override
+    public ParquetValueReader<?> list(Types.ListType expectedList, GroupType array,
+                                      ParquetValueReader<?> elementReader) {
+      GroupType repeated = array.getFields().get(0).asGroupType();
+      String[] repeatedPath = currentPath();
+
+      int repeatedD = type.getMaxDefinitionLevel(repeatedPath) - 1;
+      int repeatedR = type.getMaxRepetitionLevel(repeatedPath) - 1;
+
+      Type elementType = repeated.getType(0);
+      int elementD = type.getMaxDefinitionLevel(path(elementType.getName())) - 1;
+
+      return new ArrayReader<>(repeatedD, repeatedR, ParquetValueReaders.option(elementType, elementD, elementReader));
+    }
+
+    @Override
+    public ParquetValueReader<?> map(Types.MapType expectedMap, GroupType map,
+                                     ParquetValueReader<?> keyReader,
+                                     ParquetValueReader<?> valueReader) {
+      GroupType repeatedKeyValue = map.getFields().get(0).asGroupType();
+      String[] repeatedPath = currentPath();
+
+      int repeatedD = type.getMaxDefinitionLevel(repeatedPath) - 1;
+      int repeatedR = type.getMaxRepetitionLevel(repeatedPath) - 1;
+
+      Type keyType = repeatedKeyValue.getType(0);
+      int keyD = type.getMaxDefinitionLevel(path(keyType.getName())) - 1;
+      Type valueType = repeatedKeyValue.getType(1);
+      int valueD = type.getMaxDefinitionLevel(path(valueType.getName())) - 1;
+
+      return new MapReader<>(repeatedD, repeatedR,
+          ParquetValueReaders.option(keyType, keyD, keyReader),
+          ParquetValueReaders.option(valueType, valueD, valueReader));
+    }
+
+    @Override
+    public ParquetValueReader<?> primitive(org.apache.iceberg.types.Type.PrimitiveType expected,
+                                           PrimitiveType primitive) {
+      ColumnDescriptor desc = type.getColumnDescription(currentPath());
+
+      if (primitive.getOriginalType() != null) {
+        switch (primitive.getOriginalType()) {
+          case ENUM:
+          case JSON:
+          case UTF8:
+            return new StringReader(desc);
+          case INT_8:
+          case INT_16:
+          case INT_32:
+            if (expected != null && expected.typeId() == Types.LongType.get().typeId()) {
+              return new ParquetValueReaders.IntAsLongReader(desc);
+            } else {
+              return new ParquetValueReaders.UnboxedReader<>(desc);
+            }
+          case TIME_MICROS:
+            return new LossyMicrosToMillisTimeReader(desc);
+          case DATE:
+          case INT_64:
+            return new ParquetValueReaders.UnboxedReader<>(desc);
+          case TIMESTAMP_MICROS:

Review comment:
       Why not support `TIMESTAMP_MILLIS` as well?




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] chenjunjiedada commented on a change in pull request #1266: Flink: update parquet reader with schema visitor

Posted by GitBox <gi...@apache.org>.
chenjunjiedada commented on a change in pull request #1266:
URL: https://github.com/apache/iceberg/pull/1266#discussion_r469690608



##########
File path: flink/src/test/java/org/apache/iceberg/flink/data/TestHelpers.java
##########
@@ -0,0 +1,198 @@
+/*
+ * 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.iceberg.flink.data;
+
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.OffsetDateTime;
+import java.time.ZoneOffset;
+import java.time.temporal.ChronoUnit;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import java.util.function.Supplier;
+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.MapType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+
+public class TestHelpers {
+  private TestHelpers() {}
+
+  private static final OffsetDateTime EPOCH = Instant.ofEpochMilli(0L).atOffset(ZoneOffset.UTC);
+  private static final LocalDate EPOCH_DAY = EPOCH.toLocalDate();
+
+  public static void assertRowData(Type type, LogicalType rowType, Record expectedRecord, RowData actualRowData) {
+    if (expectedRecord == null && actualRowData == null) {
+      return;
+    }
+
+    Assert.assertTrue("expected Record and actual RowData should be both null or not null",
+        expectedRecord != null && actualRowData != null);
+
+    List<Type> types = Lists.newArrayList();
+    for (Types.NestedField field : type.asStructType().fields()) {
+      types.add(field.type());
+    }
+
+    for (int i = 0; i < types.size(); i += 1) {
+      if (expectedRecord.get(i) == null) {
+        Assert.assertTrue(actualRowData.isNullAt(i));
+        continue;
+      }
+
+      Object expected = expectedRecord.get(i);
+      LogicalType logicalType = ((RowType) rowType).getTypeAt(i);
+
+      final int fieldPos = i;
+      assertEquals(types.get(i), logicalType, expected,
+          () -> RowData.createFieldGetter(logicalType, fieldPos).getFieldOrNull(actualRowData));
+    }
+  }
+
+  private static void assertEquals(Type type, LogicalType logicalType, Object expected, Supplier<Object> supplier) {
+    switch (type.typeId()) {
+      case BOOLEAN:
+        Assert.assertEquals("boolean value should be equal", expected, supplier.get());
+        break;
+      case INTEGER:
+        Assert.assertEquals("int value should be equal", expected, supplier.get());
+        break;
+      case LONG:
+        Assert.assertEquals("long value should be equal", expected, supplier.get());
+        break;
+      case FLOAT:
+        Assert.assertEquals("float value should be equal", expected, supplier.get());
+        break;
+      case DOUBLE:
+        Assert.assertEquals("double value should be equal", expected, supplier.get());
+        break;
+      case STRING:
+        Assert.assertTrue("Should expect a CharSequence", expected instanceof CharSequence);
+        Assert.assertEquals("string should be equal",
+            String.valueOf(expected), supplier.get().toString());
+        break;
+      case DATE:
+        Assert.assertTrue("Should expect a Date", expected instanceof LocalDate);
+        LocalDate date = ChronoUnit.DAYS.addTo(EPOCH_DAY, (int) supplier.get());
+        Assert.assertEquals("date should be equal", expected, date);
+        break;
+      case TIME:
+        Assert.assertTrue("Should expect a LocalTime", expected instanceof LocalTime);
+        int milliseconds = (int) (((LocalTime) expected).toNanoOfDay() / 1000_000);
+        Assert.assertEquals("time millis should be equal", milliseconds, supplier.get());
+        break;
+      case TIMESTAMP:
+        if (((Types.TimestampType) type).shouldAdjustToUTC()) {
+          Assert.assertTrue("Should expect a OffsetDataTime", expected instanceof OffsetDateTime);
+          OffsetDateTime ts = (OffsetDateTime) expected;
+          Assert.assertEquals("OffsetDataTime should be equal", ts.toLocalDateTime(),
+              ((TimestampData) supplier.get()).toLocalDateTime());
+        } else {
+          Assert.assertTrue("Should expect a LocalDataTime", expected instanceof LocalDateTime);
+          LocalDateTime ts = (LocalDateTime) expected;
+          Assert.assertEquals("LocalDataTime should be equal", ts,
+              ((TimestampData) supplier.get()).toLocalDateTime());
+        }
+        break;
+      case BINARY:
+        Assert.assertTrue("Should expect a ByteBuffer", expected instanceof ByteBuffer);
+        Assert.assertArrayEquals("binary should be equal", ((ByteBuffer) expected).array(), (byte[]) supplier.get());

Review comment:
       Updated to compare ByteBuffer.

##########
File path: flink/src/test/java/org/apache/iceberg/flink/data/TestHelpers.java
##########
@@ -0,0 +1,198 @@
+/*
+ * 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.iceberg.flink.data;
+
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.OffsetDateTime;
+import java.time.ZoneOffset;
+import java.time.temporal.ChronoUnit;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import java.util.function.Supplier;
+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.MapType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+
+public class TestHelpers {
+  private TestHelpers() {}
+
+  private static final OffsetDateTime EPOCH = Instant.ofEpochMilli(0L).atOffset(ZoneOffset.UTC);
+  private static final LocalDate EPOCH_DAY = EPOCH.toLocalDate();
+
+  public static void assertRowData(Type type, LogicalType rowType, Record expectedRecord, RowData actualRowData) {
+    if (expectedRecord == null && actualRowData == null) {
+      return;
+    }
+
+    Assert.assertTrue("expected Record and actual RowData should be both null or not null",
+        expectedRecord != null && actualRowData != null);
+
+    List<Type> types = Lists.newArrayList();
+    for (Types.NestedField field : type.asStructType().fields()) {
+      types.add(field.type());
+    }
+
+    for (int i = 0; i < types.size(); i += 1) {
+      if (expectedRecord.get(i) == null) {
+        Assert.assertTrue(actualRowData.isNullAt(i));
+        continue;
+      }
+
+      Object expected = expectedRecord.get(i);
+      LogicalType logicalType = ((RowType) rowType).getTypeAt(i);
+
+      final int fieldPos = i;
+      assertEquals(types.get(i), logicalType, expected,
+          () -> RowData.createFieldGetter(logicalType, fieldPos).getFieldOrNull(actualRowData));
+    }
+  }
+
+  private static void assertEquals(Type type, LogicalType logicalType, Object expected, Supplier<Object> supplier) {
+    switch (type.typeId()) {
+      case BOOLEAN:
+        Assert.assertEquals("boolean value should be equal", expected, supplier.get());
+        break;
+      case INTEGER:
+        Assert.assertEquals("int value should be equal", expected, supplier.get());
+        break;
+      case LONG:
+        Assert.assertEquals("long value should be equal", expected, supplier.get());
+        break;
+      case FLOAT:
+        Assert.assertEquals("float value should be equal", expected, supplier.get());
+        break;
+      case DOUBLE:
+        Assert.assertEquals("double value should be equal", expected, supplier.get());
+        break;
+      case STRING:
+        Assert.assertTrue("Should expect a CharSequence", expected instanceof CharSequence);
+        Assert.assertEquals("string should be equal",
+            String.valueOf(expected), supplier.get().toString());
+        break;
+      case DATE:
+        Assert.assertTrue("Should expect a Date", expected instanceof LocalDate);
+        LocalDate date = ChronoUnit.DAYS.addTo(EPOCH_DAY, (int) supplier.get());
+        Assert.assertEquals("date should be equal", expected, date);
+        break;
+      case TIME:
+        Assert.assertTrue("Should expect a LocalTime", expected instanceof LocalTime);
+        int milliseconds = (int) (((LocalTime) expected).toNanoOfDay() / 1000_000);
+        Assert.assertEquals("time millis should be equal", milliseconds, supplier.get());
+        break;
+      case TIMESTAMP:
+        if (((Types.TimestampType) type).shouldAdjustToUTC()) {
+          Assert.assertTrue("Should expect a OffsetDataTime", expected instanceof OffsetDateTime);
+          OffsetDateTime ts = (OffsetDateTime) expected;
+          Assert.assertEquals("OffsetDataTime should be equal", ts.toLocalDateTime(),
+              ((TimestampData) supplier.get()).toLocalDateTime());
+        } else {
+          Assert.assertTrue("Should expect a LocalDataTime", expected instanceof LocalDateTime);
+          LocalDateTime ts = (LocalDateTime) expected;
+          Assert.assertEquals("LocalDataTime should be equal", ts,
+              ((TimestampData) supplier.get()).toLocalDateTime());
+        }
+        break;
+      case BINARY:
+        Assert.assertTrue("Should expect a ByteBuffer", expected instanceof ByteBuffer);
+        Assert.assertArrayEquals("binary should be equal", ((ByteBuffer) expected).array(), (byte[]) supplier.get());
+        break;
+      case DECIMAL:
+        Assert.assertTrue("Should expect a BigDecimal", expected instanceof BigDecimal);
+        BigDecimal bd = (BigDecimal) expected;
+        Assert.assertEquals("decimal value should be equal", bd,
+            ((DecimalData) supplier.get()).toBigDecimal());
+        break;
+      case LIST:
+        Assert.assertTrue("Should expect a Collection", expected instanceof Collection);
+        Collection<?> expectedArrayData = (Collection<?>) expected;
+        ArrayData actualArrayData = (ArrayData) supplier.get();
+        LogicalType elementType = ((ArrayType) logicalType).getElementType();
+        Assert.assertEquals("array length should be equal", expectedArrayData.size(), actualArrayData.size());
+        assertArrayValues(type.asListType().elementType(), elementType, expectedArrayData, actualArrayData);

Review comment:
       Done.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] chenjunjiedada commented on pull request #1266: Flink: update parquet reader with schema visitor

Posted by GitBox <gi...@apache.org>.
chenjunjiedada commented on pull request #1266:
URL: https://github.com/apache/iceberg/pull/1266#issuecomment-670278105


   @rdblue @openinx , Would you please help to take another look? This is ready for another round.


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] chenjunjiedada commented on pull request #1266: Flink: update parquet reader with schema visitor

Posted by GitBox <gi...@apache.org>.
chenjunjiedada commented on pull request #1266:
URL: https://github.com/apache/iceberg/pull/1266#issuecomment-666892356


   @openinx , Would you mind to take another look on this? 


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] chenjunjiedada commented on a change in pull request #1266: Flink: update parquet reader with schema visitor

Posted by GitBox <gi...@apache.org>.
chenjunjiedada commented on a change in pull request #1266:
URL: https://github.com/apache/iceberg/pull/1266#discussion_r463556929



##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java
##########
@@ -19,64 +19,719 @@
 
 package org.apache.iceberg.flink.data;
 
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.time.Instant;
 import java.util.List;
-import org.apache.flink.types.Row;
+import java.util.Map;
+import org.apache.commons.lang3.ArrayUtils;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.MapData;
+import org.apache.flink.table.data.RawValueData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
 import org.apache.iceberg.Schema;
-import org.apache.iceberg.data.parquet.BaseParquetReaders;
+import org.apache.iceberg.parquet.ParquetSchemaUtil;
 import org.apache.iceberg.parquet.ParquetValueReader;
 import org.apache.iceberg.parquet.ParquetValueReaders;
+import org.apache.iceberg.parquet.TypeWithSchemaVisitor;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
 import org.apache.iceberg.types.Types;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.io.api.Binary;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.LogicalTypeAnnotation.DecimalLogicalTypeAnnotation;
 import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
 import org.apache.parquet.schema.Type;
 
-public class FlinkParquetReaders extends BaseParquetReaders<Row> {
+class FlinkParquetReaders {
+  private FlinkParquetReaders() {
+  }
 
-  private static final FlinkParquetReaders INSTANCE = new FlinkParquetReaders();
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema, MessageType fileSchema) {
+    return buildReader(expectedSchema, fileSchema, ImmutableMap.of());
+  }
 
-  private FlinkParquetReaders() {
+  @SuppressWarnings("unchecked")
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema,
+                                                        MessageType fileSchema,
+                                                        Map<Integer, ?> idToConstant) {
+    ReadBuilder builder = new ReadBuilder(fileSchema, idToConstant);
+    if (ParquetSchemaUtil.hasIds(fileSchema)) {
+      return (ParquetValueReader<RowData>)
+          TypeWithSchemaVisitor.visit(expectedSchema.asStruct(), fileSchema, builder);
+    } else {
+      return (ParquetValueReader<RowData>)
+          TypeWithSchemaVisitor.visit(expectedSchema.asStruct(), fileSchema,
+              new FallbackReadBuilder(builder));
+    }
+  }
+
+  private static class FallbackReadBuilder extends TypeWithSchemaVisitor<ParquetValueReader<?>> {
+    private MessageType type;
+    private final TypeWithSchemaVisitor<ParquetValueReader<?>> builder;

Review comment:
       You are right.  Let me remove `FallbackReaderBuilder` at first since we don't use it. Will add it back when this is ready.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] chenjunjiedada commented on pull request #1266: Flink: update parquet reader with schema visitor

Posted by GitBox <gi...@apache.org>.
chenjunjiedada commented on pull request #1266:
URL: https://github.com/apache/iceberg/pull/1266#issuecomment-673261016


   @rdblue , I addressed all your comments. Please help to take another look at your convenience.


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] chenjunjiedada commented on a change in pull request #1266: Flink: update parquet reader with schema visitor

Posted by GitBox <gi...@apache.org>.
chenjunjiedada commented on a change in pull request #1266:
URL: https://github.com/apache/iceberg/pull/1266#discussion_r465087087



##########
File path: flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetReader.java
##########
@@ -35,51 +36,49 @@
 
 import static org.apache.iceberg.flink.data.RandomData.COMPLEX_SCHEMA;
 
-public class TestFlinkParquetReaderWriter {
+public class TestFlinkParquetReader {

Review comment:
       Make sense to me and updated.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] chenjunjiedada commented on a change in pull request #1266: Flink: update parquet reader with schema visitor

Posted by GitBox <gi...@apache.org>.
chenjunjiedada commented on a change in pull request #1266:
URL: https://github.com/apache/iceberg/pull/1266#discussion_r465498843



##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java
##########
@@ -19,64 +19,719 @@
 
 package org.apache.iceberg.flink.data;
 
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.time.Instant;
 import java.util.List;
-import org.apache.flink.types.Row;
+import java.util.Map;
+import org.apache.commons.lang3.ArrayUtils;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.MapData;
+import org.apache.flink.table.data.RawValueData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
 import org.apache.iceberg.Schema;
-import org.apache.iceberg.data.parquet.BaseParquetReaders;
+import org.apache.iceberg.parquet.ParquetSchemaUtil;
 import org.apache.iceberg.parquet.ParquetValueReader;
 import org.apache.iceberg.parquet.ParquetValueReaders;
+import org.apache.iceberg.parquet.TypeWithSchemaVisitor;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
 import org.apache.iceberg.types.Types;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.io.api.Binary;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.LogicalTypeAnnotation.DecimalLogicalTypeAnnotation;
 import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
 import org.apache.parquet.schema.Type;
 
-public class FlinkParquetReaders extends BaseParquetReaders<Row> {
+class FlinkParquetReaders {
+  private FlinkParquetReaders() {
+  }
 
-  private static final FlinkParquetReaders INSTANCE = new FlinkParquetReaders();
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema, MessageType fileSchema) {
+    return buildReader(expectedSchema, fileSchema, ImmutableMap.of());
+  }
 
-  private FlinkParquetReaders() {
+  @SuppressWarnings("unchecked")
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema,
+                                                        MessageType fileSchema,
+                                                        Map<Integer, ?> idToConstant) {
+    ReadBuilder builder = new ReadBuilder(fileSchema, idToConstant);
+    if (ParquetSchemaUtil.hasIds(fileSchema)) {
+      return (ParquetValueReader<RowData>)
+          TypeWithSchemaVisitor.visit(expectedSchema.asStruct(), fileSchema, builder);
+    } else {
+      return (ParquetValueReader<RowData>)
+          TypeWithSchemaVisitor.visit(expectedSchema.asStruct(), fileSchema,
+              new FallbackReadBuilder(builder));
+    }
+  }
+
+  private static class FallbackReadBuilder extends TypeWithSchemaVisitor<ParquetValueReader<?>> {
+    private MessageType type;
+    private final TypeWithSchemaVisitor<ParquetValueReader<?>> builder;
+
+    FallbackReadBuilder(TypeWithSchemaVisitor<ParquetValueReader<?>> builder) {
+      this.builder = builder;
+    }
+
+    @Override
+    public ParquetValueReader<?> message(Types.StructType expected, MessageType message,
+                                         List<ParquetValueReader<?>> fieldReaders) {
+      // the top level matches by ID, but the remaining IDs are missing
+      this.type = message;
+      return builder.struct(expected, message, fieldReaders);
+    }
+
+    @Override
+    public ParquetValueReader<?> struct(Types.StructType ignored, GroupType struct,
+                                        List<ParquetValueReader<?>> fieldReaders) {
+      // the expected struct is ignored because nested fields are never found when the IDs are missing
+      List<ParquetValueReader<?>> newFields = Lists.newArrayListWithExpectedSize(
+          fieldReaders.size());
+      List<Type> types = Lists.newArrayListWithExpectedSize(fieldReaders.size());
+      List<Type> fields = struct.getFields();
+      for (int i = 0; i < fields.size(); i += 1) {
+        Type fieldType = fields.get(i);
+        int fieldD = type.getMaxDefinitionLevel(path(fieldType.getName())) - 1;
+        newFields.add(ParquetValueReaders.option(fieldType, fieldD, fieldReaders.get(i)));
+        types.add(fieldType);
+      }
+
+      return new RowDataReader(types, newFields);
+    }
+  }
+
+  private static class ReadBuilder extends TypeWithSchemaVisitor<ParquetValueReader<?>> {

Review comment:
       Thanks @JingsongLi for your input, will take a look again when we start that task.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] chenjunjiedada commented on a change in pull request #1266: Flink: update parquet reader with schema visitor

Posted by GitBox <gi...@apache.org>.
chenjunjiedada commented on a change in pull request #1266:
URL: https://github.com/apache/iceberg/pull/1266#discussion_r464982270



##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/ParquetWithFlinkSchemaVisitor.java
##########
@@ -0,0 +1,199 @@
+/*
+ * 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.iceberg.flink.data;
+
+import java.util.Deque;
+import java.util.List;
+import org.apache.flink.table.types.logical.ArrayType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.MapType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.table.types.logical.RowType.RowField;
+import org.apache.iceberg.avro.AvroSchemaUtil;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.OriginalType;
+import org.apache.parquet.schema.PrimitiveType;
+import org.apache.parquet.schema.Type;
+
+public class ParquetWithFlinkSchemaVisitor<T> {

Review comment:
       Yes, let me remove this.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1266: Flink: update parquet reader with schema visitor

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1266:
URL: https://github.com/apache/iceberg/pull/1266#discussion_r469430027



##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java
##########
@@ -19,64 +19,678 @@
 
 package org.apache.iceberg.flink.data;
 
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
 import java.util.List;
-import org.apache.flink.types.Row;
+import java.util.Map;
+import org.apache.commons.lang3.ArrayUtils;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.MapData;
+import org.apache.flink.table.data.RawValueData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
 import org.apache.iceberg.Schema;
-import org.apache.iceberg.data.parquet.BaseParquetReaders;
 import org.apache.iceberg.parquet.ParquetValueReader;
 import org.apache.iceberg.parquet.ParquetValueReaders;
+import org.apache.iceberg.parquet.TypeWithSchemaVisitor;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
 import org.apache.iceberg.types.Types;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.io.api.Binary;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.LogicalTypeAnnotation.DecimalLogicalTypeAnnotation;
 import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
 import org.apache.parquet.schema.Type;
 
-public class FlinkParquetReaders extends BaseParquetReaders<Row> {
+class FlinkParquetReaders {
+  private FlinkParquetReaders() {
+  }
 
-  private static final FlinkParquetReaders INSTANCE = new FlinkParquetReaders();
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema, MessageType fileSchema) {
+    return buildReader(expectedSchema, fileSchema, ImmutableMap.of());
+  }
 
-  private FlinkParquetReaders() {
+  @SuppressWarnings("unchecked")
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema,
+                                                        MessageType fileSchema,
+                                                        Map<Integer, ?> idToConstant) {
+    return (ParquetValueReader<RowData>) TypeWithSchemaVisitor.visit(expectedSchema.asStruct(), fileSchema,
+        new ReadBuilder(fileSchema, idToConstant)
+    );
+  }
+
+  private static class ReadBuilder extends TypeWithSchemaVisitor<ParquetValueReader<?>> {
+    private final MessageType type;
+    private final Map<Integer, ?> idToConstant;
+
+    ReadBuilder(MessageType type, Map<Integer, ?> idToConstant) {
+      this.type = type;
+      this.idToConstant = idToConstant;
+    }
+
+    @Override
+    public ParquetValueReader<RowData> message(Types.StructType expected, MessageType message,
+                                               List<ParquetValueReader<?>> fieldReaders) {
+      return struct(expected, message.asGroupType(), fieldReaders);
+    }
+
+    @Override
+    public ParquetValueReader<RowData> struct(Types.StructType expected, GroupType struct,
+                                              List<ParquetValueReader<?>> fieldReaders) {
+      // match the expected struct's order
+      Map<Integer, ParquetValueReader<?>> readersById = Maps.newHashMap();
+      Map<Integer, Type> typesById = Maps.newHashMap();
+      List<Type> fields = struct.getFields();
+      for (int i = 0; i < fields.size(); i += 1) {
+        Type fieldType = fields.get(i);
+        int fieldD = type.getMaxDefinitionLevel(path(fieldType.getName())) - 1;
+        if (fieldType.getId() != null) {
+          int id = fieldType.getId().intValue();
+          readersById.put(id, ParquetValueReaders.option(fieldType, fieldD, fieldReaders.get(i)));
+          typesById.put(id, fieldType);
+        }
+      }
+
+      List<Types.NestedField> expectedFields = expected != null ?
+          expected.fields() : ImmutableList.of();
+      List<ParquetValueReader<?>> reorderedFields = Lists.newArrayListWithExpectedSize(
+          expectedFields.size());
+      List<Type> types = Lists.newArrayListWithExpectedSize(expectedFields.size());
+      for (Types.NestedField field : expectedFields) {
+        int id = field.fieldId();
+        if (idToConstant.containsKey(id)) {
+          // containsKey is used because the constant may be null
+          reorderedFields.add(ParquetValueReaders.constant(idToConstant.get(id)));
+          types.add(null);
+        } else {
+          ParquetValueReader<?> reader = readersById.get(id);
+          if (reader != null) {
+            reorderedFields.add(reader);
+            types.add(typesById.get(id));
+          } else {
+            reorderedFields.add(ParquetValueReaders.nulls());
+            types.add(null);
+          }
+        }
+      }
+
+      return new RowDataReader(types, reorderedFields);
+    }
+
+    @Override
+    public ParquetValueReader<?> list(Types.ListType expectedList, GroupType array,
+                                      ParquetValueReader<?> elementReader) {
+      GroupType repeated = array.getFields().get(0).asGroupType();
+      String[] repeatedPath = currentPath();
+
+      int repeatedD = type.getMaxDefinitionLevel(repeatedPath) - 1;
+      int repeatedR = type.getMaxRepetitionLevel(repeatedPath) - 1;
+
+      Type elementType = repeated.getType(0);
+      int elementD = type.getMaxDefinitionLevel(path(elementType.getName())) - 1;
+
+      return new ArrayReader<>(repeatedD, repeatedR, ParquetValueReaders.option(elementType, elementD, elementReader));
+    }
+
+    @Override
+    public ParquetValueReader<?> map(Types.MapType expectedMap, GroupType map,
+                                     ParquetValueReader<?> keyReader,
+                                     ParquetValueReader<?> valueReader) {
+      GroupType repeatedKeyValue = map.getFields().get(0).asGroupType();
+      String[] repeatedPath = currentPath();
+
+      int repeatedD = type.getMaxDefinitionLevel(repeatedPath) - 1;
+      int repeatedR = type.getMaxRepetitionLevel(repeatedPath) - 1;
+
+      Type keyType = repeatedKeyValue.getType(0);
+      int keyD = type.getMaxDefinitionLevel(path(keyType.getName())) - 1;
+      Type valueType = repeatedKeyValue.getType(1);
+      int valueD = type.getMaxDefinitionLevel(path(valueType.getName())) - 1;
+
+      return new MapReader<>(repeatedD, repeatedR,
+          ParquetValueReaders.option(keyType, keyD, keyReader),
+          ParquetValueReaders.option(valueType, valueD, valueReader));
+    }
+
+    @Override
+    public ParquetValueReader<?> primitive(org.apache.iceberg.types.Type.PrimitiveType expected,
+                                           PrimitiveType primitive) {
+      ColumnDescriptor desc = type.getColumnDescription(currentPath());
+
+      if (primitive.getOriginalType() != null) {
+        switch (primitive.getOriginalType()) {
+          case ENUM:
+          case JSON:
+          case UTF8:
+            return new StringReader(desc);
+          case INT_8:
+          case INT_16:
+          case INT_32:
+            if (expected != null && expected.typeId() == Types.LongType.get().typeId()) {
+              return new ParquetValueReaders.IntAsLongReader(desc);
+            } else {
+              return new ParquetValueReaders.UnboxedReader<>(desc);
+            }
+          case TIME_MICROS:
+            return new LossyMicrosToMillisTimeReader(desc);
+          case DATE:
+          case INT_64:
+            return new ParquetValueReaders.UnboxedReader<>(desc);
+          case TIMESTAMP_MICROS:
+            return new TimestampMicrosReader(desc);

Review comment:
       Shouldn't this check whether the timestamp is a `TIMESTAMP WITH ZONE` or `TIMESTAMP WITHOUT ZONE` and return the correct reader? Spark doesn't do this because it doesn't support timestamp without zone, so we know it is always a timestamptz.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] chenjunjiedada commented on pull request #1266: Flink: update parquet reader with schema visitor

Posted by GitBox <gi...@apache.org>.
chenjunjiedada commented on pull request #1266:
URL: https://github.com/apache/iceberg/pull/1266#issuecomment-671650994


   @rdblue, would you please help to take anther look? I will rebase the writer side on this.


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] chenjunjiedada commented on a change in pull request #1266: Flink: update parquet reader with schema visitor

Posted by GitBox <gi...@apache.org>.
chenjunjiedada commented on a change in pull request #1266:
URL: https://github.com/apache/iceberg/pull/1266#discussion_r464453352



##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java
##########
@@ -19,64 +19,719 @@
 
 package org.apache.iceberg.flink.data;
 
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.time.Instant;
 import java.util.List;
-import org.apache.flink.types.Row;
+import java.util.Map;
+import org.apache.commons.lang3.ArrayUtils;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.MapData;
+import org.apache.flink.table.data.RawValueData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
 import org.apache.iceberg.Schema;
-import org.apache.iceberg.data.parquet.BaseParquetReaders;
+import org.apache.iceberg.parquet.ParquetSchemaUtil;
 import org.apache.iceberg.parquet.ParquetValueReader;
 import org.apache.iceberg.parquet.ParquetValueReaders;
+import org.apache.iceberg.parquet.TypeWithSchemaVisitor;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
 import org.apache.iceberg.types.Types;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.io.api.Binary;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.LogicalTypeAnnotation.DecimalLogicalTypeAnnotation;
 import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
 import org.apache.parquet.schema.Type;
 
-public class FlinkParquetReaders extends BaseParquetReaders<Row> {
+class FlinkParquetReaders {
+  private FlinkParquetReaders() {
+  }
 
-  private static final FlinkParquetReaders INSTANCE = new FlinkParquetReaders();
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema, MessageType fileSchema) {
+    return buildReader(expectedSchema, fileSchema, ImmutableMap.of());
+  }
 
-  private FlinkParquetReaders() {
+  @SuppressWarnings("unchecked")
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema,
+                                                        MessageType fileSchema,
+                                                        Map<Integer, ?> idToConstant) {
+    ReadBuilder builder = new ReadBuilder(fileSchema, idToConstant);
+    if (ParquetSchemaUtil.hasIds(fileSchema)) {
+      return (ParquetValueReader<RowData>)
+          TypeWithSchemaVisitor.visit(expectedSchema.asStruct(), fileSchema, builder);
+    } else {
+      return (ParquetValueReader<RowData>)
+          TypeWithSchemaVisitor.visit(expectedSchema.asStruct(), fileSchema,
+              new FallbackReadBuilder(builder));
+    }
+  }
+
+  private static class FallbackReadBuilder extends TypeWithSchemaVisitor<ParquetValueReader<?>> {
+    private MessageType type;
+    private final TypeWithSchemaVisitor<ParquetValueReader<?>> builder;
+
+    FallbackReadBuilder(TypeWithSchemaVisitor<ParquetValueReader<?>> builder) {
+      this.builder = builder;
+    }
+
+    @Override
+    public ParquetValueReader<?> message(Types.StructType expected, MessageType message,
+                                         List<ParquetValueReader<?>> fieldReaders) {
+      // the top level matches by ID, but the remaining IDs are missing
+      this.type = message;
+      return builder.struct(expected, message, fieldReaders);
+    }
+
+    @Override
+    public ParquetValueReader<?> struct(Types.StructType ignored, GroupType struct,
+                                        List<ParquetValueReader<?>> fieldReaders) {
+      // the expected struct is ignored because nested fields are never found when the IDs are missing
+      List<ParquetValueReader<?>> newFields = Lists.newArrayListWithExpectedSize(
+          fieldReaders.size());
+      List<Type> types = Lists.newArrayListWithExpectedSize(fieldReaders.size());
+      List<Type> fields = struct.getFields();
+      for (int i = 0; i < fields.size(); i += 1) {
+        Type fieldType = fields.get(i);
+        int fieldD = type.getMaxDefinitionLevel(path(fieldType.getName())) - 1;
+        newFields.add(ParquetValueReaders.option(fieldType, fieldD, fieldReaders.get(i)));
+        types.add(fieldType);
+      }
+
+      return new RowDataReader(types, newFields);
+    }
+  }
+
+  private static class ReadBuilder extends TypeWithSchemaVisitor<ParquetValueReader<?>> {

Review comment:
       Sound good to me, we can take a look at `WriteBuilder` as well after all these readers and writers get in.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1266: Flink: update parquet reader with schema visitor

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1266:
URL: https://github.com/apache/iceberg/pull/1266#discussion_r469430920



##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java
##########
@@ -19,64 +19,678 @@
 
 package org.apache.iceberg.flink.data;
 
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
 import java.util.List;
-import org.apache.flink.types.Row;
+import java.util.Map;
+import org.apache.commons.lang3.ArrayUtils;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.MapData;
+import org.apache.flink.table.data.RawValueData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
 import org.apache.iceberg.Schema;
-import org.apache.iceberg.data.parquet.BaseParquetReaders;
 import org.apache.iceberg.parquet.ParquetValueReader;
 import org.apache.iceberg.parquet.ParquetValueReaders;
+import org.apache.iceberg.parquet.TypeWithSchemaVisitor;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
 import org.apache.iceberg.types.Types;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.io.api.Binary;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.LogicalTypeAnnotation.DecimalLogicalTypeAnnotation;
 import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
 import org.apache.parquet.schema.Type;
 
-public class FlinkParquetReaders extends BaseParquetReaders<Row> {
+class FlinkParquetReaders {
+  private FlinkParquetReaders() {
+  }
 
-  private static final FlinkParquetReaders INSTANCE = new FlinkParquetReaders();
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema, MessageType fileSchema) {
+    return buildReader(expectedSchema, fileSchema, ImmutableMap.of());
+  }
 
-  private FlinkParquetReaders() {
+  @SuppressWarnings("unchecked")
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema,
+                                                        MessageType fileSchema,
+                                                        Map<Integer, ?> idToConstant) {
+    return (ParquetValueReader<RowData>) TypeWithSchemaVisitor.visit(expectedSchema.asStruct(), fileSchema,
+        new ReadBuilder(fileSchema, idToConstant)
+    );
+  }
+
+  private static class ReadBuilder extends TypeWithSchemaVisitor<ParquetValueReader<?>> {
+    private final MessageType type;
+    private final Map<Integer, ?> idToConstant;
+
+    ReadBuilder(MessageType type, Map<Integer, ?> idToConstant) {
+      this.type = type;
+      this.idToConstant = idToConstant;
+    }
+
+    @Override
+    public ParquetValueReader<RowData> message(Types.StructType expected, MessageType message,
+                                               List<ParquetValueReader<?>> fieldReaders) {
+      return struct(expected, message.asGroupType(), fieldReaders);
+    }
+
+    @Override
+    public ParquetValueReader<RowData> struct(Types.StructType expected, GroupType struct,
+                                              List<ParquetValueReader<?>> fieldReaders) {
+      // match the expected struct's order
+      Map<Integer, ParquetValueReader<?>> readersById = Maps.newHashMap();
+      Map<Integer, Type> typesById = Maps.newHashMap();
+      List<Type> fields = struct.getFields();
+      for (int i = 0; i < fields.size(); i += 1) {
+        Type fieldType = fields.get(i);
+        int fieldD = type.getMaxDefinitionLevel(path(fieldType.getName())) - 1;
+        if (fieldType.getId() != null) {
+          int id = fieldType.getId().intValue();
+          readersById.put(id, ParquetValueReaders.option(fieldType, fieldD, fieldReaders.get(i)));
+          typesById.put(id, fieldType);
+        }
+      }
+
+      List<Types.NestedField> expectedFields = expected != null ?
+          expected.fields() : ImmutableList.of();
+      List<ParquetValueReader<?>> reorderedFields = Lists.newArrayListWithExpectedSize(
+          expectedFields.size());
+      List<Type> types = Lists.newArrayListWithExpectedSize(expectedFields.size());
+      for (Types.NestedField field : expectedFields) {
+        int id = field.fieldId();
+        if (idToConstant.containsKey(id)) {
+          // containsKey is used because the constant may be null
+          reorderedFields.add(ParquetValueReaders.constant(idToConstant.get(id)));
+          types.add(null);
+        } else {
+          ParquetValueReader<?> reader = readersById.get(id);
+          if (reader != null) {
+            reorderedFields.add(reader);
+            types.add(typesById.get(id));
+          } else {
+            reorderedFields.add(ParquetValueReaders.nulls());
+            types.add(null);
+          }
+        }
+      }
+
+      return new RowDataReader(types, reorderedFields);
+    }
+
+    @Override
+    public ParquetValueReader<?> list(Types.ListType expectedList, GroupType array,
+                                      ParquetValueReader<?> elementReader) {
+      GroupType repeated = array.getFields().get(0).asGroupType();
+      String[] repeatedPath = currentPath();
+
+      int repeatedD = type.getMaxDefinitionLevel(repeatedPath) - 1;
+      int repeatedR = type.getMaxRepetitionLevel(repeatedPath) - 1;
+
+      Type elementType = repeated.getType(0);
+      int elementD = type.getMaxDefinitionLevel(path(elementType.getName())) - 1;
+
+      return new ArrayReader<>(repeatedD, repeatedR, ParquetValueReaders.option(elementType, elementD, elementReader));
+    }
+
+    @Override
+    public ParquetValueReader<?> map(Types.MapType expectedMap, GroupType map,
+                                     ParquetValueReader<?> keyReader,
+                                     ParquetValueReader<?> valueReader) {
+      GroupType repeatedKeyValue = map.getFields().get(0).asGroupType();
+      String[] repeatedPath = currentPath();
+
+      int repeatedD = type.getMaxDefinitionLevel(repeatedPath) - 1;
+      int repeatedR = type.getMaxRepetitionLevel(repeatedPath) - 1;
+
+      Type keyType = repeatedKeyValue.getType(0);
+      int keyD = type.getMaxDefinitionLevel(path(keyType.getName())) - 1;
+      Type valueType = repeatedKeyValue.getType(1);
+      int valueD = type.getMaxDefinitionLevel(path(valueType.getName())) - 1;
+
+      return new MapReader<>(repeatedD, repeatedR,
+          ParquetValueReaders.option(keyType, keyD, keyReader),
+          ParquetValueReaders.option(valueType, valueD, valueReader));
+    }
+
+    @Override
+    public ParquetValueReader<?> primitive(org.apache.iceberg.types.Type.PrimitiveType expected,
+                                           PrimitiveType primitive) {
+      ColumnDescriptor desc = type.getColumnDescription(currentPath());
+
+      if (primitive.getOriginalType() != null) {
+        switch (primitive.getOriginalType()) {
+          case ENUM:
+          case JSON:
+          case UTF8:
+            return new StringReader(desc);
+          case INT_8:
+          case INT_16:
+          case INT_32:
+            if (expected != null && expected.typeId() == Types.LongType.get().typeId()) {
+              return new ParquetValueReaders.IntAsLongReader(desc);
+            } else {
+              return new ParquetValueReaders.UnboxedReader<>(desc);
+            }
+          case TIME_MICROS:

Review comment:
       What about `TIME_MILLIS`? It won't be written by Iceberg, but it is usually good to be able to read from imported files.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] chenjunjiedada commented on a change in pull request #1266: Flink: update parquet reader with schema visitor

Posted by GitBox <gi...@apache.org>.
chenjunjiedada commented on a change in pull request #1266:
URL: https://github.com/apache/iceberg/pull/1266#discussion_r464997844



##########
File path: flink/src/test/java/org/apache/iceberg/flink/data/TestHelpers.java
##########
@@ -0,0 +1,271 @@
+/*
+ * 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.iceberg.flink.data;
+
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.OffsetDateTime;
+import java.time.ZoneOffset;
+import java.time.temporal.ChronoUnit;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.RowData;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+
+public class TestHelpers {
+  private TestHelpers() {}
+
+  private static final OffsetDateTime EPOCH = Instant.ofEpochMilli(0L).atOffset(ZoneOffset.UTC);
+  private static final LocalDate EPOCH_DAY = EPOCH.toLocalDate();
+
+  public static void assertRowData(Type type, Record expected, RowData actual) {
+    if (expected == null && actual == null) {
+      return;
+    }
+
+    List<Type> types = new ArrayList<>();
+    for (Types.NestedField field : type.asStructType().fields()) {
+      types.add(field.type());
+    }
+
+    for (int i = 0; i < types.size(); i += 1) {
+      if (expected.get(i) == null) {
+        Assert.assertTrue(actual.isNullAt(i));
+        continue;
+      }
+      Type.TypeID typeId = types.get(i).typeId();
+      Object value = expected.get(i);
+      switch (typeId) {
+        case BOOLEAN:
+          Assert.assertEquals("boolean value should be equal", value, actual.getBoolean(i));
+          break;
+        case INTEGER:
+          Assert.assertEquals("int value should be equal", value, actual.getInt(i));
+          break;
+        case LONG:
+          Assert.assertEquals("long value should be equal", value, actual.getLong(i));
+          break;
+        case FLOAT:
+          Assert.assertEquals("float value should be equal", value, actual.getFloat(i));
+          break;
+        case DOUBLE:
+          Assert.assertEquals("double should be equal", value, actual.getDouble(i));
+          break;
+        case STRING:
+          Assert.assertTrue("Should expect a CharSequence", value instanceof CharSequence);
+          Assert.assertEquals("string should be equal", String.valueOf(value), actual.getString(i).toString());
+          break;
+        case DATE:
+          Assert.assertTrue("Should expect a Date", value instanceof LocalDate);
+          LocalDate date = ChronoUnit.DAYS.addTo(EPOCH_DAY, actual.getInt(i));
+          Assert.assertEquals("date should be equal", value, date);
+          break;
+        case TIME:
+          Assert.assertTrue("Should expect a LocalTime", value instanceof LocalTime);
+          int milliseconds = (int) (((LocalTime) value).toNanoOfDay() / 1000_000);
+          Assert.assertEquals("time millis should be equal", milliseconds, actual.getInt(i));
+          break;
+        case TIMESTAMP:
+          if (((Types.TimestampType) type.asPrimitiveType()).shouldAdjustToUTC()) {
+            Assert.assertTrue("Should expect a OffsetDataTime", value instanceof OffsetDateTime);
+            OffsetDateTime ts = (OffsetDateTime) value;
+            Assert.assertEquals("OffsetDataTime should be equal", ts.toLocalDateTime(),
+                actual.getTimestamp(i, 6).toLocalDateTime());
+          } else {
+            Assert.assertTrue("Should expect a LocalDataTime", value instanceof LocalDateTime);
+            LocalDateTime ts = (LocalDateTime) value;
+            Assert.assertEquals("LocalDataTime should be equal", ts,
+                actual.getTimestamp(i, 6).toLocalDateTime());
+          }
+          break;
+        case FIXED:
+          Assert.assertTrue("Should expect byte[]", value instanceof byte[]);
+          Assert.assertArrayEquals("binary should be equal", (byte[]) value, actual.getBinary(i));
+          break;
+        case BINARY:
+          Assert.assertTrue("Should expect a ByteBuffer", value instanceof ByteBuffer);
+          Assert.assertArrayEquals("binary should be equal", ((ByteBuffer) value).array(), actual.getBinary(i));
+          break;
+        case DECIMAL:

Review comment:
       I tried to extract to one common method with an interface like `assertData(Type type, Record expected, Object actual)` so that we can determine `actual` via `instanceof`, but we still have to call the getter method with casting, such as `((RowData)actual).getInt(i)` and `((ArrayData)actual).getInt(i)`, so the code still looks a bit duplicated as current. Does that make sense to you?




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #1266: Flink: update parquet reader with schema visitor

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1266:
URL: https://github.com/apache/iceberg/pull/1266#discussion_r463477035



##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java
##########
@@ -19,64 +19,719 @@
 
 package org.apache.iceberg.flink.data;
 
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.time.Instant;
 import java.util.List;
-import org.apache.flink.types.Row;
+import java.util.Map;
+import org.apache.commons.lang3.ArrayUtils;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.MapData;
+import org.apache.flink.table.data.RawValueData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
 import org.apache.iceberg.Schema;
-import org.apache.iceberg.data.parquet.BaseParquetReaders;
+import org.apache.iceberg.parquet.ParquetSchemaUtil;
 import org.apache.iceberg.parquet.ParquetValueReader;
 import org.apache.iceberg.parquet.ParquetValueReaders;
+import org.apache.iceberg.parquet.TypeWithSchemaVisitor;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
 import org.apache.iceberg.types.Types;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.io.api.Binary;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.LogicalTypeAnnotation.DecimalLogicalTypeAnnotation;
 import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
 import org.apache.parquet.schema.Type;
 
-public class FlinkParquetReaders extends BaseParquetReaders<Row> {
+class FlinkParquetReaders {
+  private FlinkParquetReaders() {
+  }
 
-  private static final FlinkParquetReaders INSTANCE = new FlinkParquetReaders();
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema, MessageType fileSchema) {
+    return buildReader(expectedSchema, fileSchema, ImmutableMap.of());
+  }
 
-  private FlinkParquetReaders() {
+  @SuppressWarnings("unchecked")
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema,
+                                                        MessageType fileSchema,
+                                                        Map<Integer, ?> idToConstant) {
+    ReadBuilder builder = new ReadBuilder(fileSchema, idToConstant);
+    if (ParquetSchemaUtil.hasIds(fileSchema)) {
+      return (ParquetValueReader<RowData>)
+          TypeWithSchemaVisitor.visit(expectedSchema.asStruct(), fileSchema, builder);
+    } else {
+      return (ParquetValueReader<RowData>)
+          TypeWithSchemaVisitor.visit(expectedSchema.asStruct(), fileSchema,
+              new FallbackReadBuilder(builder));
+    }
+  }
+
+  private static class FallbackReadBuilder extends TypeWithSchemaVisitor<ParquetValueReader<?>> {
+    private MessageType type;
+    private final TypeWithSchemaVisitor<ParquetValueReader<?>> builder;
+
+    FallbackReadBuilder(TypeWithSchemaVisitor<ParquetValueReader<?>> builder) {
+      this.builder = builder;
+    }
+
+    @Override
+    public ParquetValueReader<?> message(Types.StructType expected, MessageType message,
+                                         List<ParquetValueReader<?>> fieldReaders) {
+      // the top level matches by ID, but the remaining IDs are missing
+      this.type = message;
+      return builder.struct(expected, message, fieldReaders);
+    }
+
+    @Override
+    public ParquetValueReader<?> struct(Types.StructType ignored, GroupType struct,

Review comment:
       Here seems we could return the determinate parameter type `RowData`.   Can change it to `ParquetValueReader<RowData>`.

##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java
##########
@@ -19,64 +19,719 @@
 
 package org.apache.iceberg.flink.data;
 
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.time.Instant;
 import java.util.List;
-import org.apache.flink.types.Row;
+import java.util.Map;
+import org.apache.commons.lang3.ArrayUtils;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.MapData;
+import org.apache.flink.table.data.RawValueData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
 import org.apache.iceberg.Schema;
-import org.apache.iceberg.data.parquet.BaseParquetReaders;
+import org.apache.iceberg.parquet.ParquetSchemaUtil;
 import org.apache.iceberg.parquet.ParquetValueReader;
 import org.apache.iceberg.parquet.ParquetValueReaders;
+import org.apache.iceberg.parquet.TypeWithSchemaVisitor;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
 import org.apache.iceberg.types.Types;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.io.api.Binary;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.LogicalTypeAnnotation.DecimalLogicalTypeAnnotation;
 import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
 import org.apache.parquet.schema.Type;
 
-public class FlinkParquetReaders extends BaseParquetReaders<Row> {
+class FlinkParquetReaders {
+  private FlinkParquetReaders() {
+  }
 
-  private static final FlinkParquetReaders INSTANCE = new FlinkParquetReaders();
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema, MessageType fileSchema) {
+    return buildReader(expectedSchema, fileSchema, ImmutableMap.of());
+  }
 
-  private FlinkParquetReaders() {
+  @SuppressWarnings("unchecked")
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema,
+                                                        MessageType fileSchema,
+                                                        Map<Integer, ?> idToConstant) {
+    ReadBuilder builder = new ReadBuilder(fileSchema, idToConstant);
+    if (ParquetSchemaUtil.hasIds(fileSchema)) {
+      return (ParquetValueReader<RowData>)
+          TypeWithSchemaVisitor.visit(expectedSchema.asStruct(), fileSchema, builder);
+    } else {
+      return (ParquetValueReader<RowData>)
+          TypeWithSchemaVisitor.visit(expectedSchema.asStruct(), fileSchema,
+              new FallbackReadBuilder(builder));
+    }
+  }
+
+  private static class FallbackReadBuilder extends TypeWithSchemaVisitor<ParquetValueReader<?>> {
+    private MessageType type;
+    private final TypeWithSchemaVisitor<ParquetValueReader<?>> builder;

Review comment:
       Q: Is there any problem here ?  The `FallbackReadBuilder` don't need to implement those methods ? 
   1. `public T list(Types.ListType iList, GroupType array, T element)` ; 
   2. `public T map(Types.MapType iMap, GroupType map, T key, T value)` ; 
   3.  `public T primitive(org.apache.iceberg.types.Type.PrimitiveType iPrimitive, PrimitiveType primitive)`
   

##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java
##########
@@ -19,64 +19,719 @@
 
 package org.apache.iceberg.flink.data;
 
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.time.Instant;
 import java.util.List;
-import org.apache.flink.types.Row;
+import java.util.Map;
+import org.apache.commons.lang3.ArrayUtils;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.MapData;
+import org.apache.flink.table.data.RawValueData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
 import org.apache.iceberg.Schema;
-import org.apache.iceberg.data.parquet.BaseParquetReaders;
+import org.apache.iceberg.parquet.ParquetSchemaUtil;
 import org.apache.iceberg.parquet.ParquetValueReader;
 import org.apache.iceberg.parquet.ParquetValueReaders;
+import org.apache.iceberg.parquet.TypeWithSchemaVisitor;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
 import org.apache.iceberg.types.Types;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.io.api.Binary;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.LogicalTypeAnnotation.DecimalLogicalTypeAnnotation;
 import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
 import org.apache.parquet.schema.Type;
 
-public class FlinkParquetReaders extends BaseParquetReaders<Row> {
+class FlinkParquetReaders {
+  private FlinkParquetReaders() {
+  }
 
-  private static final FlinkParquetReaders INSTANCE = new FlinkParquetReaders();
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema, MessageType fileSchema) {
+    return buildReader(expectedSchema, fileSchema, ImmutableMap.of());
+  }
 
-  private FlinkParquetReaders() {
+  @SuppressWarnings("unchecked")
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema,
+                                                        MessageType fileSchema,
+                                                        Map<Integer, ?> idToConstant) {
+    ReadBuilder builder = new ReadBuilder(fileSchema, idToConstant);
+    if (ParquetSchemaUtil.hasIds(fileSchema)) {
+      return (ParquetValueReader<RowData>)
+          TypeWithSchemaVisitor.visit(expectedSchema.asStruct(), fileSchema, builder);
+    } else {
+      return (ParquetValueReader<RowData>)
+          TypeWithSchemaVisitor.visit(expectedSchema.asStruct(), fileSchema,
+              new FallbackReadBuilder(builder));
+    }
+  }
+
+  private static class FallbackReadBuilder extends TypeWithSchemaVisitor<ParquetValueReader<?>> {
+    private MessageType type;
+    private final TypeWithSchemaVisitor<ParquetValueReader<?>> builder;
+
+    FallbackReadBuilder(TypeWithSchemaVisitor<ParquetValueReader<?>> builder) {
+      this.builder = builder;
+    }
+
+    @Override
+    public ParquetValueReader<?> message(Types.StructType expected, MessageType message,
+                                         List<ParquetValueReader<?>> fieldReaders) {
+      // the top level matches by ID, but the remaining IDs are missing
+      this.type = message;
+      return builder.struct(expected, message, fieldReaders);
+    }
+
+    @Override
+    public ParquetValueReader<?> struct(Types.StructType ignored, GroupType struct,
+                                        List<ParquetValueReader<?>> fieldReaders) {
+      // the expected struct is ignored because nested fields are never found when the IDs are missing
+      List<ParquetValueReader<?>> newFields = Lists.newArrayListWithExpectedSize(
+          fieldReaders.size());
+      List<Type> types = Lists.newArrayListWithExpectedSize(fieldReaders.size());
+      List<Type> fields = struct.getFields();
+      for (int i = 0; i < fields.size(); i += 1) {
+        Type fieldType = fields.get(i);
+        int fieldD = type.getMaxDefinitionLevel(path(fieldType.getName())) - 1;
+        newFields.add(ParquetValueReaders.option(fieldType, fieldD, fieldReaders.get(i)));
+        types.add(fieldType);
+      }
+
+      return new RowDataReader(types, newFields);
+    }
+  }
+
+  private static class ReadBuilder extends TypeWithSchemaVisitor<ParquetValueReader<?>> {

Review comment:
       Q:  Is there neccessray to abstract a common `ReadBuilder`  and make the GenericParquetReader , SparkParquetReader, FlinkParquetReader to share it ?  I mean we could define different Reader(s) for different engine data type, for example we may have Flink's StructReader , and Spark's StructReader,  but the implementation of  TypeWithSchemaVisitor could be shared.   Does it make sense ?   CC @rdblue
   
   I raise this issue because I saw almost all the codes in ReadBuilder are the same, different copies may need extra resources (both contributor and reviewers) to maintain them. 

##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java
##########
@@ -19,64 +19,719 @@
 
 package org.apache.iceberg.flink.data;
 
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.time.Instant;
 import java.util.List;
-import org.apache.flink.types.Row;
+import java.util.Map;
+import org.apache.commons.lang3.ArrayUtils;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.MapData;
+import org.apache.flink.table.data.RawValueData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
 import org.apache.iceberg.Schema;
-import org.apache.iceberg.data.parquet.BaseParquetReaders;
+import org.apache.iceberg.parquet.ParquetSchemaUtil;
 import org.apache.iceberg.parquet.ParquetValueReader;
 import org.apache.iceberg.parquet.ParquetValueReaders;
+import org.apache.iceberg.parquet.TypeWithSchemaVisitor;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
 import org.apache.iceberg.types.Types;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.io.api.Binary;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.LogicalTypeAnnotation.DecimalLogicalTypeAnnotation;
 import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
 import org.apache.parquet.schema.Type;
 
-public class FlinkParquetReaders extends BaseParquetReaders<Row> {
+class FlinkParquetReaders {
+  private FlinkParquetReaders() {
+  }
 
-  private static final FlinkParquetReaders INSTANCE = new FlinkParquetReaders();
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema, MessageType fileSchema) {
+    return buildReader(expectedSchema, fileSchema, ImmutableMap.of());
+  }
 
-  private FlinkParquetReaders() {
+  @SuppressWarnings("unchecked")
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema,
+                                                        MessageType fileSchema,
+                                                        Map<Integer, ?> idToConstant) {
+    ReadBuilder builder = new ReadBuilder(fileSchema, idToConstant);
+    if (ParquetSchemaUtil.hasIds(fileSchema)) {
+      return (ParquetValueReader<RowData>)
+          TypeWithSchemaVisitor.visit(expectedSchema.asStruct(), fileSchema, builder);
+    } else {
+      return (ParquetValueReader<RowData>)
+          TypeWithSchemaVisitor.visit(expectedSchema.asStruct(), fileSchema,
+              new FallbackReadBuilder(builder));
+    }
+  }
+
+  private static class FallbackReadBuilder extends TypeWithSchemaVisitor<ParquetValueReader<?>> {
+    private MessageType type;
+    private final TypeWithSchemaVisitor<ParquetValueReader<?>> builder;
+
+    FallbackReadBuilder(TypeWithSchemaVisitor<ParquetValueReader<?>> builder) {
+      this.builder = builder;
+    }
+
+    @Override
+    public ParquetValueReader<?> message(Types.StructType expected, MessageType message,
+                                         List<ParquetValueReader<?>> fieldReaders) {
+      // the top level matches by ID, but the remaining IDs are missing
+      this.type = message;
+      return builder.struct(expected, message, fieldReaders);
+    }
+
+    @Override
+    public ParquetValueReader<?> struct(Types.StructType ignored, GroupType struct,
+                                        List<ParquetValueReader<?>> fieldReaders) {
+      // the expected struct is ignored because nested fields are never found when the IDs are missing
+      List<ParquetValueReader<?>> newFields = Lists.newArrayListWithExpectedSize(
+          fieldReaders.size());
+      List<Type> types = Lists.newArrayListWithExpectedSize(fieldReaders.size());
+      List<Type> fields = struct.getFields();
+      for (int i = 0; i < fields.size(); i += 1) {
+        Type fieldType = fields.get(i);
+        int fieldD = type.getMaxDefinitionLevel(path(fieldType.getName())) - 1;
+        newFields.add(ParquetValueReaders.option(fieldType, fieldD, fieldReaders.get(i)));
+        types.add(fieldType);
+      }
+
+      return new RowDataReader(types, newFields);
+    }
+  }
+
+  private static class ReadBuilder extends TypeWithSchemaVisitor<ParquetValueReader<?>> {
+    private final MessageType type;
+    private final Map<Integer, ?> idToConstant;
+
+    ReadBuilder(MessageType type, Map<Integer, ?> idToConstant) {
+      this.type = type;
+      this.idToConstant = idToConstant;
+    }
+
+    @Override
+    public ParquetValueReader<?> message(Types.StructType expected, MessageType message,
+                                         List<ParquetValueReader<?>> fieldReaders) {
+      return struct(expected, message.asGroupType(), fieldReaders);
+    }
+
+    @Override
+    public ParquetValueReader<?> struct(Types.StructType expected, GroupType struct,
+                                        List<ParquetValueReader<?>> fieldReaders) {
+      // match the expected struct's order
+      Map<Integer, ParquetValueReader<?>> readersById = Maps.newHashMap();
+      Map<Integer, Type> typesById = Maps.newHashMap();
+      List<Type> fields = struct.getFields();
+      for (int i = 0; i < fields.size(); i += 1) {
+        Type fieldType = fields.get(i);
+        int fieldD = type.getMaxDefinitionLevel(path(fieldType.getName())) - 1;
+        if (fieldType.getId() != null) {
+          int id = fieldType.getId().intValue();
+          readersById.put(id, ParquetValueReaders.option(fieldType, fieldD, fieldReaders.get(i)));
+          typesById.put(id, fieldType);
+        }
+      }
+
+      List<Types.NestedField> expectedFields = expected != null ?
+          expected.fields() : ImmutableList.of();
+      List<ParquetValueReader<?>> reorderedFields = Lists.newArrayListWithExpectedSize(
+          expectedFields.size());
+      List<Type> types = Lists.newArrayListWithExpectedSize(expectedFields.size());
+      for (Types.NestedField field : expectedFields) {
+        int id = field.fieldId();
+        if (idToConstant.containsKey(id)) {
+          // containsKey is used because the constant may be null
+          reorderedFields.add(ParquetValueReaders.constant(idToConstant.get(id)));
+          types.add(null);
+        } else {
+          ParquetValueReader<?> reader = readersById.get(id);
+          if (reader != null) {
+            reorderedFields.add(reader);
+            types.add(typesById.get(id));
+          } else {
+            reorderedFields.add(ParquetValueReaders.nulls());
+            types.add(null);
+          }
+        }
+      }
+
+      return new RowDataReader(types, reorderedFields);
+    }
+
+    @Override
+    public ParquetValueReader<?> list(Types.ListType expectedList, GroupType array,
+                                      ParquetValueReader<?> elementReader) {
+      GroupType repeated = array.getFields().get(0).asGroupType();
+      String[] repeatedPath = currentPath();
+
+      int repeatedD = type.getMaxDefinitionLevel(repeatedPath) - 1;
+      int repeatedR = type.getMaxRepetitionLevel(repeatedPath) - 1;
+
+      Type elementType = repeated.getType(0);
+      int elementD = type.getMaxDefinitionLevel(path(elementType.getName())) - 1;
+
+      return new ArrayReader<>(repeatedD, repeatedR, ParquetValueReaders.option(elementType, elementD, elementReader));
+    }
+
+    @Override
+    public ParquetValueReader<?> map(Types.MapType expectedMap, GroupType map,
+                                     ParquetValueReader<?> keyReader,
+                                     ParquetValueReader<?> valueReader) {
+      GroupType repeatedKeyValue = map.getFields().get(0).asGroupType();
+      String[] repeatedPath = currentPath();
+
+      int repeatedD = type.getMaxDefinitionLevel(repeatedPath) - 1;
+      int repeatedR = type.getMaxRepetitionLevel(repeatedPath) - 1;
+
+      Type keyType = repeatedKeyValue.getType(0);
+      int keyD = type.getMaxDefinitionLevel(path(keyType.getName())) - 1;
+      Type valueType = repeatedKeyValue.getType(1);
+      int valueD = type.getMaxDefinitionLevel(path(valueType.getName())) - 1;
+
+      return new MapReader<>(repeatedD, repeatedR,
+          ParquetValueReaders.option(keyType, keyD, keyReader),
+          ParquetValueReaders.option(valueType, valueD, valueReader));
+    }
+
+    @Override
+    public ParquetValueReader<?> primitive(org.apache.iceberg.types.Type.PrimitiveType expected,
+                                           PrimitiveType primitive) {
+      ColumnDescriptor desc = type.getColumnDescription(currentPath());
+
+      if (primitive.getOriginalType() != null) {
+        switch (primitive.getOriginalType()) {
+          case ENUM:
+          case JSON:
+          case UTF8:
+            return new StringReader(desc);
+          case INT_8:
+          case INT_16:
+          case INT_32:
+          case DATE:
+            if (expected != null && expected.typeId() == Types.LongType.get().typeId()) {
+              return new ParquetValueReaders.IntAsLongReader(desc);
+            } else {
+              return new ParquetValueReaders.UnboxedReader<>(desc);
+            }
+          case TIME_MICROS:
+            return new TimeMillisReader(desc);
+          case INT_64:
+            return new ParquetValueReaders.UnboxedReader<>(desc);
+          case TIMESTAMP_MICROS:
+            return new TimestampMicroReader(desc);
+          case DECIMAL:
+            DecimalLogicalTypeAnnotation decimal = (DecimalLogicalTypeAnnotation) primitive.getLogicalTypeAnnotation();
+            switch (primitive.getPrimitiveTypeName()) {
+              case BINARY:
+              case FIXED_LEN_BYTE_ARRAY:
+                return new BinaryDecimalReader(desc, decimal.getScale());
+              case INT64:
+                return new LongDecimalReader(desc, decimal.getPrecision(), decimal.getScale());
+              case INT32:
+                return new IntegerDecimalReader(desc, decimal.getPrecision(), decimal.getScale());
+              default:
+                throw new UnsupportedOperationException(
+                    "Unsupported base type for decimal: " + primitive.getPrimitiveTypeName());
+            }
+          case BSON:
+            return new ParquetValueReaders.ByteArrayReader(desc);
+          default:
+            throw new UnsupportedOperationException(
+                "Unsupported logical type: " + primitive.getOriginalType());
+        }
+      }
+
+      switch (primitive.getPrimitiveTypeName()) {
+        case FIXED_LEN_BYTE_ARRAY:
+        case BINARY:
+          return new ParquetValueReaders.ByteArrayReader(desc);
+        case INT32:
+          if (expected != null && expected.typeId() == org.apache.iceberg.types.Type.TypeID.LONG) {
+            return new ParquetValueReaders.IntAsLongReader(desc);
+          } else {
+            return new ParquetValueReaders.UnboxedReader<>(desc);
+          }
+        case FLOAT:
+          if (expected != null && expected.typeId() == org.apache.iceberg.types.Type.TypeID.DOUBLE) {
+            return new ParquetValueReaders.FloatAsDoubleReader(desc);
+          } else {
+            return new ParquetValueReaders.UnboxedReader<>(desc);
+          }
+        case BOOLEAN:
+        case INT64:
+        case DOUBLE:
+          return new ParquetValueReaders.UnboxedReader<>(desc);
+        default:
+          throw new UnsupportedOperationException("Unsupported type: " + primitive);
+      }
+    }
+  }
+
+  private static class BinaryDecimalReader extends ParquetValueReaders.PrimitiveReader<DecimalData> {
+    private final int scale;
+
+    BinaryDecimalReader(ColumnDescriptor desc, int scale) {
+      super(desc);
+      this.scale = scale;
+    }
+
+    @Override
+    public DecimalData read(DecimalData ignored) {
+      Binary binary = column.nextBinary();
+      BigDecimal bigDecimal = new BigDecimal(new BigInteger(binary.getBytes()), scale);
+      return DecimalData.fromBigDecimal(bigDecimal, bigDecimal.precision(), scale);
+    }
+  }
+
+  private static class IntegerDecimalReader extends ParquetValueReaders.PrimitiveReader<DecimalData> {
+    private final int precision;
+    private final int scale;
+
+    IntegerDecimalReader(ColumnDescriptor desc, int precision, int scale) {
+      super(desc);
+      this.precision = precision;
+      this.scale = scale;
+    }
+
+    @Override
+    public DecimalData read(DecimalData ignored) {
+      return DecimalData.fromUnscaledLong(column.nextInteger(), precision, scale);
+    }
+  }
+
+  private static class LongDecimalReader extends ParquetValueReaders.PrimitiveReader<DecimalData> {
+    private final int precision;
+    private final int scale;
+
+    LongDecimalReader(ColumnDescriptor desc, int precision, int scale) {
+      super(desc);
+      this.precision = precision;
+      this.scale = scale;
+    }
+
+    @Override
+    public DecimalData read(DecimalData ignored) {
+      return DecimalData.fromUnscaledLong(column.nextLong(), precision, scale);
+    }
   }
 
-  public static ParquetValueReader<Row> buildReader(Schema expectedSchema, MessageType fileSchema) {
-    return INSTANCE.createReader(expectedSchema, fileSchema);
+  private static class TimestampMicroReader extends ParquetValueReaders.UnboxedReader<TimestampData> {
+    TimestampMicroReader(ColumnDescriptor desc) {
+      super(desc);
+    }
+
+    @Override
+    public TimestampData read(TimestampData ignored) {
+      long value = readLong();
+      return TimestampData.fromInstant(Instant.ofEpochSecond(value / 1000_000, (value % 1000_000) * 1000));
+    }
+
+    @Override
+    public long readLong() {
+      return column.nextLong();
+    }
+  }
+
+  private static class StringReader extends ParquetValueReaders.PrimitiveReader<StringData> {
+    StringReader(ColumnDescriptor desc) {
+      super(desc);
+    }
+
+    @Override
+    public StringData read(StringData ignored) {
+      Binary binary = column.nextBinary();
+      ByteBuffer buffer = binary.toByteBuffer();
+      if (buffer.hasArray()) {
+        return StringData.fromBytes(
+            buffer.array(), buffer.arrayOffset() + buffer.position(), buffer.remaining());
+      } else {
+        return StringData.fromBytes(binary.getBytes());
+      }
+    }
   }
 
-  @Override
-  protected ParquetValueReader<Row> createStructReader(List<Type> types,
-                                                       List<ParquetValueReader<?>> fieldReaders,
-                                                       Types.StructType structType) {
-    return new RowReader(types, fieldReaders, structType);
+  private static class TimeMillisReader extends ParquetValueReaders.PrimitiveReader<Integer> {
+    TimeMillisReader(ColumnDescriptor desc) {
+      super(desc);
+    }
+
+    @Override
+    public Integer read(Integer reuse) {
+      // Flink only supports millisecond, so we discard microseconds in millisecond
+      return (int) column.nextLong() / 1000;

Review comment:
       ditto. 

##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java
##########
@@ -19,64 +19,719 @@
 
 package org.apache.iceberg.flink.data;
 
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.time.Instant;
 import java.util.List;
-import org.apache.flink.types.Row;
+import java.util.Map;
+import org.apache.commons.lang3.ArrayUtils;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.MapData;
+import org.apache.flink.table.data.RawValueData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
 import org.apache.iceberg.Schema;
-import org.apache.iceberg.data.parquet.BaseParquetReaders;
+import org.apache.iceberg.parquet.ParquetSchemaUtil;
 import org.apache.iceberg.parquet.ParquetValueReader;
 import org.apache.iceberg.parquet.ParquetValueReaders;
+import org.apache.iceberg.parquet.TypeWithSchemaVisitor;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
 import org.apache.iceberg.types.Types;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.io.api.Binary;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.LogicalTypeAnnotation.DecimalLogicalTypeAnnotation;
 import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
 import org.apache.parquet.schema.Type;
 
-public class FlinkParquetReaders extends BaseParquetReaders<Row> {
+class FlinkParquetReaders {
+  private FlinkParquetReaders() {
+  }
 
-  private static final FlinkParquetReaders INSTANCE = new FlinkParquetReaders();
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema, MessageType fileSchema) {
+    return buildReader(expectedSchema, fileSchema, ImmutableMap.of());
+  }
 
-  private FlinkParquetReaders() {
+  @SuppressWarnings("unchecked")
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema,
+                                                        MessageType fileSchema,
+                                                        Map<Integer, ?> idToConstant) {
+    ReadBuilder builder = new ReadBuilder(fileSchema, idToConstant);
+    if (ParquetSchemaUtil.hasIds(fileSchema)) {
+      return (ParquetValueReader<RowData>)
+          TypeWithSchemaVisitor.visit(expectedSchema.asStruct(), fileSchema, builder);
+    } else {
+      return (ParquetValueReader<RowData>)
+          TypeWithSchemaVisitor.visit(expectedSchema.asStruct(), fileSchema,
+              new FallbackReadBuilder(builder));
+    }
+  }
+
+  private static class FallbackReadBuilder extends TypeWithSchemaVisitor<ParquetValueReader<?>> {
+    private MessageType type;
+    private final TypeWithSchemaVisitor<ParquetValueReader<?>> builder;
+
+    FallbackReadBuilder(TypeWithSchemaVisitor<ParquetValueReader<?>> builder) {
+      this.builder = builder;
+    }
+
+    @Override
+    public ParquetValueReader<?> message(Types.StructType expected, MessageType message,
+                                         List<ParquetValueReader<?>> fieldReaders) {
+      // the top level matches by ID, but the remaining IDs are missing
+      this.type = message;
+      return builder.struct(expected, message, fieldReaders);
+    }
+
+    @Override
+    public ParquetValueReader<?> struct(Types.StructType ignored, GroupType struct,
+                                        List<ParquetValueReader<?>> fieldReaders) {
+      // the expected struct is ignored because nested fields are never found when the IDs are missing
+      List<ParquetValueReader<?>> newFields = Lists.newArrayListWithExpectedSize(
+          fieldReaders.size());
+      List<Type> types = Lists.newArrayListWithExpectedSize(fieldReaders.size());
+      List<Type> fields = struct.getFields();
+      for (int i = 0; i < fields.size(); i += 1) {
+        Type fieldType = fields.get(i);
+        int fieldD = type.getMaxDefinitionLevel(path(fieldType.getName())) - 1;
+        newFields.add(ParquetValueReaders.option(fieldType, fieldD, fieldReaders.get(i)));
+        types.add(fieldType);
+      }
+
+      return new RowDataReader(types, newFields);
+    }
+  }
+
+  private static class ReadBuilder extends TypeWithSchemaVisitor<ParquetValueReader<?>> {
+    private final MessageType type;
+    private final Map<Integer, ?> idToConstant;
+
+    ReadBuilder(MessageType type, Map<Integer, ?> idToConstant) {
+      this.type = type;
+      this.idToConstant = idToConstant;
+    }
+
+    @Override
+    public ParquetValueReader<?> message(Types.StructType expected, MessageType message,
+                                         List<ParquetValueReader<?>> fieldReaders) {
+      return struct(expected, message.asGroupType(), fieldReaders);
+    }
+
+    @Override
+    public ParquetValueReader<?> struct(Types.StructType expected, GroupType struct,
+                                        List<ParquetValueReader<?>> fieldReaders) {
+      // match the expected struct's order
+      Map<Integer, ParquetValueReader<?>> readersById = Maps.newHashMap();
+      Map<Integer, Type> typesById = Maps.newHashMap();
+      List<Type> fields = struct.getFields();
+      for (int i = 0; i < fields.size(); i += 1) {
+        Type fieldType = fields.get(i);
+        int fieldD = type.getMaxDefinitionLevel(path(fieldType.getName())) - 1;
+        if (fieldType.getId() != null) {
+          int id = fieldType.getId().intValue();
+          readersById.put(id, ParquetValueReaders.option(fieldType, fieldD, fieldReaders.get(i)));
+          typesById.put(id, fieldType);
+        }
+      }
+
+      List<Types.NestedField> expectedFields = expected != null ?
+          expected.fields() : ImmutableList.of();
+      List<ParquetValueReader<?>> reorderedFields = Lists.newArrayListWithExpectedSize(
+          expectedFields.size());
+      List<Type> types = Lists.newArrayListWithExpectedSize(expectedFields.size());
+      for (Types.NestedField field : expectedFields) {
+        int id = field.fieldId();
+        if (idToConstant.containsKey(id)) {
+          // containsKey is used because the constant may be null
+          reorderedFields.add(ParquetValueReaders.constant(idToConstant.get(id)));
+          types.add(null);
+        } else {
+          ParquetValueReader<?> reader = readersById.get(id);
+          if (reader != null) {
+            reorderedFields.add(reader);
+            types.add(typesById.get(id));
+          } else {
+            reorderedFields.add(ParquetValueReaders.nulls());
+            types.add(null);
+          }
+        }
+      }
+
+      return new RowDataReader(types, reorderedFields);
+    }
+
+    @Override
+    public ParquetValueReader<?> list(Types.ListType expectedList, GroupType array,
+                                      ParquetValueReader<?> elementReader) {
+      GroupType repeated = array.getFields().get(0).asGroupType();
+      String[] repeatedPath = currentPath();
+
+      int repeatedD = type.getMaxDefinitionLevel(repeatedPath) - 1;
+      int repeatedR = type.getMaxRepetitionLevel(repeatedPath) - 1;
+
+      Type elementType = repeated.getType(0);
+      int elementD = type.getMaxDefinitionLevel(path(elementType.getName())) - 1;
+
+      return new ArrayReader<>(repeatedD, repeatedR, ParquetValueReaders.option(elementType, elementD, elementReader));
+    }
+
+    @Override
+    public ParquetValueReader<?> map(Types.MapType expectedMap, GroupType map,
+                                     ParquetValueReader<?> keyReader,
+                                     ParquetValueReader<?> valueReader) {
+      GroupType repeatedKeyValue = map.getFields().get(0).asGroupType();
+      String[] repeatedPath = currentPath();
+
+      int repeatedD = type.getMaxDefinitionLevel(repeatedPath) - 1;
+      int repeatedR = type.getMaxRepetitionLevel(repeatedPath) - 1;
+
+      Type keyType = repeatedKeyValue.getType(0);
+      int keyD = type.getMaxDefinitionLevel(path(keyType.getName())) - 1;
+      Type valueType = repeatedKeyValue.getType(1);
+      int valueD = type.getMaxDefinitionLevel(path(valueType.getName())) - 1;
+
+      return new MapReader<>(repeatedD, repeatedR,
+          ParquetValueReaders.option(keyType, keyD, keyReader),
+          ParquetValueReaders.option(valueType, valueD, valueReader));
+    }
+
+    @Override
+    public ParquetValueReader<?> primitive(org.apache.iceberg.types.Type.PrimitiveType expected,
+                                           PrimitiveType primitive) {
+      ColumnDescriptor desc = type.getColumnDescription(currentPath());
+
+      if (primitive.getOriginalType() != null) {
+        switch (primitive.getOriginalType()) {
+          case ENUM:
+          case JSON:
+          case UTF8:
+            return new StringReader(desc);
+          case INT_8:
+          case INT_16:
+          case INT_32:
+          case DATE:
+            if (expected != null && expected.typeId() == Types.LongType.get().typeId()) {
+              return new ParquetValueReaders.IntAsLongReader(desc);
+            } else {
+              return new ParquetValueReaders.UnboxedReader<>(desc);
+            }
+          case TIME_MICROS:
+            return new TimeMillisReader(desc);
+          case INT_64:
+            return new ParquetValueReaders.UnboxedReader<>(desc);
+          case TIMESTAMP_MICROS:
+            return new TimestampMicroReader(desc);
+          case DECIMAL:
+            DecimalLogicalTypeAnnotation decimal = (DecimalLogicalTypeAnnotation) primitive.getLogicalTypeAnnotation();
+            switch (primitive.getPrimitiveTypeName()) {
+              case BINARY:
+              case FIXED_LEN_BYTE_ARRAY:
+                return new BinaryDecimalReader(desc, decimal.getScale());
+              case INT64:
+                return new LongDecimalReader(desc, decimal.getPrecision(), decimal.getScale());
+              case INT32:
+                return new IntegerDecimalReader(desc, decimal.getPrecision(), decimal.getScale());
+              default:
+                throw new UnsupportedOperationException(
+                    "Unsupported base type for decimal: " + primitive.getPrimitiveTypeName());
+            }
+          case BSON:
+            return new ParquetValueReaders.ByteArrayReader(desc);
+          default:
+            throw new UnsupportedOperationException(
+                "Unsupported logical type: " + primitive.getOriginalType());
+        }
+      }
+
+      switch (primitive.getPrimitiveTypeName()) {
+        case FIXED_LEN_BYTE_ARRAY:
+        case BINARY:
+          return new ParquetValueReaders.ByteArrayReader(desc);
+        case INT32:
+          if (expected != null && expected.typeId() == org.apache.iceberg.types.Type.TypeID.LONG) {
+            return new ParquetValueReaders.IntAsLongReader(desc);
+          } else {
+            return new ParquetValueReaders.UnboxedReader<>(desc);
+          }
+        case FLOAT:
+          if (expected != null && expected.typeId() == org.apache.iceberg.types.Type.TypeID.DOUBLE) {
+            return new ParquetValueReaders.FloatAsDoubleReader(desc);
+          } else {
+            return new ParquetValueReaders.UnboxedReader<>(desc);
+          }
+        case BOOLEAN:
+        case INT64:
+        case DOUBLE:
+          return new ParquetValueReaders.UnboxedReader<>(desc);
+        default:
+          throw new UnsupportedOperationException("Unsupported type: " + primitive);
+      }
+    }
+  }
+
+  private static class BinaryDecimalReader extends ParquetValueReaders.PrimitiveReader<DecimalData> {
+    private final int scale;
+
+    BinaryDecimalReader(ColumnDescriptor desc, int scale) {
+      super(desc);
+      this.scale = scale;
+    }
+
+    @Override
+    public DecimalData read(DecimalData ignored) {
+      Binary binary = column.nextBinary();
+      BigDecimal bigDecimal = new BigDecimal(new BigInteger(binary.getBytes()), scale);
+      return DecimalData.fromBigDecimal(bigDecimal, bigDecimal.precision(), scale);
+    }
+  }
+
+  private static class IntegerDecimalReader extends ParquetValueReaders.PrimitiveReader<DecimalData> {
+    private final int precision;
+    private final int scale;
+
+    IntegerDecimalReader(ColumnDescriptor desc, int precision, int scale) {
+      super(desc);
+      this.precision = precision;
+      this.scale = scale;
+    }
+
+    @Override
+    public DecimalData read(DecimalData ignored) {
+      return DecimalData.fromUnscaledLong(column.nextInteger(), precision, scale);
+    }
+  }
+
+  private static class LongDecimalReader extends ParquetValueReaders.PrimitiveReader<DecimalData> {
+    private final int precision;
+    private final int scale;
+
+    LongDecimalReader(ColumnDescriptor desc, int precision, int scale) {
+      super(desc);
+      this.precision = precision;
+      this.scale = scale;
+    }
+
+    @Override
+    public DecimalData read(DecimalData ignored) {
+      return DecimalData.fromUnscaledLong(column.nextLong(), precision, scale);
+    }
   }
 
-  public static ParquetValueReader<Row> buildReader(Schema expectedSchema, MessageType fileSchema) {
-    return INSTANCE.createReader(expectedSchema, fileSchema);
+  private static class TimestampMicroReader extends ParquetValueReaders.UnboxedReader<TimestampData> {
+    TimestampMicroReader(ColumnDescriptor desc) {
+      super(desc);
+    }
+
+    @Override
+    public TimestampData read(TimestampData ignored) {
+      long value = readLong();
+      return TimestampData.fromInstant(Instant.ofEpochSecond(value / 1000_000, (value % 1000_000) * 1000));
+    }
+
+    @Override
+    public long readLong() {
+      return column.nextLong();
+    }
+  }
+
+  private static class StringReader extends ParquetValueReaders.PrimitiveReader<StringData> {
+    StringReader(ColumnDescriptor desc) {
+      super(desc);
+    }
+
+    @Override
+    public StringData read(StringData ignored) {
+      Binary binary = column.nextBinary();
+      ByteBuffer buffer = binary.toByteBuffer();
+      if (buffer.hasArray()) {
+        return StringData.fromBytes(
+            buffer.array(), buffer.arrayOffset() + buffer.position(), buffer.remaining());
+      } else {
+        return StringData.fromBytes(binary.getBytes());
+      }
+    }
   }
 
-  @Override
-  protected ParquetValueReader<Row> createStructReader(List<Type> types,
-                                                       List<ParquetValueReader<?>> fieldReaders,
-                                                       Types.StructType structType) {
-    return new RowReader(types, fieldReaders, structType);
+  private static class TimeMillisReader extends ParquetValueReaders.PrimitiveReader<Integer> {
+    TimeMillisReader(ColumnDescriptor desc) {
+      super(desc);
+    }
+
+    @Override
+    public Integer read(Integer reuse) {
+      // Flink only supports millisecond, so we discard microseconds in millisecond
+      return (int) column.nextLong() / 1000;
+    }
   }
 
-  private static class RowReader extends ParquetValueReaders.StructReader<Row, Row> {
-    private final Types.StructType structType;
+  private static class ArrayReader<E> extends ParquetValueReaders.RepeatedReader<ArrayData, ReusableArrayData, E> {
+    private int readPos = 0;
+    private int writePos = 0;
+
+    ArrayReader(int definitionLevel, int repetitionLevel, ParquetValueReader<E> reader) {
+      super(definitionLevel, repetitionLevel, reader);
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")

Review comment:
       it could be removed now ? 

##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java
##########
@@ -19,64 +19,719 @@
 
 package org.apache.iceberg.flink.data;
 
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.time.Instant;
 import java.util.List;
-import org.apache.flink.types.Row;
+import java.util.Map;
+import org.apache.commons.lang3.ArrayUtils;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.MapData;
+import org.apache.flink.table.data.RawValueData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
 import org.apache.iceberg.Schema;
-import org.apache.iceberg.data.parquet.BaseParquetReaders;
+import org.apache.iceberg.parquet.ParquetSchemaUtil;
 import org.apache.iceberg.parquet.ParquetValueReader;
 import org.apache.iceberg.parquet.ParquetValueReaders;
+import org.apache.iceberg.parquet.TypeWithSchemaVisitor;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
 import org.apache.iceberg.types.Types;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.io.api.Binary;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.LogicalTypeAnnotation.DecimalLogicalTypeAnnotation;
 import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
 import org.apache.parquet.schema.Type;
 
-public class FlinkParquetReaders extends BaseParquetReaders<Row> {
+class FlinkParquetReaders {
+  private FlinkParquetReaders() {
+  }
 
-  private static final FlinkParquetReaders INSTANCE = new FlinkParquetReaders();
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema, MessageType fileSchema) {
+    return buildReader(expectedSchema, fileSchema, ImmutableMap.of());
+  }
 
-  private FlinkParquetReaders() {
+  @SuppressWarnings("unchecked")
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema,
+                                                        MessageType fileSchema,
+                                                        Map<Integer, ?> idToConstant) {
+    ReadBuilder builder = new ReadBuilder(fileSchema, idToConstant);
+    if (ParquetSchemaUtil.hasIds(fileSchema)) {
+      return (ParquetValueReader<RowData>)
+          TypeWithSchemaVisitor.visit(expectedSchema.asStruct(), fileSchema, builder);
+    } else {
+      return (ParquetValueReader<RowData>)
+          TypeWithSchemaVisitor.visit(expectedSchema.asStruct(), fileSchema,
+              new FallbackReadBuilder(builder));
+    }
+  }
+
+  private static class FallbackReadBuilder extends TypeWithSchemaVisitor<ParquetValueReader<?>> {
+    private MessageType type;
+    private final TypeWithSchemaVisitor<ParquetValueReader<?>> builder;
+
+    FallbackReadBuilder(TypeWithSchemaVisitor<ParquetValueReader<?>> builder) {
+      this.builder = builder;
+    }
+
+    @Override
+    public ParquetValueReader<?> message(Types.StructType expected, MessageType message,
+                                         List<ParquetValueReader<?>> fieldReaders) {
+      // the top level matches by ID, but the remaining IDs are missing
+      this.type = message;
+      return builder.struct(expected, message, fieldReaders);
+    }
+
+    @Override
+    public ParquetValueReader<?> struct(Types.StructType ignored, GroupType struct,
+                                        List<ParquetValueReader<?>> fieldReaders) {
+      // the expected struct is ignored because nested fields are never found when the IDs are missing
+      List<ParquetValueReader<?>> newFields = Lists.newArrayListWithExpectedSize(
+          fieldReaders.size());
+      List<Type> types = Lists.newArrayListWithExpectedSize(fieldReaders.size());
+      List<Type> fields = struct.getFields();
+      for (int i = 0; i < fields.size(); i += 1) {
+        Type fieldType = fields.get(i);
+        int fieldD = type.getMaxDefinitionLevel(path(fieldType.getName())) - 1;
+        newFields.add(ParquetValueReaders.option(fieldType, fieldD, fieldReaders.get(i)));
+        types.add(fieldType);
+      }
+
+      return new RowDataReader(types, newFields);
+    }
+  }
+
+  private static class ReadBuilder extends TypeWithSchemaVisitor<ParquetValueReader<?>> {
+    private final MessageType type;
+    private final Map<Integer, ?> idToConstant;
+
+    ReadBuilder(MessageType type, Map<Integer, ?> idToConstant) {
+      this.type = type;
+      this.idToConstant = idToConstant;
+    }
+
+    @Override
+    public ParquetValueReader<?> message(Types.StructType expected, MessageType message,
+                                         List<ParquetValueReader<?>> fieldReaders) {
+      return struct(expected, message.asGroupType(), fieldReaders);
+    }
+
+    @Override
+    public ParquetValueReader<?> struct(Types.StructType expected, GroupType struct,
+                                        List<ParquetValueReader<?>> fieldReaders) {
+      // match the expected struct's order
+      Map<Integer, ParquetValueReader<?>> readersById = Maps.newHashMap();
+      Map<Integer, Type> typesById = Maps.newHashMap();
+      List<Type> fields = struct.getFields();
+      for (int i = 0; i < fields.size(); i += 1) {
+        Type fieldType = fields.get(i);
+        int fieldD = type.getMaxDefinitionLevel(path(fieldType.getName())) - 1;
+        if (fieldType.getId() != null) {
+          int id = fieldType.getId().intValue();
+          readersById.put(id, ParquetValueReaders.option(fieldType, fieldD, fieldReaders.get(i)));
+          typesById.put(id, fieldType);
+        }
+      }
+
+      List<Types.NestedField> expectedFields = expected != null ?
+          expected.fields() : ImmutableList.of();
+      List<ParquetValueReader<?>> reorderedFields = Lists.newArrayListWithExpectedSize(
+          expectedFields.size());
+      List<Type> types = Lists.newArrayListWithExpectedSize(expectedFields.size());
+      for (Types.NestedField field : expectedFields) {
+        int id = field.fieldId();
+        if (idToConstant.containsKey(id)) {
+          // containsKey is used because the constant may be null
+          reorderedFields.add(ParquetValueReaders.constant(idToConstant.get(id)));
+          types.add(null);
+        } else {
+          ParquetValueReader<?> reader = readersById.get(id);
+          if (reader != null) {
+            reorderedFields.add(reader);
+            types.add(typesById.get(id));
+          } else {
+            reorderedFields.add(ParquetValueReaders.nulls());
+            types.add(null);
+          }
+        }
+      }
+
+      return new RowDataReader(types, reorderedFields);
+    }
+
+    @Override
+    public ParquetValueReader<?> list(Types.ListType expectedList, GroupType array,
+                                      ParquetValueReader<?> elementReader) {
+      GroupType repeated = array.getFields().get(0).asGroupType();
+      String[] repeatedPath = currentPath();
+
+      int repeatedD = type.getMaxDefinitionLevel(repeatedPath) - 1;
+      int repeatedR = type.getMaxRepetitionLevel(repeatedPath) - 1;
+
+      Type elementType = repeated.getType(0);
+      int elementD = type.getMaxDefinitionLevel(path(elementType.getName())) - 1;
+
+      return new ArrayReader<>(repeatedD, repeatedR, ParquetValueReaders.option(elementType, elementD, elementReader));
+    }
+
+    @Override
+    public ParquetValueReader<?> map(Types.MapType expectedMap, GroupType map,
+                                     ParquetValueReader<?> keyReader,
+                                     ParquetValueReader<?> valueReader) {
+      GroupType repeatedKeyValue = map.getFields().get(0).asGroupType();
+      String[] repeatedPath = currentPath();
+
+      int repeatedD = type.getMaxDefinitionLevel(repeatedPath) - 1;
+      int repeatedR = type.getMaxRepetitionLevel(repeatedPath) - 1;
+
+      Type keyType = repeatedKeyValue.getType(0);
+      int keyD = type.getMaxDefinitionLevel(path(keyType.getName())) - 1;
+      Type valueType = repeatedKeyValue.getType(1);
+      int valueD = type.getMaxDefinitionLevel(path(valueType.getName())) - 1;
+
+      return new MapReader<>(repeatedD, repeatedR,
+          ParquetValueReaders.option(keyType, keyD, keyReader),
+          ParquetValueReaders.option(valueType, valueD, valueReader));
+    }
+
+    @Override
+    public ParquetValueReader<?> primitive(org.apache.iceberg.types.Type.PrimitiveType expected,
+                                           PrimitiveType primitive) {
+      ColumnDescriptor desc = type.getColumnDescription(currentPath());
+
+      if (primitive.getOriginalType() != null) {
+        switch (primitive.getOriginalType()) {
+          case ENUM:
+          case JSON:
+          case UTF8:
+            return new StringReader(desc);
+          case INT_8:
+          case INT_16:
+          case INT_32:
+          case DATE:
+            if (expected != null && expected.typeId() == Types.LongType.get().typeId()) {
+              return new ParquetValueReaders.IntAsLongReader(desc);
+            } else {
+              return new ParquetValueReaders.UnboxedReader<>(desc);
+            }
+          case TIME_MICROS:
+            return new TimeMillisReader(desc);
+          case INT_64:
+            return new ParquetValueReaders.UnboxedReader<>(desc);
+          case TIMESTAMP_MICROS:
+            return new TimestampMicroReader(desc);
+          case DECIMAL:
+            DecimalLogicalTypeAnnotation decimal = (DecimalLogicalTypeAnnotation) primitive.getLogicalTypeAnnotation();
+            switch (primitive.getPrimitiveTypeName()) {
+              case BINARY:
+              case FIXED_LEN_BYTE_ARRAY:
+                return new BinaryDecimalReader(desc, decimal.getScale());
+              case INT64:
+                return new LongDecimalReader(desc, decimal.getPrecision(), decimal.getScale());
+              case INT32:
+                return new IntegerDecimalReader(desc, decimal.getPrecision(), decimal.getScale());
+              default:
+                throw new UnsupportedOperationException(
+                    "Unsupported base type for decimal: " + primitive.getPrimitiveTypeName());
+            }
+          case BSON:
+            return new ParquetValueReaders.ByteArrayReader(desc);
+          default:
+            throw new UnsupportedOperationException(
+                "Unsupported logical type: " + primitive.getOriginalType());
+        }
+      }
+
+      switch (primitive.getPrimitiveTypeName()) {
+        case FIXED_LEN_BYTE_ARRAY:
+        case BINARY:
+          return new ParquetValueReaders.ByteArrayReader(desc);
+        case INT32:
+          if (expected != null && expected.typeId() == org.apache.iceberg.types.Type.TypeID.LONG) {
+            return new ParquetValueReaders.IntAsLongReader(desc);
+          } else {
+            return new ParquetValueReaders.UnboxedReader<>(desc);
+          }
+        case FLOAT:
+          if (expected != null && expected.typeId() == org.apache.iceberg.types.Type.TypeID.DOUBLE) {
+            return new ParquetValueReaders.FloatAsDoubleReader(desc);
+          } else {
+            return new ParquetValueReaders.UnboxedReader<>(desc);
+          }
+        case BOOLEAN:
+        case INT64:
+        case DOUBLE:
+          return new ParquetValueReaders.UnboxedReader<>(desc);
+        default:
+          throw new UnsupportedOperationException("Unsupported type: " + primitive);
+      }
+    }
+  }
+
+  private static class BinaryDecimalReader extends ParquetValueReaders.PrimitiveReader<DecimalData> {
+    private final int scale;
+
+    BinaryDecimalReader(ColumnDescriptor desc, int scale) {
+      super(desc);
+      this.scale = scale;
+    }
+
+    @Override
+    public DecimalData read(DecimalData ignored) {
+      Binary binary = column.nextBinary();
+      BigDecimal bigDecimal = new BigDecimal(new BigInteger(binary.getBytes()), scale);
+      return DecimalData.fromBigDecimal(bigDecimal, bigDecimal.precision(), scale);
+    }
+  }
+
+  private static class IntegerDecimalReader extends ParquetValueReaders.PrimitiveReader<DecimalData> {
+    private final int precision;
+    private final int scale;
+
+    IntegerDecimalReader(ColumnDescriptor desc, int precision, int scale) {
+      super(desc);
+      this.precision = precision;
+      this.scale = scale;
+    }
+
+    @Override
+    public DecimalData read(DecimalData ignored) {
+      return DecimalData.fromUnscaledLong(column.nextInteger(), precision, scale);
+    }
+  }
+
+  private static class LongDecimalReader extends ParquetValueReaders.PrimitiveReader<DecimalData> {
+    private final int precision;
+    private final int scale;
+
+    LongDecimalReader(ColumnDescriptor desc, int precision, int scale) {
+      super(desc);
+      this.precision = precision;
+      this.scale = scale;
+    }
+
+    @Override
+    public DecimalData read(DecimalData ignored) {
+      return DecimalData.fromUnscaledLong(column.nextLong(), precision, scale);
+    }
   }
 
-  public static ParquetValueReader<Row> buildReader(Schema expectedSchema, MessageType fileSchema) {
-    return INSTANCE.createReader(expectedSchema, fileSchema);
+  private static class TimestampMicroReader extends ParquetValueReaders.UnboxedReader<TimestampData> {
+    TimestampMicroReader(ColumnDescriptor desc) {
+      super(desc);
+    }
+
+    @Override
+    public TimestampData read(TimestampData ignored) {
+      long value = readLong();
+      return TimestampData.fromInstant(Instant.ofEpochSecond(value / 1000_000, (value % 1000_000) * 1000));

Review comment:
       This is timestamp with zone or without zone ?  If it's timestamp with zone,  then the `value` could be negative, then we could not just  use `/` and '%',  instead we should use `Math.floorDiv` and `Math.floorMod`.  Please see this pull request: https://github.com/apache/iceberg/pull/1271 .   ( for Java,  `-5/2=-2`,  while `Math.floorDiv(-5, 2)=-3`, actually we need the `-3`  when considering the epoch second). 
   
   Spark's parquet `TimestampMillisReader` don't have this issue because it does not depend any div or mod operation.

##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java
##########
@@ -19,64 +19,719 @@
 
 package org.apache.iceberg.flink.data;
 
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.time.Instant;
 import java.util.List;
-import org.apache.flink.types.Row;
+import java.util.Map;
+import org.apache.commons.lang3.ArrayUtils;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.MapData;
+import org.apache.flink.table.data.RawValueData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
 import org.apache.iceberg.Schema;
-import org.apache.iceberg.data.parquet.BaseParquetReaders;
+import org.apache.iceberg.parquet.ParquetSchemaUtil;
 import org.apache.iceberg.parquet.ParquetValueReader;
 import org.apache.iceberg.parquet.ParquetValueReaders;
+import org.apache.iceberg.parquet.TypeWithSchemaVisitor;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
 import org.apache.iceberg.types.Types;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.io.api.Binary;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.LogicalTypeAnnotation.DecimalLogicalTypeAnnotation;
 import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
 import org.apache.parquet.schema.Type;
 
-public class FlinkParquetReaders extends BaseParquetReaders<Row> {
+class FlinkParquetReaders {
+  private FlinkParquetReaders() {
+  }
 
-  private static final FlinkParquetReaders INSTANCE = new FlinkParquetReaders();
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema, MessageType fileSchema) {
+    return buildReader(expectedSchema, fileSchema, ImmutableMap.of());
+  }
 
-  private FlinkParquetReaders() {
+  @SuppressWarnings("unchecked")
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema,
+                                                        MessageType fileSchema,
+                                                        Map<Integer, ?> idToConstant) {
+    ReadBuilder builder = new ReadBuilder(fileSchema, idToConstant);
+    if (ParquetSchemaUtil.hasIds(fileSchema)) {
+      return (ParquetValueReader<RowData>)
+          TypeWithSchemaVisitor.visit(expectedSchema.asStruct(), fileSchema, builder);
+    } else {
+      return (ParquetValueReader<RowData>)
+          TypeWithSchemaVisitor.visit(expectedSchema.asStruct(), fileSchema,
+              new FallbackReadBuilder(builder));
+    }
+  }
+
+  private static class FallbackReadBuilder extends TypeWithSchemaVisitor<ParquetValueReader<?>> {
+    private MessageType type;
+    private final TypeWithSchemaVisitor<ParquetValueReader<?>> builder;
+
+    FallbackReadBuilder(TypeWithSchemaVisitor<ParquetValueReader<?>> builder) {
+      this.builder = builder;
+    }
+
+    @Override
+    public ParquetValueReader<?> message(Types.StructType expected, MessageType message,
+                                         List<ParquetValueReader<?>> fieldReaders) {
+      // the top level matches by ID, but the remaining IDs are missing
+      this.type = message;
+      return builder.struct(expected, message, fieldReaders);
+    }
+
+    @Override
+    public ParquetValueReader<?> struct(Types.StructType ignored, GroupType struct,
+                                        List<ParquetValueReader<?>> fieldReaders) {
+      // the expected struct is ignored because nested fields are never found when the IDs are missing
+      List<ParquetValueReader<?>> newFields = Lists.newArrayListWithExpectedSize(
+          fieldReaders.size());
+      List<Type> types = Lists.newArrayListWithExpectedSize(fieldReaders.size());
+      List<Type> fields = struct.getFields();
+      for (int i = 0; i < fields.size(); i += 1) {
+        Type fieldType = fields.get(i);
+        int fieldD = type.getMaxDefinitionLevel(path(fieldType.getName())) - 1;
+        newFields.add(ParquetValueReaders.option(fieldType, fieldD, fieldReaders.get(i)));
+        types.add(fieldType);
+      }
+
+      return new RowDataReader(types, newFields);
+    }
+  }
+
+  private static class ReadBuilder extends TypeWithSchemaVisitor<ParquetValueReader<?>> {
+    private final MessageType type;
+    private final Map<Integer, ?> idToConstant;
+
+    ReadBuilder(MessageType type, Map<Integer, ?> idToConstant) {
+      this.type = type;
+      this.idToConstant = idToConstant;
+    }
+
+    @Override
+    public ParquetValueReader<?> message(Types.StructType expected, MessageType message,
+                                         List<ParquetValueReader<?>> fieldReaders) {
+      return struct(expected, message.asGroupType(), fieldReaders);
+    }
+
+    @Override
+    public ParquetValueReader<?> struct(Types.StructType expected, GroupType struct,
+                                        List<ParquetValueReader<?>> fieldReaders) {
+      // match the expected struct's order
+      Map<Integer, ParquetValueReader<?>> readersById = Maps.newHashMap();
+      Map<Integer, Type> typesById = Maps.newHashMap();
+      List<Type> fields = struct.getFields();
+      for (int i = 0; i < fields.size(); i += 1) {
+        Type fieldType = fields.get(i);
+        int fieldD = type.getMaxDefinitionLevel(path(fieldType.getName())) - 1;
+        if (fieldType.getId() != null) {
+          int id = fieldType.getId().intValue();
+          readersById.put(id, ParquetValueReaders.option(fieldType, fieldD, fieldReaders.get(i)));
+          typesById.put(id, fieldType);
+        }
+      }
+
+      List<Types.NestedField> expectedFields = expected != null ?
+          expected.fields() : ImmutableList.of();
+      List<ParquetValueReader<?>> reorderedFields = Lists.newArrayListWithExpectedSize(
+          expectedFields.size());
+      List<Type> types = Lists.newArrayListWithExpectedSize(expectedFields.size());
+      for (Types.NestedField field : expectedFields) {
+        int id = field.fieldId();
+        if (idToConstant.containsKey(id)) {
+          // containsKey is used because the constant may be null
+          reorderedFields.add(ParquetValueReaders.constant(idToConstant.get(id)));
+          types.add(null);
+        } else {
+          ParquetValueReader<?> reader = readersById.get(id);
+          if (reader != null) {
+            reorderedFields.add(reader);
+            types.add(typesById.get(id));
+          } else {
+            reorderedFields.add(ParquetValueReaders.nulls());
+            types.add(null);
+          }
+        }
+      }
+
+      return new RowDataReader(types, reorderedFields);
+    }
+
+    @Override
+    public ParquetValueReader<?> list(Types.ListType expectedList, GroupType array,
+                                      ParquetValueReader<?> elementReader) {
+      GroupType repeated = array.getFields().get(0).asGroupType();
+      String[] repeatedPath = currentPath();
+
+      int repeatedD = type.getMaxDefinitionLevel(repeatedPath) - 1;
+      int repeatedR = type.getMaxRepetitionLevel(repeatedPath) - 1;
+
+      Type elementType = repeated.getType(0);
+      int elementD = type.getMaxDefinitionLevel(path(elementType.getName())) - 1;
+
+      return new ArrayReader<>(repeatedD, repeatedR, ParquetValueReaders.option(elementType, elementD, elementReader));
+    }
+
+    @Override
+    public ParquetValueReader<?> map(Types.MapType expectedMap, GroupType map,
+                                     ParquetValueReader<?> keyReader,
+                                     ParquetValueReader<?> valueReader) {
+      GroupType repeatedKeyValue = map.getFields().get(0).asGroupType();
+      String[] repeatedPath = currentPath();
+
+      int repeatedD = type.getMaxDefinitionLevel(repeatedPath) - 1;
+      int repeatedR = type.getMaxRepetitionLevel(repeatedPath) - 1;
+
+      Type keyType = repeatedKeyValue.getType(0);
+      int keyD = type.getMaxDefinitionLevel(path(keyType.getName())) - 1;
+      Type valueType = repeatedKeyValue.getType(1);
+      int valueD = type.getMaxDefinitionLevel(path(valueType.getName())) - 1;
+
+      return new MapReader<>(repeatedD, repeatedR,
+          ParquetValueReaders.option(keyType, keyD, keyReader),
+          ParquetValueReaders.option(valueType, valueD, valueReader));
+    }
+
+    @Override
+    public ParquetValueReader<?> primitive(org.apache.iceberg.types.Type.PrimitiveType expected,
+                                           PrimitiveType primitive) {
+      ColumnDescriptor desc = type.getColumnDescription(currentPath());
+
+      if (primitive.getOriginalType() != null) {
+        switch (primitive.getOriginalType()) {
+          case ENUM:
+          case JSON:
+          case UTF8:
+            return new StringReader(desc);
+          case INT_8:
+          case INT_16:
+          case INT_32:
+          case DATE:
+            if (expected != null && expected.typeId() == Types.LongType.get().typeId()) {
+              return new ParquetValueReaders.IntAsLongReader(desc);
+            } else {
+              return new ParquetValueReaders.UnboxedReader<>(desc);
+            }
+          case TIME_MICROS:
+            return new TimeMillisReader(desc);
+          case INT_64:
+            return new ParquetValueReaders.UnboxedReader<>(desc);
+          case TIMESTAMP_MICROS:
+            return new TimestampMicroReader(desc);
+          case DECIMAL:
+            DecimalLogicalTypeAnnotation decimal = (DecimalLogicalTypeAnnotation) primitive.getLogicalTypeAnnotation();
+            switch (primitive.getPrimitiveTypeName()) {
+              case BINARY:
+              case FIXED_LEN_BYTE_ARRAY:
+                return new BinaryDecimalReader(desc, decimal.getScale());
+              case INT64:
+                return new LongDecimalReader(desc, decimal.getPrecision(), decimal.getScale());
+              case INT32:
+                return new IntegerDecimalReader(desc, decimal.getPrecision(), decimal.getScale());
+              default:
+                throw new UnsupportedOperationException(
+                    "Unsupported base type for decimal: " + primitive.getPrimitiveTypeName());
+            }
+          case BSON:
+            return new ParquetValueReaders.ByteArrayReader(desc);
+          default:
+            throw new UnsupportedOperationException(
+                "Unsupported logical type: " + primitive.getOriginalType());
+        }
+      }
+
+      switch (primitive.getPrimitiveTypeName()) {
+        case FIXED_LEN_BYTE_ARRAY:
+        case BINARY:
+          return new ParquetValueReaders.ByteArrayReader(desc);
+        case INT32:
+          if (expected != null && expected.typeId() == org.apache.iceberg.types.Type.TypeID.LONG) {
+            return new ParquetValueReaders.IntAsLongReader(desc);
+          } else {
+            return new ParquetValueReaders.UnboxedReader<>(desc);
+          }
+        case FLOAT:
+          if (expected != null && expected.typeId() == org.apache.iceberg.types.Type.TypeID.DOUBLE) {
+            return new ParquetValueReaders.FloatAsDoubleReader(desc);
+          } else {
+            return new ParquetValueReaders.UnboxedReader<>(desc);
+          }
+        case BOOLEAN:
+        case INT64:
+        case DOUBLE:
+          return new ParquetValueReaders.UnboxedReader<>(desc);
+        default:
+          throw new UnsupportedOperationException("Unsupported type: " + primitive);
+      }
+    }
+  }
+
+  private static class BinaryDecimalReader extends ParquetValueReaders.PrimitiveReader<DecimalData> {
+    private final int scale;
+
+    BinaryDecimalReader(ColumnDescriptor desc, int scale) {
+      super(desc);
+      this.scale = scale;
+    }
+
+    @Override
+    public DecimalData read(DecimalData ignored) {
+      Binary binary = column.nextBinary();
+      BigDecimal bigDecimal = new BigDecimal(new BigInteger(binary.getBytes()), scale);
+      return DecimalData.fromBigDecimal(bigDecimal, bigDecimal.precision(), scale);
+    }
+  }
+
+  private static class IntegerDecimalReader extends ParquetValueReaders.PrimitiveReader<DecimalData> {
+    private final int precision;
+    private final int scale;
+
+    IntegerDecimalReader(ColumnDescriptor desc, int precision, int scale) {
+      super(desc);
+      this.precision = precision;
+      this.scale = scale;
+    }
+
+    @Override
+    public DecimalData read(DecimalData ignored) {
+      return DecimalData.fromUnscaledLong(column.nextInteger(), precision, scale);
+    }
+  }
+
+  private static class LongDecimalReader extends ParquetValueReaders.PrimitiveReader<DecimalData> {
+    private final int precision;
+    private final int scale;
+
+    LongDecimalReader(ColumnDescriptor desc, int precision, int scale) {
+      super(desc);
+      this.precision = precision;
+      this.scale = scale;
+    }
+
+    @Override
+    public DecimalData read(DecimalData ignored) {
+      return DecimalData.fromUnscaledLong(column.nextLong(), precision, scale);
+    }
   }
 
-  public static ParquetValueReader<Row> buildReader(Schema expectedSchema, MessageType fileSchema) {
-    return INSTANCE.createReader(expectedSchema, fileSchema);
+  private static class TimestampMicroReader extends ParquetValueReaders.UnboxedReader<TimestampData> {
+    TimestampMicroReader(ColumnDescriptor desc) {
+      super(desc);
+    }
+
+    @Override
+    public TimestampData read(TimestampData ignored) {
+      long value = readLong();
+      return TimestampData.fromInstant(Instant.ofEpochSecond(value / 1000_000, (value % 1000_000) * 1000));
+    }
+
+    @Override
+    public long readLong() {
+      return column.nextLong();
+    }
+  }
+
+  private static class StringReader extends ParquetValueReaders.PrimitiveReader<StringData> {
+    StringReader(ColumnDescriptor desc) {
+      super(desc);
+    }
+
+    @Override
+    public StringData read(StringData ignored) {
+      Binary binary = column.nextBinary();
+      ByteBuffer buffer = binary.toByteBuffer();
+      if (buffer.hasArray()) {
+        return StringData.fromBytes(
+            buffer.array(), buffer.arrayOffset() + buffer.position(), buffer.remaining());
+      } else {
+        return StringData.fromBytes(binary.getBytes());
+      }
+    }
   }
 
-  @Override
-  protected ParquetValueReader<Row> createStructReader(List<Type> types,
-                                                       List<ParquetValueReader<?>> fieldReaders,
-                                                       Types.StructType structType) {
-    return new RowReader(types, fieldReaders, structType);
+  private static class TimeMillisReader extends ParquetValueReaders.PrimitiveReader<Integer> {
+    TimeMillisReader(ColumnDescriptor desc) {
+      super(desc);
+    }
+
+    @Override
+    public Integer read(Integer reuse) {
+      // Flink only supports millisecond, so we discard microseconds in millisecond
+      return (int) column.nextLong() / 1000;
+    }
   }
 
-  private static class RowReader extends ParquetValueReaders.StructReader<Row, Row> {
-    private final Types.StructType structType;
+  private static class ArrayReader<E> extends ParquetValueReaders.RepeatedReader<ArrayData, ReusableArrayData, E> {
+    private int readPos = 0;
+    private int writePos = 0;
+
+    ArrayReader(int definitionLevel, int repetitionLevel, ParquetValueReader<E> reader) {
+      super(definitionLevel, repetitionLevel, reader);
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    protected ReusableArrayData newListData(ArrayData reuse) {
+      this.readPos = 0;
+      this.writePos = 0;
+
+      if (reuse instanceof ReusableArrayData) {
+        return (ReusableArrayData) reuse;
+      } else {
+        return new ReusableArrayData();
+      }
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    protected E getElement(ReusableArrayData list) {
+      E value = null;
+      if (readPos < list.capacity()) {
+        value = (E) list.values[readPos];
+      }
+
+      readPos += 1;
+
+      return value;
+    }
+
+    @Override
+    protected void addElement(ReusableArrayData reused, E element) {
+      if (writePos >= reused.capacity()) {
+        reused.grow();
+      }
+
+      reused.values[writePos] = element;
+
+      writePos += 1;
+    }
+
+    @Override
+    protected ArrayData buildList(ReusableArrayData list) {
+      list.setNumElements(writePos);
+      return list;
+    }
+  }
+
+  private static class MapReader<K, V> extends
+      ParquetValueReaders.RepeatedKeyValueReader<MapData, ReusableMapData, K, V> {
+    private int readPos = 0;
+    private int writePos = 0;
+
+    private final ParquetValueReaders.ReusableEntry<K, V> entry = new ParquetValueReaders.ReusableEntry<>();
+    private final ParquetValueReaders.ReusableEntry<K, V> nullEntry = new ParquetValueReaders.ReusableEntry<>();
+
+    MapReader(int definitionLevel, int repetitionLevel,
+              ParquetValueReader<K> keyReader, ParquetValueReader<V> valueReader) {
+      super(definitionLevel, repetitionLevel, keyReader, valueReader);
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    protected ReusableMapData newMapData(MapData reuse) {
+      this.readPos = 0;
+      this.writePos = 0;
+
+      if (reuse instanceof ReusableMapData) {
+        return (ReusableMapData) reuse;
+      } else {
+        return new ReusableMapData();
+      }
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    protected Map.Entry<K, V> getPair(ReusableMapData map) {
+      Map.Entry<K, V> kv = nullEntry;
+      if (readPos < map.capacity()) {
+        entry.set((K) map.keys.values[readPos], (V) map.values.values[readPos]);
+        kv = entry;
+      }
+
+      readPos += 1;
+
+      return kv;
+    }
+
+    @Override
+    protected void addPair(ReusableMapData map, K key, V value) {
+      if (writePos >= map.capacity()) {
+        map.grow();
+      }
+
+      map.keys.values[writePos] = key;
+      map.values.values[writePos] = value;
 
-    RowReader(List<Type> types, List<ParquetValueReader<?>> readers, Types.StructType struct) {
+      writePos += 1;
+    }
+
+    @Override
+    protected MapData buildMap(ReusableMapData map) {
+      map.setNumElements(writePos);
+      return map;
+    }
+  }
+
+  private static class RowDataReader extends ParquetValueReaders.StructReader<RowData, GenericRowData> {
+    private final int numFields;
+
+    RowDataReader(List<Type> types, List<ParquetValueReader<?>> readers) {
       super(types, readers);
-      this.structType = struct;
+      this.numFields = readers.size();
     }
 
     @Override
-    protected Row newStructData(Row reuse) {
-      if (reuse != null) {
-        return reuse;
+    protected GenericRowData newStructData(RowData reuse) {
+      if (reuse instanceof GenericRowData) {
+        return (GenericRowData) reuse;
       } else {
-        return new Row(structType.fields().size());
+        return new GenericRowData(numFields);
       }
     }
 
     @Override
-    protected Object getField(Row row, int pos) {
-      return row.getField(pos);
+    protected Object getField(GenericRowData intermediate, int pos) {
+      return intermediate.getField(pos);
+    }
+
+    @Override
+    protected RowData buildStruct(GenericRowData struct) {
+      return struct;
+    }
+
+    @Override
+    protected void set(GenericRowData row, int pos, Object value) {
+      row.setField(pos, value);
+    }
+
+    @Override
+    protected void setNull(GenericRowData row, int pos) {
+      row.setField(pos, null);
     }
 
     @Override
-    protected Row buildStruct(Row row) {
-      return row;
+    protected void setBoolean(GenericRowData row, int pos, boolean value) {
+      row.setField(pos, value);
+    }
+
+    @Override
+    protected void setInteger(GenericRowData row, int pos, int value) {
+      row.setField(pos, value);
+    }
+
+    @Override
+    protected void setLong(GenericRowData row, int pos, long value) {
+      row.setField(pos, value);
     }
 
     @Override
-    protected void set(Row row, int pos, Object value) {
+    protected void setFloat(GenericRowData row, int pos, float value) {
       row.setField(pos, value);
     }
+
+    @Override
+    protected void setDouble(GenericRowData row, int pos, double value) {
+      row.setField(pos, value);
+    }
+  }
+
+  private static class ReusableMapData implements MapData {
+    private final ReusableArrayData keys;
+    private final ReusableArrayData values;
+
+    private int numElements;
+
+    private ReusableMapData() {
+      this.keys = new ReusableArrayData();
+      this.values = new ReusableArrayData();
+    }
+
+    private void grow() {
+      keys.grow();
+      values.grow();
+    }
+
+    private int capacity() {
+      return keys.capacity();
+    }
+
+    public void setNumElements(int numElements) {
+      this.numElements = numElements;
+      keys.setNumElements(numElements);
+      values.setNumElements(numElements);
+    }
+
+    @Override
+    public int size() {
+      return numElements;
+    }
+
+    @Override
+    public ReusableArrayData keyArray() {
+      return keys;
+    }
+
+    @Override
+    public ReusableArrayData valueArray() {
+      return values;
+    }
+  }
+
+  private static class ReusableArrayData implements ArrayData {
+    private static final Object[] EMPTY = new Object[0];
+
+    private Object[] values = EMPTY;
+    private int numElements = 0;
+
+    private void grow() {
+      if (values.length == 0) {
+        this.values = new Object[20];
+      } else {
+        Object[] old = values;
+        this.values = new Object[old.length << 2];

Review comment:
       Is this right ?    should be `old.length << 1`  ?    CC @rdblue ,  spark's ReusableArrayData have the same issue ? 




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #1266: Flink: update parquet reader with schema visitor

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1266:
URL: https://github.com/apache/iceberg/pull/1266#discussion_r466923059



##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java
##########
@@ -19,64 +19,701 @@
 
 package org.apache.iceberg.flink.data;
 
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.time.Instant;
+import java.time.ZoneOffset;
 import java.util.List;
-import org.apache.flink.types.Row;
+import java.util.Map;
+import org.apache.commons.lang3.ArrayUtils;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.MapData;
+import org.apache.flink.table.data.RawValueData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
 import org.apache.iceberg.Schema;
-import org.apache.iceberg.data.parquet.BaseParquetReaders;
 import org.apache.iceberg.parquet.ParquetValueReader;
 import org.apache.iceberg.parquet.ParquetValueReaders;
+import org.apache.iceberg.parquet.TypeWithSchemaVisitor;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
 import org.apache.iceberg.types.Types;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.io.api.Binary;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.LogicalTypeAnnotation.DecimalLogicalTypeAnnotation;
 import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
 import org.apache.parquet.schema.Type;
 
-public class FlinkParquetReaders extends BaseParquetReaders<Row> {
+class FlinkParquetReaders {
+  private FlinkParquetReaders() {
+  }
 
-  private static final FlinkParquetReaders INSTANCE = new FlinkParquetReaders();
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema, MessageType fileSchema) {
+    return buildReader(expectedSchema, fileSchema, ImmutableMap.of());
+  }
 
-  private FlinkParquetReaders() {
+  @SuppressWarnings("unchecked")
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema,
+                                                        MessageType fileSchema,
+                                                        Map<Integer, ?> idToConstant) {
+    return (ParquetValueReader<RowData>) TypeWithSchemaVisitor.visit(expectedSchema.asStruct(), fileSchema,
+        new ReadBuilder(fileSchema, idToConstant)
+    );
+  }
+
+  private static class ReadBuilder extends TypeWithSchemaVisitor<ParquetValueReader<?>> {
+    private final MessageType type;
+    private final Map<Integer, ?> idToConstant;
+
+    ReadBuilder(MessageType type, Map<Integer, ?> idToConstant) {
+      this.type = type;
+      this.idToConstant = idToConstant;
+    }
+
+    @Override
+    public ParquetValueReader<RowData> message(Types.StructType expected, MessageType message,
+                                               List<ParquetValueReader<?>> fieldReaders) {
+      return struct(expected, message.asGroupType(), fieldReaders);
+    }
+
+    @Override
+    public ParquetValueReader<RowData> struct(Types.StructType expected, GroupType struct,
+                                              List<ParquetValueReader<?>> fieldReaders) {
+      // match the expected struct's order
+      Map<Integer, ParquetValueReader<?>> readersById = Maps.newHashMap();
+      Map<Integer, Type> typesById = Maps.newHashMap();
+      List<Type> fields = struct.getFields();
+      for (int i = 0; i < fields.size(); i += 1) {
+        Type fieldType = fields.get(i);
+        int fieldD = type.getMaxDefinitionLevel(path(fieldType.getName())) - 1;
+        if (fieldType.getId() != null) {
+          int id = fieldType.getId().intValue();
+          readersById.put(id, ParquetValueReaders.option(fieldType, fieldD, fieldReaders.get(i)));
+          typesById.put(id, fieldType);
+        }
+      }
+
+      List<Types.NestedField> expectedFields = expected != null ?
+          expected.fields() : ImmutableList.of();
+      List<ParquetValueReader<?>> reorderedFields = Lists.newArrayListWithExpectedSize(
+          expectedFields.size());
+      List<Type> types = Lists.newArrayListWithExpectedSize(expectedFields.size());
+      for (Types.NestedField field : expectedFields) {
+        int id = field.fieldId();
+        if (idToConstant.containsKey(id)) {
+          // containsKey is used because the constant may be null
+          reorderedFields.add(ParquetValueReaders.constant(idToConstant.get(id)));
+          types.add(null);
+        } else {
+          ParquetValueReader<?> reader = readersById.get(id);
+          if (reader != null) {
+            reorderedFields.add(reader);
+            types.add(typesById.get(id));
+          } else {
+            reorderedFields.add(ParquetValueReaders.nulls());
+            types.add(null);
+          }
+        }
+      }
+
+      return new RowDataReader(types, reorderedFields);
+    }
+
+    @Override
+    public ParquetValueReader<?> list(Types.ListType expectedList, GroupType array,
+                                      ParquetValueReader<?> elementReader) {
+      GroupType repeated = array.getFields().get(0).asGroupType();
+      String[] repeatedPath = currentPath();
+
+      int repeatedD = type.getMaxDefinitionLevel(repeatedPath) - 1;
+      int repeatedR = type.getMaxRepetitionLevel(repeatedPath) - 1;
+
+      Type elementType = repeated.getType(0);
+      int elementD = type.getMaxDefinitionLevel(path(elementType.getName())) - 1;
+
+      return new ArrayReader<>(repeatedD, repeatedR, ParquetValueReaders.option(elementType, elementD, elementReader));
+    }
+
+    @Override
+    public ParquetValueReader<?> map(Types.MapType expectedMap, GroupType map,
+                                     ParquetValueReader<?> keyReader,
+                                     ParquetValueReader<?> valueReader) {
+      GroupType repeatedKeyValue = map.getFields().get(0).asGroupType();
+      String[] repeatedPath = currentPath();
+
+      int repeatedD = type.getMaxDefinitionLevel(repeatedPath) - 1;
+      int repeatedR = type.getMaxRepetitionLevel(repeatedPath) - 1;
+
+      Type keyType = repeatedKeyValue.getType(0);
+      int keyD = type.getMaxDefinitionLevel(path(keyType.getName())) - 1;
+      Type valueType = repeatedKeyValue.getType(1);
+      int valueD = type.getMaxDefinitionLevel(path(valueType.getName())) - 1;
+
+      return new MapReader<>(repeatedD, repeatedR,
+          ParquetValueReaders.option(keyType, keyD, keyReader),
+          ParquetValueReaders.option(valueType, valueD, valueReader));
+    }
+
+    @Override
+    public ParquetValueReader<?> primitive(org.apache.iceberg.types.Type.PrimitiveType expected,
+                                           PrimitiveType primitive) {
+      ColumnDescriptor desc = type.getColumnDescription(currentPath());
+
+      if (primitive.getOriginalType() != null) {
+        switch (primitive.getOriginalType()) {
+          case ENUM:
+          case JSON:
+          case UTF8:
+            return new StringReader(desc);
+          case INT_8:
+          case INT_16:
+          case INT_32:
+          case DATE:
+            if (expected != null && expected.typeId() == Types.LongType.get().typeId()) {
+              return new ParquetValueReaders.IntAsLongReader(desc);
+            } else {
+              return new ParquetValueReaders.UnboxedReader<>(desc);
+            }
+          case TIME_MICROS:
+            return new LossyMicrosToMillisTimeReader(desc);
+          case INT_64:
+            return new ParquetValueReaders.UnboxedReader<>(desc);
+          case TIMESTAMP_MICROS:
+            if (((Types.TimestampType) expected).shouldAdjustToUTC()) {
+              return new TimestampTzReader(desc);
+            } else {
+              return new TimestampReader(desc);
+            }
+          case DECIMAL:
+            DecimalLogicalTypeAnnotation decimal = (DecimalLogicalTypeAnnotation) primitive.getLogicalTypeAnnotation();
+            switch (primitive.getPrimitiveTypeName()) {
+              case BINARY:
+              case FIXED_LEN_BYTE_ARRAY:
+                return new BinaryDecimalReader(desc, decimal.getScale());
+              case INT64:
+                return new LongDecimalReader(desc, decimal.getPrecision(), decimal.getScale());
+              case INT32:
+                return new IntegerDecimalReader(desc, decimal.getPrecision(), decimal.getScale());
+              default:
+                throw new UnsupportedOperationException(
+                    "Unsupported base type for decimal: " + primitive.getPrimitiveTypeName());
+            }
+          case BSON:
+            return new ParquetValueReaders.ByteArrayReader(desc);
+          default:
+            throw new UnsupportedOperationException(
+                "Unsupported logical type: " + primitive.getOriginalType());
+        }
+      }
+
+      switch (primitive.getPrimitiveTypeName()) {
+        case FIXED_LEN_BYTE_ARRAY:
+        case BINARY:
+          return new ParquetValueReaders.ByteArrayReader(desc);
+        case INT32:
+          if (expected != null && expected.typeId() == org.apache.iceberg.types.Type.TypeID.LONG) {
+            return new ParquetValueReaders.IntAsLongReader(desc);
+          } else {
+            return new ParquetValueReaders.UnboxedReader<>(desc);
+          }
+        case FLOAT:
+          if (expected != null && expected.typeId() == org.apache.iceberg.types.Type.TypeID.DOUBLE) {
+            return new ParquetValueReaders.FloatAsDoubleReader(desc);
+          } else {
+            return new ParquetValueReaders.UnboxedReader<>(desc);
+          }
+        case BOOLEAN:
+        case INT64:
+        case DOUBLE:
+          return new ParquetValueReaders.UnboxedReader<>(desc);
+        default:
+          throw new UnsupportedOperationException("Unsupported type: " + primitive);
+      }
+    }
+  }
+
+  private static class BinaryDecimalReader extends ParquetValueReaders.PrimitiveReader<DecimalData> {
+    private final int scale;
+
+    BinaryDecimalReader(ColumnDescriptor desc, int scale) {
+      super(desc);
+      this.scale = scale;
+    }
+
+    @Override
+    public DecimalData read(DecimalData ignored) {
+      Binary binary = column.nextBinary();
+      BigDecimal bigDecimal = new BigDecimal(new BigInteger(binary.getBytes()), scale);
+      return DecimalData.fromBigDecimal(bigDecimal, bigDecimal.precision(), scale);
+    }
+  }
+
+  private static class IntegerDecimalReader extends ParquetValueReaders.PrimitiveReader<DecimalData> {
+    private final int precision;
+    private final int scale;
+
+    IntegerDecimalReader(ColumnDescriptor desc, int precision, int scale) {
+      super(desc);
+      this.precision = precision;
+      this.scale = scale;
+    }
+
+    @Override
+    public DecimalData read(DecimalData ignored) {
+      return DecimalData.fromUnscaledLong(column.nextInteger(), precision, scale);
+    }
+  }
+
+  private static class LongDecimalReader extends ParquetValueReaders.PrimitiveReader<DecimalData> {
+    private final int precision;
+    private final int scale;
+
+    LongDecimalReader(ColumnDescriptor desc, int precision, int scale) {
+      super(desc);
+      this.precision = precision;
+      this.scale = scale;
+    }
+
+    @Override
+    public DecimalData read(DecimalData ignored) {
+      return DecimalData.fromUnscaledLong(column.nextLong(), precision, scale);
+    }
+  }
+
+  private static class TimestampTzReader extends ParquetValueReaders.UnboxedReader<TimestampData> {
+    TimestampTzReader(ColumnDescriptor desc) {
+      super(desc);
+    }
+
+    @Override
+    public TimestampData read(TimestampData ignored) {
+      long value = readLong();
+      return TimestampData.fromLocalDateTime(Instant.ofEpochSecond(Math.floorDiv(value, 1000_000),
+          Math.floorMod(value, 1000_000) * 1000)
+          .atOffset(ZoneOffset.UTC)
+          .toLocalDateTime());

Review comment:
       OK. But I think we could construct the `TimestampData` by a simply way like following: 
   
   ```java
         long micros = readLong();
         return TimestampData.fromEpochMillis(Math.floorDiv(micros, 1_000),
             (int) Math.floorMod(micros, 1_000) * 1_000);
   ```
   
   BTW,  we don't have two timestamp readers: `TimestampTzReader` and `TimestampReader`.  Because for both cases, we just read the timestamp into a `TimestampData`  and there's no difference.  So we could only keep one timestamp reader,  name it as `TimestampMicrosReader`.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on pull request #1266: Flink: update parquet reader with schema visitor

Posted by GitBox <gi...@apache.org>.
rdblue commented on pull request #1266:
URL: https://github.com/apache/iceberg/pull/1266#issuecomment-671663866


   I should have time to review this tomorrow, or Wednesday at the latest. Thanks, @chenjunjiedada!


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #1266: Flink: update parquet reader with schema visitor

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1266:
URL: https://github.com/apache/iceberg/pull/1266#discussion_r466924207



##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java
##########
@@ -19,64 +19,701 @@
 
 package org.apache.iceberg.flink.data;
 
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.time.Instant;
+import java.time.ZoneOffset;
 import java.util.List;
-import org.apache.flink.types.Row;
+import java.util.Map;
+import org.apache.commons.lang3.ArrayUtils;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.MapData;
+import org.apache.flink.table.data.RawValueData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
 import org.apache.iceberg.Schema;
-import org.apache.iceberg.data.parquet.BaseParquetReaders;
 import org.apache.iceberg.parquet.ParquetValueReader;
 import org.apache.iceberg.parquet.ParquetValueReaders;
+import org.apache.iceberg.parquet.TypeWithSchemaVisitor;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
 import org.apache.iceberg.types.Types;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.io.api.Binary;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.LogicalTypeAnnotation.DecimalLogicalTypeAnnotation;
 import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
 import org.apache.parquet.schema.Type;
 
-public class FlinkParquetReaders extends BaseParquetReaders<Row> {
+class FlinkParquetReaders {
+  private FlinkParquetReaders() {
+  }
 
-  private static final FlinkParquetReaders INSTANCE = new FlinkParquetReaders();
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema, MessageType fileSchema) {
+    return buildReader(expectedSchema, fileSchema, ImmutableMap.of());
+  }
 
-  private FlinkParquetReaders() {
+  @SuppressWarnings("unchecked")
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema,
+                                                        MessageType fileSchema,
+                                                        Map<Integer, ?> idToConstant) {
+    return (ParquetValueReader<RowData>) TypeWithSchemaVisitor.visit(expectedSchema.asStruct(), fileSchema,
+        new ReadBuilder(fileSchema, idToConstant)
+    );
+  }
+
+  private static class ReadBuilder extends TypeWithSchemaVisitor<ParquetValueReader<?>> {
+    private final MessageType type;
+    private final Map<Integer, ?> idToConstant;
+
+    ReadBuilder(MessageType type, Map<Integer, ?> idToConstant) {
+      this.type = type;
+      this.idToConstant = idToConstant;
+    }
+
+    @Override
+    public ParquetValueReader<RowData> message(Types.StructType expected, MessageType message,
+                                               List<ParquetValueReader<?>> fieldReaders) {
+      return struct(expected, message.asGroupType(), fieldReaders);
+    }
+
+    @Override
+    public ParquetValueReader<RowData> struct(Types.StructType expected, GroupType struct,
+                                              List<ParquetValueReader<?>> fieldReaders) {
+      // match the expected struct's order
+      Map<Integer, ParquetValueReader<?>> readersById = Maps.newHashMap();
+      Map<Integer, Type> typesById = Maps.newHashMap();
+      List<Type> fields = struct.getFields();
+      for (int i = 0; i < fields.size(); i += 1) {
+        Type fieldType = fields.get(i);
+        int fieldD = type.getMaxDefinitionLevel(path(fieldType.getName())) - 1;
+        if (fieldType.getId() != null) {
+          int id = fieldType.getId().intValue();
+          readersById.put(id, ParquetValueReaders.option(fieldType, fieldD, fieldReaders.get(i)));
+          typesById.put(id, fieldType);
+        }
+      }
+
+      List<Types.NestedField> expectedFields = expected != null ?
+          expected.fields() : ImmutableList.of();
+      List<ParquetValueReader<?>> reorderedFields = Lists.newArrayListWithExpectedSize(
+          expectedFields.size());
+      List<Type> types = Lists.newArrayListWithExpectedSize(expectedFields.size());
+      for (Types.NestedField field : expectedFields) {
+        int id = field.fieldId();
+        if (idToConstant.containsKey(id)) {
+          // containsKey is used because the constant may be null
+          reorderedFields.add(ParquetValueReaders.constant(idToConstant.get(id)));
+          types.add(null);
+        } else {
+          ParquetValueReader<?> reader = readersById.get(id);
+          if (reader != null) {
+            reorderedFields.add(reader);
+            types.add(typesById.get(id));
+          } else {
+            reorderedFields.add(ParquetValueReaders.nulls());
+            types.add(null);
+          }
+        }
+      }
+
+      return new RowDataReader(types, reorderedFields);
+    }
+
+    @Override
+    public ParquetValueReader<?> list(Types.ListType expectedList, GroupType array,
+                                      ParquetValueReader<?> elementReader) {
+      GroupType repeated = array.getFields().get(0).asGroupType();
+      String[] repeatedPath = currentPath();
+
+      int repeatedD = type.getMaxDefinitionLevel(repeatedPath) - 1;
+      int repeatedR = type.getMaxRepetitionLevel(repeatedPath) - 1;
+
+      Type elementType = repeated.getType(0);
+      int elementD = type.getMaxDefinitionLevel(path(elementType.getName())) - 1;
+
+      return new ArrayReader<>(repeatedD, repeatedR, ParquetValueReaders.option(elementType, elementD, elementReader));
+    }
+
+    @Override
+    public ParquetValueReader<?> map(Types.MapType expectedMap, GroupType map,
+                                     ParquetValueReader<?> keyReader,
+                                     ParquetValueReader<?> valueReader) {
+      GroupType repeatedKeyValue = map.getFields().get(0).asGroupType();
+      String[] repeatedPath = currentPath();
+
+      int repeatedD = type.getMaxDefinitionLevel(repeatedPath) - 1;
+      int repeatedR = type.getMaxRepetitionLevel(repeatedPath) - 1;
+
+      Type keyType = repeatedKeyValue.getType(0);
+      int keyD = type.getMaxDefinitionLevel(path(keyType.getName())) - 1;
+      Type valueType = repeatedKeyValue.getType(1);
+      int valueD = type.getMaxDefinitionLevel(path(valueType.getName())) - 1;
+
+      return new MapReader<>(repeatedD, repeatedR,
+          ParquetValueReaders.option(keyType, keyD, keyReader),
+          ParquetValueReaders.option(valueType, valueD, valueReader));
+    }
+
+    @Override
+    public ParquetValueReader<?> primitive(org.apache.iceberg.types.Type.PrimitiveType expected,
+                                           PrimitiveType primitive) {
+      ColumnDescriptor desc = type.getColumnDescription(currentPath());
+
+      if (primitive.getOriginalType() != null) {
+        switch (primitive.getOriginalType()) {
+          case ENUM:
+          case JSON:
+          case UTF8:
+            return new StringReader(desc);
+          case INT_8:
+          case INT_16:
+          case INT_32:
+          case DATE:
+            if (expected != null && expected.typeId() == Types.LongType.get().typeId()) {
+              return new ParquetValueReaders.IntAsLongReader(desc);
+            } else {
+              return new ParquetValueReaders.UnboxedReader<>(desc);
+            }
+          case TIME_MICROS:
+            return new LossyMicrosToMillisTimeReader(desc);
+          case INT_64:
+            return new ParquetValueReaders.UnboxedReader<>(desc);
+          case TIMESTAMP_MICROS:
+            if (((Types.TimestampType) expected).shouldAdjustToUTC()) {
+              return new TimestampTzReader(desc);
+            } else {
+              return new TimestampReader(desc);
+            }
+          case DECIMAL:
+            DecimalLogicalTypeAnnotation decimal = (DecimalLogicalTypeAnnotation) primitive.getLogicalTypeAnnotation();
+            switch (primitive.getPrimitiveTypeName()) {
+              case BINARY:
+              case FIXED_LEN_BYTE_ARRAY:
+                return new BinaryDecimalReader(desc, decimal.getScale());
+              case INT64:
+                return new LongDecimalReader(desc, decimal.getPrecision(), decimal.getScale());
+              case INT32:
+                return new IntegerDecimalReader(desc, decimal.getPrecision(), decimal.getScale());
+              default:
+                throw new UnsupportedOperationException(
+                    "Unsupported base type for decimal: " + primitive.getPrimitiveTypeName());
+            }
+          case BSON:
+            return new ParquetValueReaders.ByteArrayReader(desc);
+          default:
+            throw new UnsupportedOperationException(
+                "Unsupported logical type: " + primitive.getOriginalType());
+        }
+      }
+
+      switch (primitive.getPrimitiveTypeName()) {
+        case FIXED_LEN_BYTE_ARRAY:
+        case BINARY:
+          return new ParquetValueReaders.ByteArrayReader(desc);
+        case INT32:
+          if (expected != null && expected.typeId() == org.apache.iceberg.types.Type.TypeID.LONG) {
+            return new ParquetValueReaders.IntAsLongReader(desc);
+          } else {
+            return new ParquetValueReaders.UnboxedReader<>(desc);
+          }
+        case FLOAT:
+          if (expected != null && expected.typeId() == org.apache.iceberg.types.Type.TypeID.DOUBLE) {
+            return new ParquetValueReaders.FloatAsDoubleReader(desc);
+          } else {
+            return new ParquetValueReaders.UnboxedReader<>(desc);
+          }
+        case BOOLEAN:
+        case INT64:
+        case DOUBLE:
+          return new ParquetValueReaders.UnboxedReader<>(desc);
+        default:
+          throw new UnsupportedOperationException("Unsupported type: " + primitive);
+      }
+    }
+  }
+
+  private static class BinaryDecimalReader extends ParquetValueReaders.PrimitiveReader<DecimalData> {
+    private final int scale;
+
+    BinaryDecimalReader(ColumnDescriptor desc, int scale) {
+      super(desc);
+      this.scale = scale;
+    }
+
+    @Override
+    public DecimalData read(DecimalData ignored) {
+      Binary binary = column.nextBinary();
+      BigDecimal bigDecimal = new BigDecimal(new BigInteger(binary.getBytes()), scale);
+      return DecimalData.fromBigDecimal(bigDecimal, bigDecimal.precision(), scale);
+    }
+  }
+
+  private static class IntegerDecimalReader extends ParquetValueReaders.PrimitiveReader<DecimalData> {
+    private final int precision;
+    private final int scale;
+
+    IntegerDecimalReader(ColumnDescriptor desc, int precision, int scale) {
+      super(desc);
+      this.precision = precision;
+      this.scale = scale;
+    }
+
+    @Override
+    public DecimalData read(DecimalData ignored) {
+      return DecimalData.fromUnscaledLong(column.nextInteger(), precision, scale);
+    }
+  }
+
+  private static class LongDecimalReader extends ParquetValueReaders.PrimitiveReader<DecimalData> {
+    private final int precision;
+    private final int scale;
+
+    LongDecimalReader(ColumnDescriptor desc, int precision, int scale) {
+      super(desc);
+      this.precision = precision;
+      this.scale = scale;
+    }
+
+    @Override
+    public DecimalData read(DecimalData ignored) {
+      return DecimalData.fromUnscaledLong(column.nextLong(), precision, scale);
+    }
+  }
+
+  private static class TimestampTzReader extends ParquetValueReaders.UnboxedReader<TimestampData> {
+    TimestampTzReader(ColumnDescriptor desc) {
+      super(desc);
+    }
+
+    @Override
+    public TimestampData read(TimestampData ignored) {
+      long value = readLong();
+      return TimestampData.fromLocalDateTime(Instant.ofEpochSecond(Math.floorDiv(value, 1000_000),
+          Math.floorMod(value, 1000_000) * 1000)
+          .atOffset(ZoneOffset.UTC)
+          .toLocalDateTime());
+    }
+
+    @Override
+    public long readLong() {
+      return column.nextLong();
+    }
   }
 
-  public static ParquetValueReader<Row> buildReader(Schema expectedSchema, MessageType fileSchema) {
-    return INSTANCE.createReader(expectedSchema, fileSchema);
+  private static class TimestampReader extends ParquetValueReaders.UnboxedReader<TimestampData> {
+    TimestampReader(ColumnDescriptor desc) {
+      super(desc);
+    }
+
+    @Override
+    public TimestampData read(TimestampData ignored) {
+      long value = readLong();
+      return TimestampData.fromInstant(Instant.ofEpochSecond(Math.floorDiv(value, 1000_000),
+          Math.floorMod(value, 1000_000) * 1000));
+    }
+
+    @Override
+    public long readLong() {
+      return column.nextLong();
+    }
   }
 
-  @Override
-  protected ParquetValueReader<Row> createStructReader(List<Type> types,
-                                                       List<ParquetValueReader<?>> fieldReaders,
-                                                       Types.StructType structType) {
-    return new RowReader(types, fieldReaders, structType);
+  private static class StringReader extends ParquetValueReaders.PrimitiveReader<StringData> {
+    StringReader(ColumnDescriptor desc) {
+      super(desc);
+    }
+
+    @Override
+    public StringData read(StringData ignored) {
+      Binary binary = column.nextBinary();
+      ByteBuffer buffer = binary.toByteBuffer();
+      if (buffer.hasArray()) {
+        return StringData.fromBytes(
+            buffer.array(), buffer.arrayOffset() + buffer.position(), buffer.remaining());
+      } else {
+        return StringData.fromBytes(binary.getBytes());
+      }
+    }
+  }
+
+  private static class LossyMicrosToMillisTimeReader extends ParquetValueReaders.PrimitiveReader<Integer> {
+    LossyMicrosToMillisTimeReader(ColumnDescriptor desc) {
+      super(desc);
+    }
+
+    @Override
+    public Integer read(Integer reuse) {
+      // Discard microseconds since Flink uses millisecond unit for TIME type.
+      return (int) Math.floorDiv(column.nextLong(), 1000);
+    }
+  }
+
+  private static class ArrayReader<E> extends ParquetValueReaders.RepeatedReader<ArrayData, ReusableArrayData, E> {
+    private int readPos = 0;
+    private int writePos = 0;
+
+    ArrayReader(int definitionLevel, int repetitionLevel, ParquetValueReader<E> reader) {
+      super(definitionLevel, repetitionLevel, reader);
+    }
+
+    @Override
+    protected ReusableArrayData newListData(ArrayData reuse) {
+      this.readPos = 0;
+      this.writePos = 0;
+
+      if (reuse instanceof ReusableArrayData) {
+        return (ReusableArrayData) reuse;
+      } else {
+        return new ReusableArrayData();
+      }
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    protected E getElement(ReusableArrayData list) {
+      E value = null;
+      if (readPos < list.capacity()) {
+        value = (E) list.values[readPos];
+      }
+
+      readPos += 1;

Review comment:
       OK, you are right. 




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1266: Flink: update parquet reader with schema visitor

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1266:
URL: https://github.com/apache/iceberg/pull/1266#discussion_r469438023



##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java
##########
@@ -19,64 +19,678 @@
 
 package org.apache.iceberg.flink.data;
 
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
 import java.util.List;
-import org.apache.flink.types.Row;
+import java.util.Map;
+import org.apache.commons.lang3.ArrayUtils;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.MapData;
+import org.apache.flink.table.data.RawValueData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
 import org.apache.iceberg.Schema;
-import org.apache.iceberg.data.parquet.BaseParquetReaders;
 import org.apache.iceberg.parquet.ParquetValueReader;
 import org.apache.iceberg.parquet.ParquetValueReaders;
+import org.apache.iceberg.parquet.TypeWithSchemaVisitor;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
 import org.apache.iceberg.types.Types;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.io.api.Binary;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.LogicalTypeAnnotation.DecimalLogicalTypeAnnotation;
 import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
 import org.apache.parquet.schema.Type;
 
-public class FlinkParquetReaders extends BaseParquetReaders<Row> {
+class FlinkParquetReaders {
+  private FlinkParquetReaders() {
+  }
 
-  private static final FlinkParquetReaders INSTANCE = new FlinkParquetReaders();
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema, MessageType fileSchema) {
+    return buildReader(expectedSchema, fileSchema, ImmutableMap.of());
+  }
 
-  private FlinkParquetReaders() {
+  @SuppressWarnings("unchecked")
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema,
+                                                        MessageType fileSchema,
+                                                        Map<Integer, ?> idToConstant) {
+    return (ParquetValueReader<RowData>) TypeWithSchemaVisitor.visit(expectedSchema.asStruct(), fileSchema,
+        new ReadBuilder(fileSchema, idToConstant)
+    );
+  }
+
+  private static class ReadBuilder extends TypeWithSchemaVisitor<ParquetValueReader<?>> {
+    private final MessageType type;
+    private final Map<Integer, ?> idToConstant;
+
+    ReadBuilder(MessageType type, Map<Integer, ?> idToConstant) {
+      this.type = type;
+      this.idToConstant = idToConstant;
+    }
+
+    @Override
+    public ParquetValueReader<RowData> message(Types.StructType expected, MessageType message,
+                                               List<ParquetValueReader<?>> fieldReaders) {
+      return struct(expected, message.asGroupType(), fieldReaders);
+    }
+
+    @Override
+    public ParquetValueReader<RowData> struct(Types.StructType expected, GroupType struct,
+                                              List<ParquetValueReader<?>> fieldReaders) {
+      // match the expected struct's order
+      Map<Integer, ParquetValueReader<?>> readersById = Maps.newHashMap();
+      Map<Integer, Type> typesById = Maps.newHashMap();
+      List<Type> fields = struct.getFields();
+      for (int i = 0; i < fields.size(); i += 1) {
+        Type fieldType = fields.get(i);
+        int fieldD = type.getMaxDefinitionLevel(path(fieldType.getName())) - 1;
+        if (fieldType.getId() != null) {
+          int id = fieldType.getId().intValue();
+          readersById.put(id, ParquetValueReaders.option(fieldType, fieldD, fieldReaders.get(i)));
+          typesById.put(id, fieldType);
+        }
+      }
+
+      List<Types.NestedField> expectedFields = expected != null ?
+          expected.fields() : ImmutableList.of();
+      List<ParquetValueReader<?>> reorderedFields = Lists.newArrayListWithExpectedSize(
+          expectedFields.size());
+      List<Type> types = Lists.newArrayListWithExpectedSize(expectedFields.size());
+      for (Types.NestedField field : expectedFields) {
+        int id = field.fieldId();
+        if (idToConstant.containsKey(id)) {
+          // containsKey is used because the constant may be null
+          reorderedFields.add(ParquetValueReaders.constant(idToConstant.get(id)));
+          types.add(null);
+        } else {
+          ParquetValueReader<?> reader = readersById.get(id);
+          if (reader != null) {
+            reorderedFields.add(reader);
+            types.add(typesById.get(id));
+          } else {
+            reorderedFields.add(ParquetValueReaders.nulls());
+            types.add(null);
+          }
+        }
+      }
+
+      return new RowDataReader(types, reorderedFields);
+    }
+
+    @Override
+    public ParquetValueReader<?> list(Types.ListType expectedList, GroupType array,
+                                      ParquetValueReader<?> elementReader) {
+      GroupType repeated = array.getFields().get(0).asGroupType();
+      String[] repeatedPath = currentPath();
+
+      int repeatedD = type.getMaxDefinitionLevel(repeatedPath) - 1;
+      int repeatedR = type.getMaxRepetitionLevel(repeatedPath) - 1;
+
+      Type elementType = repeated.getType(0);
+      int elementD = type.getMaxDefinitionLevel(path(elementType.getName())) - 1;
+
+      return new ArrayReader<>(repeatedD, repeatedR, ParquetValueReaders.option(elementType, elementD, elementReader));
+    }
+
+    @Override
+    public ParquetValueReader<?> map(Types.MapType expectedMap, GroupType map,
+                                     ParquetValueReader<?> keyReader,
+                                     ParquetValueReader<?> valueReader) {
+      GroupType repeatedKeyValue = map.getFields().get(0).asGroupType();
+      String[] repeatedPath = currentPath();
+
+      int repeatedD = type.getMaxDefinitionLevel(repeatedPath) - 1;
+      int repeatedR = type.getMaxRepetitionLevel(repeatedPath) - 1;
+
+      Type keyType = repeatedKeyValue.getType(0);
+      int keyD = type.getMaxDefinitionLevel(path(keyType.getName())) - 1;
+      Type valueType = repeatedKeyValue.getType(1);
+      int valueD = type.getMaxDefinitionLevel(path(valueType.getName())) - 1;
+
+      return new MapReader<>(repeatedD, repeatedR,
+          ParquetValueReaders.option(keyType, keyD, keyReader),
+          ParquetValueReaders.option(valueType, valueD, valueReader));
+    }
+
+    @Override
+    public ParquetValueReader<?> primitive(org.apache.iceberg.types.Type.PrimitiveType expected,
+                                           PrimitiveType primitive) {
+      ColumnDescriptor desc = type.getColumnDescription(currentPath());
+
+      if (primitive.getOriginalType() != null) {
+        switch (primitive.getOriginalType()) {
+          case ENUM:
+          case JSON:
+          case UTF8:
+            return new StringReader(desc);
+          case INT_8:
+          case INT_16:
+          case INT_32:
+            if (expected != null && expected.typeId() == Types.LongType.get().typeId()) {
+              return new ParquetValueReaders.IntAsLongReader(desc);
+            } else {
+              return new ParquetValueReaders.UnboxedReader<>(desc);
+            }
+          case TIME_MICROS:
+            return new LossyMicrosToMillisTimeReader(desc);
+          case DATE:
+          case INT_64:
+            return new ParquetValueReaders.UnboxedReader<>(desc);
+          case TIMESTAMP_MICROS:
+            return new TimestampMicrosReader(desc);
+          case DECIMAL:
+            DecimalLogicalTypeAnnotation decimal = (DecimalLogicalTypeAnnotation) primitive.getLogicalTypeAnnotation();
+            switch (primitive.getPrimitiveTypeName()) {
+              case BINARY:
+              case FIXED_LEN_BYTE_ARRAY:
+                return new BinaryDecimalReader(desc, decimal.getPrecision(), decimal.getScale());
+              case INT64:
+                return new LongDecimalReader(desc, decimal.getPrecision(), decimal.getScale());
+              case INT32:
+                return new IntegerDecimalReader(desc, decimal.getPrecision(), decimal.getScale());
+              default:
+                throw new UnsupportedOperationException(
+                    "Unsupported base type for decimal: " + primitive.getPrimitiveTypeName());
+            }
+          case BSON:
+            return new ParquetValueReaders.ByteArrayReader(desc);
+          default:
+            throw new UnsupportedOperationException(
+                "Unsupported logical type: " + primitive.getOriginalType());
+        }
+      }
+
+      switch (primitive.getPrimitiveTypeName()) {
+        case FIXED_LEN_BYTE_ARRAY:
+        case BINARY:
+          return new ParquetValueReaders.ByteArrayReader(desc);
+        case INT32:
+          if (expected != null && expected.typeId() == org.apache.iceberg.types.Type.TypeID.LONG) {
+            return new ParquetValueReaders.IntAsLongReader(desc);
+          } else {
+            return new ParquetValueReaders.UnboxedReader<>(desc);
+          }
+        case FLOAT:
+          if (expected != null && expected.typeId() == org.apache.iceberg.types.Type.TypeID.DOUBLE) {
+            return new ParquetValueReaders.FloatAsDoubleReader(desc);
+          } else {
+            return new ParquetValueReaders.UnboxedReader<>(desc);
+          }
+        case BOOLEAN:
+        case INT64:
+        case DOUBLE:
+          return new ParquetValueReaders.UnboxedReader<>(desc);
+        default:
+          throw new UnsupportedOperationException("Unsupported type: " + primitive);
+      }
+    }
+  }
+
+  private static class BinaryDecimalReader extends ParquetValueReaders.PrimitiveReader<DecimalData> {
+    private final int precision;
+    private final int scale;
+
+    BinaryDecimalReader(ColumnDescriptor desc, int precision, int scale) {
+      super(desc);
+      this.precision = precision;
+      this.scale = scale;
+    }
+
+    @Override
+    public DecimalData read(DecimalData ignored) {
+      Binary binary = column.nextBinary();
+      BigDecimal bigDecimal = new BigDecimal(new BigInteger(binary.getBytes()), scale);
+      // TODO: need a unit test to write-read-validate decimal via FlinkParquetWrite/Reader
+      return DecimalData.fromBigDecimal(bigDecimal, precision, scale);
+    }
+  }
+
+  private static class IntegerDecimalReader extends ParquetValueReaders.PrimitiveReader<DecimalData> {
+    private final int precision;
+    private final int scale;
+
+    IntegerDecimalReader(ColumnDescriptor desc, int precision, int scale) {
+      super(desc);
+      this.precision = precision;
+      this.scale = scale;
+    }
+
+    @Override
+    public DecimalData read(DecimalData ignored) {
+      return DecimalData.fromUnscaledLong(column.nextInteger(), precision, scale);
+    }
   }
 
-  public static ParquetValueReader<Row> buildReader(Schema expectedSchema, MessageType fileSchema) {
-    return INSTANCE.createReader(expectedSchema, fileSchema);
+  private static class LongDecimalReader extends ParquetValueReaders.PrimitiveReader<DecimalData> {
+    private final int precision;
+    private final int scale;
+
+    LongDecimalReader(ColumnDescriptor desc, int precision, int scale) {
+      super(desc);
+      this.precision = precision;
+      this.scale = scale;
+    }
+
+    @Override
+    public DecimalData read(DecimalData ignored) {
+      return DecimalData.fromUnscaledLong(column.nextLong(), precision, scale);
+    }
   }
 
-  @Override
-  protected ParquetValueReader<Row> createStructReader(List<Type> types,
-                                                       List<ParquetValueReader<?>> fieldReaders,
-                                                       Types.StructType structType) {
-    return new RowReader(types, fieldReaders, structType);
+  private static class TimestampMicrosReader extends ParquetValueReaders.UnboxedReader<TimestampData> {
+    TimestampMicrosReader(ColumnDescriptor desc) {
+      super(desc);
+    }
+
+    @Override
+    public TimestampData read(TimestampData ignored) {
+      long micros = readLong();
+      return TimestampData.fromEpochMillis(Math.floorDiv(micros, 1_000),
+          (int) Math.floorMod(micros, 1_000) * 1_000);
+    }
+
+    @Override
+    public long readLong() {
+      return column.nextLong();
+    }
+  }
+
+  private static class StringReader extends ParquetValueReaders.PrimitiveReader<StringData> {
+    StringReader(ColumnDescriptor desc) {
+      super(desc);
+    }
+
+    @Override
+    public StringData read(StringData ignored) {
+      Binary binary = column.nextBinary();
+      ByteBuffer buffer = binary.toByteBuffer();
+      if (buffer.hasArray()) {
+        return StringData.fromBytes(
+            buffer.array(), buffer.arrayOffset() + buffer.position(), buffer.remaining());
+      } else {
+        return StringData.fromBytes(binary.getBytes());
+      }
+    }
+  }
+
+  private static class LossyMicrosToMillisTimeReader extends ParquetValueReaders.PrimitiveReader<Integer> {
+    LossyMicrosToMillisTimeReader(ColumnDescriptor desc) {
+      super(desc);
+    }
+
+    @Override
+    public Integer read(Integer reuse) {
+      // Discard microseconds since Flink uses millisecond unit for TIME type.
+      return (int) Math.floorDiv(column.nextLong(), 1000);
+    }
+  }
+
+  private static class ArrayReader<E> extends ParquetValueReaders.RepeatedReader<ArrayData, ReusableArrayData, E> {
+    private int readPos = 0;
+    private int writePos = 0;
+
+    ArrayReader(int definitionLevel, int repetitionLevel, ParquetValueReader<E> reader) {
+      super(definitionLevel, repetitionLevel, reader);
+    }
+
+    @Override
+    protected ReusableArrayData newListData(ArrayData reuse) {
+      this.readPos = 0;
+      this.writePos = 0;
+
+      if (reuse instanceof ReusableArrayData) {
+        return (ReusableArrayData) reuse;
+      } else {
+        return new ReusableArrayData();
+      }
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    protected E getElement(ReusableArrayData list) {
+      E value = null;
+      if (readPos < list.capacity()) {
+        value = (E) list.values[readPos];
+      }
+
+      readPos += 1;
+
+      return value;
+    }
+
+    @Override
+    protected void addElement(ReusableArrayData reused, E element) {
+      if (writePos >= reused.capacity()) {
+        reused.grow();
+      }
+
+      reused.values[writePos] = element;
+
+      writePos += 1;
+    }
+
+    @Override
+    protected ArrayData buildList(ReusableArrayData list) {
+      list.setNumElements(writePos);
+      return list;
+    }
+  }
+
+  private static class MapReader<K, V> extends
+      ParquetValueReaders.RepeatedKeyValueReader<MapData, ReusableMapData, K, V> {
+    private int readPos = 0;
+    private int writePos = 0;
+
+    private final ParquetValueReaders.ReusableEntry<K, V> entry = new ParquetValueReaders.ReusableEntry<>();
+    private final ParquetValueReaders.ReusableEntry<K, V> nullEntry = new ParquetValueReaders.ReusableEntry<>();
+
+    MapReader(int definitionLevel, int repetitionLevel,
+              ParquetValueReader<K> keyReader, ParquetValueReader<V> valueReader) {
+      super(definitionLevel, repetitionLevel, keyReader, valueReader);
+    }
+
+    @Override
+    protected ReusableMapData newMapData(MapData reuse) {
+      this.readPos = 0;
+      this.writePos = 0;
+
+      if (reuse instanceof ReusableMapData) {
+        return (ReusableMapData) reuse;
+      } else {
+        return new ReusableMapData();
+      }
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    protected Map.Entry<K, V> getPair(ReusableMapData map) {
+      Map.Entry<K, V> kv = nullEntry;
+      if (readPos < map.capacity()) {
+        entry.set((K) map.keys.values[readPos], (V) map.values.values[readPos]);
+        kv = entry;
+      }
+
+      readPos += 1;
+
+      return kv;
+    }
+
+    @Override
+    protected void addPair(ReusableMapData map, K key, V value) {
+      if (writePos >= map.capacity()) {
+        map.grow();
+      }
+
+      map.keys.values[writePos] = key;
+      map.values.values[writePos] = value;
+
+      writePos += 1;
+    }
+
+    @Override
+    protected MapData buildMap(ReusableMapData map) {
+      map.setNumElements(writePos);
+      return map;
+    }
   }
 
-  private static class RowReader extends ParquetValueReaders.StructReader<Row, Row> {
-    private final Types.StructType structType;
+  private static class RowDataReader extends ParquetValueReaders.StructReader<RowData, GenericRowData> {
+    private final int numFields;
 
-    RowReader(List<Type> types, List<ParquetValueReader<?>> readers, Types.StructType struct) {
+    RowDataReader(List<Type> types, List<ParquetValueReader<?>> readers) {
       super(types, readers);
-      this.structType = struct;
+      this.numFields = readers.size();
     }
 
     @Override
-    protected Row newStructData(Row reuse) {
-      if (reuse != null) {
-        return reuse;
+    protected GenericRowData newStructData(RowData reuse) {
+      if (reuse instanceof GenericRowData) {
+        return (GenericRowData) reuse;
       } else {
-        return new Row(structType.fields().size());
+        return new GenericRowData(numFields);
       }
     }
 
     @Override
-    protected Object getField(Row row, int pos) {
-      return row.getField(pos);
+    protected Object getField(GenericRowData intermediate, int pos) {
+      return intermediate.getField(pos);
+    }
+
+    @Override
+    protected RowData buildStruct(GenericRowData struct) {
+      return struct;
+    }
+
+    @Override
+    protected void set(GenericRowData row, int pos, Object value) {
+      row.setField(pos, value);
+    }
+
+    @Override
+    protected void setNull(GenericRowData row, int pos) {
+      row.setField(pos, null);
+    }
+
+    @Override
+    protected void setBoolean(GenericRowData row, int pos, boolean value) {
+      row.setField(pos, value);
+    }
+
+    @Override
+    protected void setInteger(GenericRowData row, int pos, int value) {
+      row.setField(pos, value);
+    }
+
+    @Override
+    protected void setLong(GenericRowData row, int pos, long value) {
+      row.setField(pos, value);
     }
 
     @Override
-    protected Row buildStruct(Row row) {
-      return row;
+    protected void setFloat(GenericRowData row, int pos, float value) {
+      row.setField(pos, value);
     }
 
     @Override
-    protected void set(Row row, int pos, Object value) {
+    protected void setDouble(GenericRowData row, int pos, double value) {
       row.setField(pos, value);
     }
   }
+
+  private static class ReusableMapData implements MapData {

Review comment:
       I think we can use the same ReusableMapData class for both Spark and Flink, since they are so similar. Some of the data access methods in the concrete classes would need to be implemented in a concrete class the engine-specific interface, but the main parts could be shared. And I think that would mean we could share significant parts of the reader classes as well.
   
   I don't think we need to do this refactor now, but we should try to deduplicate this.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] chenjunjiedada commented on a change in pull request #1266: Flink: update parquet reader with schema visitor

Posted by GitBox <gi...@apache.org>.
chenjunjiedada commented on a change in pull request #1266:
URL: https://github.com/apache/iceberg/pull/1266#discussion_r466906862



##########
File path: data/src/test/java/org/apache/iceberg/data/RandomGenericData.java
##########
@@ -55,6 +59,14 @@ private RandomGenericData() {}
     return records;
   }
 
+  public static Iterable<Record> generateFallbackRecords(Schema schema, int numRecords, long seed, long numDictRows) {
+    return generateRecords(schema, numRecords, new FallbackGenerator(seed, numDictRows));

Review comment:
       Fixed.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] chenjunjiedada commented on a change in pull request #1266: Flink: update parquet reader with schema visitor

Posted by GitBox <gi...@apache.org>.
chenjunjiedada commented on a change in pull request #1266:
URL: https://github.com/apache/iceberg/pull/1266#discussion_r466908525



##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java
##########
@@ -19,64 +19,701 @@
 
 package org.apache.iceberg.flink.data;
 
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.time.Instant;
+import java.time.ZoneOffset;
 import java.util.List;
-import org.apache.flink.types.Row;
+import java.util.Map;
+import org.apache.commons.lang3.ArrayUtils;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.MapData;
+import org.apache.flink.table.data.RawValueData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
 import org.apache.iceberg.Schema;
-import org.apache.iceberg.data.parquet.BaseParquetReaders;
 import org.apache.iceberg.parquet.ParquetValueReader;
 import org.apache.iceberg.parquet.ParquetValueReaders;
+import org.apache.iceberg.parquet.TypeWithSchemaVisitor;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
 import org.apache.iceberg.types.Types;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.io.api.Binary;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.LogicalTypeAnnotation.DecimalLogicalTypeAnnotation;
 import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
 import org.apache.parquet.schema.Type;
 
-public class FlinkParquetReaders extends BaseParquetReaders<Row> {
+class FlinkParquetReaders {
+  private FlinkParquetReaders() {
+  }
 
-  private static final FlinkParquetReaders INSTANCE = new FlinkParquetReaders();
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema, MessageType fileSchema) {
+    return buildReader(expectedSchema, fileSchema, ImmutableMap.of());
+  }
 
-  private FlinkParquetReaders() {
+  @SuppressWarnings("unchecked")
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema,
+                                                        MessageType fileSchema,
+                                                        Map<Integer, ?> idToConstant) {
+    return (ParquetValueReader<RowData>) TypeWithSchemaVisitor.visit(expectedSchema.asStruct(), fileSchema,
+        new ReadBuilder(fileSchema, idToConstant)
+    );
+  }
+
+  private static class ReadBuilder extends TypeWithSchemaVisitor<ParquetValueReader<?>> {
+    private final MessageType type;
+    private final Map<Integer, ?> idToConstant;
+
+    ReadBuilder(MessageType type, Map<Integer, ?> idToConstant) {
+      this.type = type;
+      this.idToConstant = idToConstant;
+    }
+
+    @Override
+    public ParquetValueReader<RowData> message(Types.StructType expected, MessageType message,
+                                               List<ParquetValueReader<?>> fieldReaders) {
+      return struct(expected, message.asGroupType(), fieldReaders);
+    }
+
+    @Override
+    public ParquetValueReader<RowData> struct(Types.StructType expected, GroupType struct,
+                                              List<ParquetValueReader<?>> fieldReaders) {
+      // match the expected struct's order
+      Map<Integer, ParquetValueReader<?>> readersById = Maps.newHashMap();
+      Map<Integer, Type> typesById = Maps.newHashMap();
+      List<Type> fields = struct.getFields();
+      for (int i = 0; i < fields.size(); i += 1) {
+        Type fieldType = fields.get(i);
+        int fieldD = type.getMaxDefinitionLevel(path(fieldType.getName())) - 1;
+        if (fieldType.getId() != null) {
+          int id = fieldType.getId().intValue();
+          readersById.put(id, ParquetValueReaders.option(fieldType, fieldD, fieldReaders.get(i)));
+          typesById.put(id, fieldType);
+        }
+      }
+
+      List<Types.NestedField> expectedFields = expected != null ?
+          expected.fields() : ImmutableList.of();
+      List<ParquetValueReader<?>> reorderedFields = Lists.newArrayListWithExpectedSize(
+          expectedFields.size());
+      List<Type> types = Lists.newArrayListWithExpectedSize(expectedFields.size());
+      for (Types.NestedField field : expectedFields) {
+        int id = field.fieldId();
+        if (idToConstant.containsKey(id)) {
+          // containsKey is used because the constant may be null
+          reorderedFields.add(ParquetValueReaders.constant(idToConstant.get(id)));
+          types.add(null);
+        } else {
+          ParquetValueReader<?> reader = readersById.get(id);
+          if (reader != null) {
+            reorderedFields.add(reader);
+            types.add(typesById.get(id));
+          } else {
+            reorderedFields.add(ParquetValueReaders.nulls());
+            types.add(null);
+          }
+        }
+      }
+
+      return new RowDataReader(types, reorderedFields);
+    }
+
+    @Override
+    public ParquetValueReader<?> list(Types.ListType expectedList, GroupType array,
+                                      ParquetValueReader<?> elementReader) {
+      GroupType repeated = array.getFields().get(0).asGroupType();
+      String[] repeatedPath = currentPath();
+
+      int repeatedD = type.getMaxDefinitionLevel(repeatedPath) - 1;
+      int repeatedR = type.getMaxRepetitionLevel(repeatedPath) - 1;
+
+      Type elementType = repeated.getType(0);
+      int elementD = type.getMaxDefinitionLevel(path(elementType.getName())) - 1;
+
+      return new ArrayReader<>(repeatedD, repeatedR, ParquetValueReaders.option(elementType, elementD, elementReader));
+    }
+
+    @Override
+    public ParquetValueReader<?> map(Types.MapType expectedMap, GroupType map,
+                                     ParquetValueReader<?> keyReader,
+                                     ParquetValueReader<?> valueReader) {
+      GroupType repeatedKeyValue = map.getFields().get(0).asGroupType();
+      String[] repeatedPath = currentPath();
+
+      int repeatedD = type.getMaxDefinitionLevel(repeatedPath) - 1;
+      int repeatedR = type.getMaxRepetitionLevel(repeatedPath) - 1;
+
+      Type keyType = repeatedKeyValue.getType(0);
+      int keyD = type.getMaxDefinitionLevel(path(keyType.getName())) - 1;
+      Type valueType = repeatedKeyValue.getType(1);
+      int valueD = type.getMaxDefinitionLevel(path(valueType.getName())) - 1;
+
+      return new MapReader<>(repeatedD, repeatedR,
+          ParquetValueReaders.option(keyType, keyD, keyReader),
+          ParquetValueReaders.option(valueType, valueD, valueReader));
+    }
+
+    @Override
+    public ParquetValueReader<?> primitive(org.apache.iceberg.types.Type.PrimitiveType expected,
+                                           PrimitiveType primitive) {
+      ColumnDescriptor desc = type.getColumnDescription(currentPath());
+
+      if (primitive.getOriginalType() != null) {
+        switch (primitive.getOriginalType()) {
+          case ENUM:
+          case JSON:
+          case UTF8:
+            return new StringReader(desc);
+          case INT_8:
+          case INT_16:
+          case INT_32:
+          case DATE:
+            if (expected != null && expected.typeId() == Types.LongType.get().typeId()) {
+              return new ParquetValueReaders.IntAsLongReader(desc);
+            } else {
+              return new ParquetValueReaders.UnboxedReader<>(desc);
+            }
+          case TIME_MICROS:
+            return new LossyMicrosToMillisTimeReader(desc);
+          case INT_64:
+            return new ParquetValueReaders.UnboxedReader<>(desc);
+          case TIMESTAMP_MICROS:
+            if (((Types.TimestampType) expected).shouldAdjustToUTC()) {
+              return new TimestampTzReader(desc);
+            } else {
+              return new TimestampReader(desc);
+            }
+          case DECIMAL:
+            DecimalLogicalTypeAnnotation decimal = (DecimalLogicalTypeAnnotation) primitive.getLogicalTypeAnnotation();
+            switch (primitive.getPrimitiveTypeName()) {
+              case BINARY:
+              case FIXED_LEN_BYTE_ARRAY:
+                return new BinaryDecimalReader(desc, decimal.getScale());
+              case INT64:
+                return new LongDecimalReader(desc, decimal.getPrecision(), decimal.getScale());
+              case INT32:
+                return new IntegerDecimalReader(desc, decimal.getPrecision(), decimal.getScale());
+              default:
+                throw new UnsupportedOperationException(
+                    "Unsupported base type for decimal: " + primitive.getPrimitiveTypeName());
+            }
+          case BSON:
+            return new ParquetValueReaders.ByteArrayReader(desc);
+          default:
+            throw new UnsupportedOperationException(
+                "Unsupported logical type: " + primitive.getOriginalType());
+        }
+      }
+
+      switch (primitive.getPrimitiveTypeName()) {
+        case FIXED_LEN_BYTE_ARRAY:
+        case BINARY:
+          return new ParquetValueReaders.ByteArrayReader(desc);
+        case INT32:
+          if (expected != null && expected.typeId() == org.apache.iceberg.types.Type.TypeID.LONG) {
+            return new ParquetValueReaders.IntAsLongReader(desc);
+          } else {
+            return new ParquetValueReaders.UnboxedReader<>(desc);
+          }
+        case FLOAT:
+          if (expected != null && expected.typeId() == org.apache.iceberg.types.Type.TypeID.DOUBLE) {
+            return new ParquetValueReaders.FloatAsDoubleReader(desc);
+          } else {
+            return new ParquetValueReaders.UnboxedReader<>(desc);
+          }
+        case BOOLEAN:
+        case INT64:
+        case DOUBLE:
+          return new ParquetValueReaders.UnboxedReader<>(desc);
+        default:
+          throw new UnsupportedOperationException("Unsupported type: " + primitive);
+      }
+    }
+  }
+
+  private static class BinaryDecimalReader extends ParquetValueReaders.PrimitiveReader<DecimalData> {
+    private final int scale;
+
+    BinaryDecimalReader(ColumnDescriptor desc, int scale) {
+      super(desc);
+      this.scale = scale;
+    }
+
+    @Override
+    public DecimalData read(DecimalData ignored) {
+      Binary binary = column.nextBinary();
+      BigDecimal bigDecimal = new BigDecimal(new BigInteger(binary.getBytes()), scale);
+      return DecimalData.fromBigDecimal(bigDecimal, bigDecimal.precision(), scale);

Review comment:
       Fixed.  I think we can add unit tests when write side is ready so that we can write RowData and read out to validate. Does that make sense to you?

##########
File path: flink/src/test/java/org/apache/iceberg/flink/data/TestHelpers.java
##########
@@ -0,0 +1,269 @@
+/*
+ * 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.iceberg.flink.data;
+
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.OffsetDateTime;
+import java.time.ZoneOffset;
+import java.time.temporal.ChronoUnit;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.RowData;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+
+public class TestHelpers {
+  private TestHelpers() {}
+
+  private static final OffsetDateTime EPOCH = Instant.ofEpochMilli(0L).atOffset(ZoneOffset.UTC);
+  private static final LocalDate EPOCH_DAY = EPOCH.toLocalDate();
+
+  public static void assertRowData(Type type, Record expectedRecord, RowData actualRowData) {
+    if (expectedRecord == null && actualRowData == null) {
+      return;
+    }
+
+    Assert.assertTrue("expected Record and actual RowData should be both null or not null",
+        expectedRecord != null && actualRowData != null);
+
+    List<Type> types = new ArrayList<>();

Review comment:
       fixed.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] chenjunjiedada commented on a change in pull request #1266: Flink: update parquet reader with schema visitor

Posted by GitBox <gi...@apache.org>.
chenjunjiedada commented on a change in pull request #1266:
URL: https://github.com/apache/iceberg/pull/1266#discussion_r464451995



##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java
##########
@@ -19,64 +19,719 @@
 
 package org.apache.iceberg.flink.data;
 
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.time.Instant;
 import java.util.List;
-import org.apache.flink.types.Row;
+import java.util.Map;
+import org.apache.commons.lang3.ArrayUtils;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.MapData;
+import org.apache.flink.table.data.RawValueData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
 import org.apache.iceberg.Schema;
-import org.apache.iceberg.data.parquet.BaseParquetReaders;
+import org.apache.iceberg.parquet.ParquetSchemaUtil;
 import org.apache.iceberg.parquet.ParquetValueReader;
 import org.apache.iceberg.parquet.ParquetValueReaders;
+import org.apache.iceberg.parquet.TypeWithSchemaVisitor;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
 import org.apache.iceberg.types.Types;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.io.api.Binary;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.LogicalTypeAnnotation.DecimalLogicalTypeAnnotation;
 import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
 import org.apache.parquet.schema.Type;
 
-public class FlinkParquetReaders extends BaseParquetReaders<Row> {
+class FlinkParquetReaders {
+  private FlinkParquetReaders() {
+  }
 
-  private static final FlinkParquetReaders INSTANCE = new FlinkParquetReaders();
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema, MessageType fileSchema) {
+    return buildReader(expectedSchema, fileSchema, ImmutableMap.of());
+  }
 
-  private FlinkParquetReaders() {
+  @SuppressWarnings("unchecked")
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema,
+                                                        MessageType fileSchema,
+                                                        Map<Integer, ?> idToConstant) {
+    ReadBuilder builder = new ReadBuilder(fileSchema, idToConstant);
+    if (ParquetSchemaUtil.hasIds(fileSchema)) {
+      return (ParquetValueReader<RowData>)
+          TypeWithSchemaVisitor.visit(expectedSchema.asStruct(), fileSchema, builder);
+    } else {
+      return (ParquetValueReader<RowData>)
+          TypeWithSchemaVisitor.visit(expectedSchema.asStruct(), fileSchema,
+              new FallbackReadBuilder(builder));
+    }
+  }
+
+  private static class FallbackReadBuilder extends TypeWithSchemaVisitor<ParquetValueReader<?>> {
+    private MessageType type;
+    private final TypeWithSchemaVisitor<ParquetValueReader<?>> builder;

Review comment:
       OK, sound good to me.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] chenjunjiedada commented on a change in pull request #1266: Flink: update parquet reader with schema visitor

Posted by GitBox <gi...@apache.org>.
chenjunjiedada commented on a change in pull request #1266:
URL: https://github.com/apache/iceberg/pull/1266#discussion_r465087333



##########
File path: data/src/test/java/org/apache/iceberg/data/RandomGenericData.java
##########
@@ -55,8 +55,8 @@ private RandomGenericData() {}
     return records;
   }
 
-  private static class RandomRecordGenerator extends RandomDataGenerator<Record> {
-    private RandomRecordGenerator(long seed) {
+  public static class RandomRecordGenerator extends RandomDataGenerator<Record> {

Review comment:
       Fixed.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] chenjunjiedada commented on a change in pull request #1266: Flink: update parquet reader with schema visitor

Posted by GitBox <gi...@apache.org>.
chenjunjiedada commented on a change in pull request #1266:
URL: https://github.com/apache/iceberg/pull/1266#discussion_r465086661



##########
File path: flink/src/test/java/org/apache/iceberg/flink/data/RandomData.java
##########
@@ -88,20 +93,46 @@ public Row next() {
     };
   }
 
+  private static Iterable<Record> generateIcebergGenerics(Schema schema, int numRecords,
+                                                   Supplier<RandomGenericData.RandomDataGenerator<Record>> supplier) {
+    return () -> new Iterator<Record>() {
+      private final RandomGenericData.RandomDataGenerator<Record> generator = supplier.get();
+      private int count = 0;
+
+      @Override
+      public boolean hasNext() {
+        return count < numRecords;
+      }
+
+      @Override
+      public Record next() {
+        if (!hasNext()) {
+          throw new NoSuchElementException();
+        }
+        ++count;
+        return (Record) TypeUtil.visit(schema, generator);
+      }
+    };
+  }
+
+
+  public static Iterable<Record> generateRecords(Schema schema, int numRecords, long seed) {
+    return RandomGenericData.generate(schema, numRecords, seed);
+  }
+
   public static Iterable<Row> generate(Schema schema, int numRecords, long seed) {
     return generateData(schema, numRecords, () -> new RandomRowGenerator(seed));
   }
 
-  public static Iterable<Row> generateFallbackData(Schema schema, int numRecords, long seed, long numDictRows) {
-    return generateData(schema, numRecords, () -> new FallbackGenerator(seed, numDictRows));
+  public static Iterable<Record> generateFallbackRecords(Schema schema, int numRecords, long seed, long numDictRows) {
+    return generateIcebergGenerics(schema, numRecords, () -> new FallbackGenerator(seed, numDictRows));
   }
 
-  public static Iterable<Row> generateDictionaryEncodableData(Schema schema, int numRecords, long seed) {
-    return generateData(schema, numRecords, () -> new DictionaryEncodedGenerator(seed));
+  public static Iterable<Record> generateDictionaryEncodableRecords(Schema schema, int numRecords, long seed) {

Review comment:
       Done.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org