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:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]