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 2022/11/19 09:18:49 UTC

[GitHub] [iceberg] hililiwei opened a new pull request, #6222: Flink: Support inspecting table

hililiwei opened a new pull request, #6222:
URL: https://github.com/apache/iceberg/pull/6222

   ## What is the purpose of the change
   
   Support inspecting table in flink sql.
   ```
   select * from {tableName}${metadataTableName}
   ```
   
   The  '$' syntax borrows from [Flink-Table-Store](https://github.com/apache/flink-table-store), under the Flink umbrella.
   
   
   ## Support List
   *  History
   
   To show table history:
   
   ```sql
   SELECT * FROM prod.db.table$history;
   ```
   
   *   Metadata Log Entries
   
   To show table metadata log entries:
   
   ```sql
   SELECT * from prod.db.table$metadata_log_entries;
   ```
   
   *   Snapshots
   
   To show the valid snapshots for a table:
   
   ```sql
   SELECT * FROM prod.db.table$snapshots;
   ```
   
   *   Files
   
   To show a table's current data files:
   
   ```sql
   SELECT * FROM prod.db.table$files;
   ```
   
   *   Manifests
   
   To show a table's current file manifests:
   
   ```sql
   SELECT * FROM prod.db.table$manifests;
   ```
   
   *   Partitions
   
   To show a table's current partitions:
   
   ```sql
   SELECT * FROM prod.db.table$partitions;
   ```
   
   *   All Metadata Tables
   
   These tables are unions of the metadata tables specific to the current snapshot, and return metadata across all snapshots.
   
   - -   All Data Files
   
   To show all of the table's data files and each file's metadata:
   
   ```sql
   SELECT * FROM prod.db.table$all_data_files;
   ```
   
   - -   All Manifests
   
   To show all of the table's manifest files:
   
   ```sql
   SELECT * FROM prod.db.table$all_manifests;
   ```
   
   -  References
   
   To show a table's known snapshot references:
   
   ```sql
   SELECT * FROM prod.db.table$refs;
   ```
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] hililiwei commented on a diff in pull request #6222: Flink: Support inspecting table

Posted by GitBox <gi...@apache.org>.
hililiwei commented on code in PR #6222:
URL: https://github.com/apache/iceberg/pull/6222#discussion_r1043419483


##########
flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java:
##########
@@ -295,6 +299,161 @@ private static void assertEquals(
     }
   }
 
+  public static void assertEqualsSafe(
+      Schema schema, List<GenericData.Record> recs, List<Row> rows) {

Review Comment:
   `assertEqualsSafe(Types.StructType struct, GenericData.Record rec, Row row)` has beed changed to private. The other two are not changed because they are used in `TestFlinkMetaDataTable`.
   Thx.
   
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] hililiwei commented on a diff in pull request #6222: Flink: Support inspecting table

Posted by GitBox <gi...@apache.org>.
hililiwei commented on code in PR #6222:
URL: https://github.com/apache/iceberg/pull/6222#discussion_r1042976674


##########
core/src/main/java/org/apache/iceberg/BaseFilesTable.java:
##########
@@ -223,34 +225,28 @@ ManifestFile manifest() {
   static class ContentFileStructWithMetrics implements StructLike {
     private final StructLike fileAsStruct;
     private final MetricsUtil.ReadableMetricsStruct readableMetrics;
-    private final int expectedSize;
+    private final int position;
 
     ContentFileStructWithMetrics(
-        int expectedSize,
-        StructLike fileAsStruct,
-        MetricsUtil.ReadableMetricsStruct readableMetrics) {
+        int position, StructLike fileAsStruct, MetricsUtil.ReadableMetricsStruct readableMetrics) {
       this.fileAsStruct = fileAsStruct;
       this.readableMetrics = readableMetrics;
-      this.expectedSize = expectedSize;
+      this.position = position;
     }
 
     @Override
     public int size() {
-      return expectedSize;
+      return position;

Review Comment:
   Yes, I ignored this. It was a mistake.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] stevenzwu commented on a diff in pull request #6222: Flink: Support inspecting table

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on code in PR #6222:
URL: https://github.com/apache/iceberg/pull/6222#discussion_r1039841272


##########
docs/flink-getting-started.md:
##########
@@ -712,9 +712,188 @@ INSERT INTO tableName /*+ OPTIONS('upsert-enabled'='true') */
 | compression-strategy   | Table write.orc.compression-strategy       | Overrides this table's compression strategy for ORC tables for this write                                  |
 
 
-## Inspecting tables.
+## Inspecting tables
 
-Iceberg does not support inspecting table in flink sql now, we need to use [iceberg's Java API](../api) to read iceberg's meta data to get those table information.
+To inspect a table's history, snapshots, and other metadata, Iceberg supports metadata tables.
+
+Metadata tables are identified by adding the metadata table name after the original table name. For example, history for `db.table` is read using `db.table$history`.
+
+### History
+
+To show table history:
+
+```sql
+SELECT * FROM prod.db.table$history;
+```
+
+| made_current_at         | snapshot_id         | parent_id           | is_current_ancestor |
+| ----------------------- | ------------------- | ------------------- | ------------------- |
+| 2019-02-08 03:29:51.215 | 5781947118336215154 | NULL                | true                |
+| 2019-02-08 03:47:55.948 | 5179299526185056830 | 5781947118336215154 | true                |
+| 2019-02-09 16:24:30.13  | 296410040247533544  | 5179299526185056830 | false               |
+| 2019-02-09 16:32:47.336 | 2999875608062437330 | 5179299526185056830 | true                |
+| 2019-02-09 19:42:03.919 | 8924558786060583479 | 2999875608062437330 | true                |
+| 2019-02-09 19:49:16.343 | 6536733823181975045 | 8924558786060583479 | true                |
+
+{{< hint info >}}
+**This shows a commit that was rolled back.** The example has two snapshots with the same parent, and one is *not* an ancestor of the current table state.

Review Comment:
   maybe more explicit?
   
   ```
   In this example, snapshot 296410040247533544 and 2999875608062437330 have the same parent snapshot 5179299526185056830. Snapshot 296410040247533544 was rolled back and is *not* an ancestor of the current table state.
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] hililiwei commented on a diff in pull request #6222: Flink: Support inspecting table

Posted by GitBox <gi...@apache.org>.
hililiwei commented on code in PR #6222:
URL: https://github.com/apache/iceberg/pull/6222#discussion_r1051360946


##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java:
##########
@@ -0,0 +1,286 @@
+/*
+ * 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.Duration;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.OffsetDateTime;
+import java.util.List;
+import java.util.Map;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericMapData;
+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.flink.types.RowKind;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.ByteBuffers;
+
+@Internal
+public class StructRowData implements RowData {
+  private final Types.StructType type;
+  private RowKind kind;
+  private StructLike struct;
+
+  public StructRowData(Types.StructType type) {
+    this(type, RowKind.INSERT);
+  }
+
+  public StructRowData(Types.StructType type, RowKind kind) {
+    this(type, null, kind);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct) {
+    this(type, struct, RowKind.INSERT);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct, RowKind kind) {
+    this.type = type;
+    this.struct = struct;
+    this.kind = kind;
+  }
+
+  public StructRowData setStruct(StructLike newStruct) {
+    this.struct = newStruct;
+    return this;
+  }
+
+  @Override
+  public int getArity() {
+    return struct.size();
+  }
+
+  @Override
+  public RowKind getRowKind() {
+    return kind;
+  }
+
+  @Override
+  public void setRowKind(RowKind newKind) {
+    Preconditions.checkNotNull(newKind, "kind can not be null");
+    this.kind = newKind;
+  }
+
+  @Override
+  public boolean isNullAt(int pos) {
+    return struct.get(pos, Object.class) == null;
+  }
+
+  @Override
+  public boolean getBoolean(int pos) {
+    return struct.get(pos, Boolean.class);
+  }
+
+  @Override
+  public byte getByte(int pos) {
+    return (byte) (int) struct.get(pos, Integer.class);
+  }
+
+  @Override
+  public short getShort(int pos) {
+    return (short) (int) struct.get(pos, Integer.class);
+  }
+
+  @Override
+  public int getInt(int pos) {
+    Object integer = struct.get(pos, Object.class);
+
+    if (integer instanceof Integer) {
+      return (int) integer;
+    } else if (integer instanceof LocalDate) {
+      return (int) ((LocalDate) integer).toEpochDay();
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for int field. Type name: " + integer.getClass().getName());
+    }
+  }
+
+  @Override
+  public long getLong(int pos) {
+    Object longVal = struct.get(pos, Object.class);
+
+    if (longVal instanceof Long) {
+      return (long) longVal;
+    } else if (longVal instanceof OffsetDateTime) {
+      return Duration.between(Instant.EPOCH, (OffsetDateTime) longVal).toNanos() / 1000;
+    } else if (longVal instanceof LocalDate) {
+      return ((LocalDate) longVal).toEpochDay();
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for long field. Type name: " + longVal.getClass().getName());
+    }
+  }
+
+  @Override
+  public float getFloat(int pos) {
+    return struct.get(pos, Float.class);
+  }
+
+  @Override
+  public double getDouble(int pos) {
+    return struct.get(pos, Double.class);
+  }
+
+  @Override
+  public StringData getString(int pos) {
+    return isNullAt(pos) ? null : getStringDataInternal(pos);
+  }
+
+  private StringData getStringDataInternal(int pos) {
+    CharSequence seq = struct.get(pos, CharSequence.class);
+    return StringData.fromString(seq.toString());
+  }
+
+  @Override
+  public DecimalData getDecimal(int pos, int precision, int scale) {
+    return isNullAt(pos)
+        ? null
+        : DecimalData.fromBigDecimal(getDecimalInternal(pos), precision, scale);
+  }
+
+  private BigDecimal getDecimalInternal(int pos) {
+    return struct.get(pos, BigDecimal.class);
+  }
+
+  @Override
+  public TimestampData getTimestamp(int pos, int precision) {
+    long timeLong = getLong(pos);
+    return TimestampData.fromEpochMillis(timeLong / 1000, (int) (timeLong % 1000) * 1000);
+  }
+
+  @Override
+  public <T> RawValueData<T> getRawValue(int pos) {
+    throw new UnsupportedOperationException("Not supported yet.");
+  }
+
+  @Override
+  public byte[] getBinary(int pos) {
+    return isNullAt(pos) ? null : getBinaryInternal(pos);
+  }
+
+  private byte[] getBinaryInternal(int pos) {
+    Object bytes = struct.get(pos, Object.class);
+
+    // should only be either ByteBuffer or byte[]
+    if (bytes instanceof ByteBuffer) {
+      return ByteBuffers.toByteArray((ByteBuffer) bytes);
+    } else if (bytes instanceof byte[]) {
+      return (byte[]) bytes;
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for binary field. Type name: " + bytes.getClass().getName());
+    }
+  }
+
+  @Override
+  public ArrayData getArray(int pos) {
+    return isNullAt(pos)
+        ? null
+        : (ArrayData)
+            covertFromStructToRowData(
+                type.fields().get(pos).type().asListType(), struct.get(pos, List.class));
+  }
+
+  @Override
+  public MapData getMap(int pos) {
+    return isNullAt(pos)
+        ? null
+        : (MapData)
+            covertFromStructToRowData(
+                type.fields().get(pos).type().asMapType(), struct.get(pos, Map.class));
+  }
+
+  @Override
+  public RowData getRow(int pos, int numFields) {
+    return isNullAt(pos) ? null : getStructRowData(pos, numFields);
+  }
+
+  private StructRowData getStructRowData(int pos, int numFields) {
+    return new StructRowData(
+        type.fields().get(pos).type().asStructType(), struct.get(pos, StructLike.class));
+  }
+
+  private Object covertFromStructToRowData(Type elementType, Object value) {
+    switch (elementType.typeId()) {
+      case BOOLEAN:
+      case INTEGER:
+      case DATE:
+      case TIME:
+      case LONG:
+      case FLOAT:
+      case DOUBLE:
+      case DECIMAL:
+        return value;
+      case TIMESTAMP:
+        return TimestampData.fromEpochMillis(
+            (Long) value / 1000, (int) ((Long) value % 1000) * 1000);
+      case STRING:
+        return StringData.fromString(value.toString());
+      case FIXED:
+      case BINARY:
+        return ByteBuffers.toByteArray((ByteBuffer) value);
+      case STRUCT:
+        return new StructRowData(elementType.asStructType(), (StructLike) value);
+      case LIST:
+        List<?> list = (List<?>) value;
+        Object[] array = new Object[list.size()];
+
+        int index = 0;
+        for (Object element : list) {
+          if (element == null) {
+            array[index] = null;
+          } else {
+            array[index] =
+                covertFromStructToRowData(elementType.asListType().elementType(), element);
+          }
+
+          index += 1;
+        }
+        return new GenericArrayData(array);
+      case MAP:
+        Types.MapType mapType = elementType.asMapType();
+        // make a defensive copy to ensure entries do not change
+        List<Map.Entry<?, ?>> entries = ImmutableList.copyOf(((Map<?, ?>) value).entrySet());
+
+        Map<Object, Object> result = Maps.newHashMap();
+        for (Map.Entry<?, ?> entry : entries) {
+          final Object keyValue = covertFromStructToRowData(mapType.keyType(), entry.getKey());
+          final Object valueValue =
+              covertFromStructToRowData(mapType.valueType(), entry.getValue());
+          result.put(keyValue, valueValue);
+        }

Review Comment:
   added. I didn't add it because the line below it is `return`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] hililiwei commented on a diff in pull request #6222: Flink: Support inspecting table

Posted by GitBox <gi...@apache.org>.
hililiwei commented on code in PR #6222:
URL: https://github.com/apache/iceberg/pull/6222#discussion_r1051382070


##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java:
##########
@@ -0,0 +1,286 @@
+/*
+ * 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.Duration;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.OffsetDateTime;
+import java.util.List;
+import java.util.Map;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericMapData;
+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.flink.types.RowKind;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.ByteBuffers;
+
+@Internal
+public class StructRowData implements RowData {
+  private final Types.StructType type;
+  private RowKind kind;
+  private StructLike struct;
+
+  public StructRowData(Types.StructType type) {
+    this(type, RowKind.INSERT);
+  }
+
+  public StructRowData(Types.StructType type, RowKind kind) {
+    this(type, null, kind);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct) {
+    this(type, struct, RowKind.INSERT);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct, RowKind kind) {
+    this.type = type;
+    this.struct = struct;
+    this.kind = kind;
+  }
+
+  public StructRowData setStruct(StructLike newStruct) {
+    this.struct = newStruct;
+    return this;
+  }
+
+  @Override
+  public int getArity() {
+    return struct.size();
+  }
+
+  @Override
+  public RowKind getRowKind() {
+    return kind;
+  }
+
+  @Override
+  public void setRowKind(RowKind newKind) {
+    Preconditions.checkNotNull(newKind, "kind can not be null");
+    this.kind = newKind;
+  }
+
+  @Override
+  public boolean isNullAt(int pos) {
+    return struct.get(pos, Object.class) == null;
+  }
+
+  @Override
+  public boolean getBoolean(int pos) {
+    return struct.get(pos, Boolean.class);
+  }
+
+  @Override
+  public byte getByte(int pos) {
+    return (byte) (int) struct.get(pos, Integer.class);
+  }
+
+  @Override
+  public short getShort(int pos) {
+    return (short) (int) struct.get(pos, Integer.class);
+  }
+
+  @Override
+  public int getInt(int pos) {
+    Object integer = struct.get(pos, Object.class);
+
+    if (integer instanceof Integer) {
+      return (int) integer;
+    } else if (integer instanceof LocalDate) {
+      return (int) ((LocalDate) integer).toEpochDay();
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for int field. Type name: " + integer.getClass().getName());
+    }
+  }
+
+  @Override
+  public long getLong(int pos) {
+    Object longVal = struct.get(pos, Object.class);
+
+    if (longVal instanceof Long) {
+      return (long) longVal;
+    } else if (longVal instanceof OffsetDateTime) {
+      return Duration.between(Instant.EPOCH, (OffsetDateTime) longVal).toNanos() / 1000;
+    } else if (longVal instanceof LocalDate) {
+      return ((LocalDate) longVal).toEpochDay();
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for long field. Type name: " + longVal.getClass().getName());
+    }
+  }
+
+  @Override
+  public float getFloat(int pos) {
+    return struct.get(pos, Float.class);
+  }
+
+  @Override
+  public double getDouble(int pos) {
+    return struct.get(pos, Double.class);
+  }
+
+  @Override
+  public StringData getString(int pos) {
+    return isNullAt(pos) ? null : getStringDataInternal(pos);
+  }
+
+  private StringData getStringDataInternal(int pos) {
+    CharSequence seq = struct.get(pos, CharSequence.class);
+    return StringData.fromString(seq.toString());
+  }
+
+  @Override
+  public DecimalData getDecimal(int pos, int precision, int scale) {
+    return isNullAt(pos)
+        ? null
+        : DecimalData.fromBigDecimal(getDecimalInternal(pos), precision, scale);
+  }
+
+  private BigDecimal getDecimalInternal(int pos) {
+    return struct.get(pos, BigDecimal.class);
+  }
+
+  @Override
+  public TimestampData getTimestamp(int pos, int precision) {
+    long timeLong = getLong(pos);
+    return TimestampData.fromEpochMillis(timeLong / 1000, (int) (timeLong % 1000) * 1000);
+  }
+
+  @Override
+  public <T> RawValueData<T> getRawValue(int pos) {
+    throw new UnsupportedOperationException("Not supported yet.");
+  }
+
+  @Override
+  public byte[] getBinary(int pos) {
+    return isNullAt(pos) ? null : getBinaryInternal(pos);
+  }
+
+  private byte[] getBinaryInternal(int pos) {
+    Object bytes = struct.get(pos, Object.class);
+
+    // should only be either ByteBuffer or byte[]
+    if (bytes instanceof ByteBuffer) {
+      return ByteBuffers.toByteArray((ByteBuffer) bytes);
+    } else if (bytes instanceof byte[]) {
+      return (byte[]) bytes;
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for binary field. Type name: " + bytes.getClass().getName());
+    }
+  }
+
+  @Override
+  public ArrayData getArray(int pos) {
+    return isNullAt(pos)
+        ? null
+        : (ArrayData)
+            covertFromStructToRowData(
+                type.fields().get(pos).type().asListType(), struct.get(pos, List.class));
+  }
+
+  @Override
+  public MapData getMap(int pos) {
+    return isNullAt(pos)
+        ? null
+        : (MapData)
+            covertFromStructToRowData(
+                type.fields().get(pos).type().asMapType(), struct.get(pos, Map.class));
+  }
+
+  @Override
+  public RowData getRow(int pos, int numFields) {
+    return isNullAt(pos) ? null : getStructRowData(pos, numFields);
+  }
+
+  private StructRowData getStructRowData(int pos, int numFields) {
+    return new StructRowData(
+        type.fields().get(pos).type().asStructType(), struct.get(pos, StructLike.class));
+  }
+
+  private Object covertFromStructToRowData(Type elementType, Object value) {
+    switch (elementType.typeId()) {
+      case BOOLEAN:
+      case INTEGER:
+      case DATE:
+      case TIME:
+      case LONG:
+      case FLOAT:
+      case DOUBLE:
+      case DECIMAL:
+        return value;
+      case TIMESTAMP:
+        return TimestampData.fromEpochMillis(
+            (Long) value / 1000, (int) ((Long) value % 1000) * 1000);
+      case STRING:
+        return StringData.fromString(value.toString());
+      case FIXED:
+      case BINARY:
+        return ByteBuffers.toByteArray((ByteBuffer) value);
+      case STRUCT:
+        return new StructRowData(elementType.asStructType(), (StructLike) value);
+      case LIST:
+        List<?> list = (List<?>) value;
+        Object[] array = new Object[list.size()];
+
+        int index = 0;
+        for (Object element : list) {
+          if (element == null) {
+            array[index] = null;
+          } else {
+            array[index] =
+                covertFromStructToRowData(elementType.asListType().elementType(), element);
+          }
+
+          index += 1;
+        }
+        return new GenericArrayData(array);
+      case MAP:
+        Types.MapType mapType = elementType.asMapType();
+        // make a defensive copy to ensure entries do not change
+        List<Map.Entry<?, ?>> entries = ImmutableList.copyOf(((Map<?, ?>) value).entrySet());
+
+        Map<Object, Object> result = Maps.newHashMap();
+        for (Map.Entry<?, ?> entry : entries) {
+          final Object keyValue = covertFromStructToRowData(mapType.keyType(), entry.getKey());
+          final Object valueValue =
+              covertFromStructToRowData(mapType.valueType(), entry.getValue());
+          result.put(keyValue, valueValue);
+        }
+        return new GenericMapData(result);
+      default:
+        throw new UnsupportedOperationException("Unsupported array element type: " + elementType);

Review Comment:
   change to `Unsupported element type`? 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] stevenzwu commented on a diff in pull request #6222: Flink: Support inspecting table

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on code in PR #6222:
URL: https://github.com/apache/iceberg/pull/6222#discussion_r1042794568


##########
core/src/main/java/org/apache/iceberg/BaseFilesTable.java:
##########
@@ -177,14 +177,17 @@ public CloseableIterable<StructLike> rows() {
         Set<Integer> readableMetricsIds = TypeUtil.getProjectedIds(readableMetricsField.type());
         Schema fileProjection = TypeUtil.selectNot(projection, readableMetricsIds);
 
+        int position = projection.columns().indexOf(readableMetricsField);

Review Comment:
   nit: maybe remove the empty line before 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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] stevenzwu commented on a diff in pull request #6222: Flink: Support inspecting table

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on code in PR #6222:
URL: https://github.com/apache/iceberg/pull/6222#discussion_r1042779986


##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java:
##########
@@ -104,11 +105,38 @@ public static Schema convert(Schema baseSchema, TableSchema flinkSchema) {
     Types.StructType struct = convert(flinkSchema).asStruct();
     // reassign ids to match the base schema
     Schema schema = TypeUtil.reassignIds(new Schema(struct.fields()), baseSchema);
+    // reassign doc to match the base schema
+    schema = reassignDoc(schema, baseSchema);

Review Comment:
   it seems that `reassignDoc` can be put inside `TypeUtil` like `reassignIds`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] hililiwei commented on a diff in pull request #6222: Flink: Support inspecting table

Posted by GitBox <gi...@apache.org>.
hililiwei commented on code in PR #6222:
URL: https://github.com/apache/iceberg/pull/6222#discussion_r1042990145


##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java:
##########
@@ -104,11 +105,38 @@ public static Schema convert(Schema baseSchema, TableSchema flinkSchema) {
     Types.StructType struct = convert(flinkSchema).asStruct();
     // reassign ids to match the base schema
     Schema schema = TypeUtil.reassignIds(new Schema(struct.fields()), baseSchema);
+    // reassign doc to match the base schema
+    schema = reassignDoc(schema, baseSchema);

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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] hililiwei commented on a diff in pull request #6222: Flink: Support inspecting table

Posted by GitBox <gi...@apache.org>.
hililiwei commented on code in PR #6222:
URL: https://github.com/apache/iceberg/pull/6222#discussion_r1040554968


##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java:
##########
@@ -0,0 +1,340 @@
+/*
+ * 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.Duration;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.OffsetDateTime;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.function.BiConsumer;
+import java.util.function.Function;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericMapData;
+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.flink.types.RowKind;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+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.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.ByteBuffers;
+
+public class StructRowData implements RowData {
+  private final Types.StructType type;
+  private RowKind kind;
+  private StructLike struct;
+
+  public StructRowData(Types.StructType type) {
+    this(type, RowKind.INSERT);
+  }
+
+  public StructRowData(Types.StructType type, RowKind kind) {
+    this(type, null, kind);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct) {
+    this(type, struct, RowKind.INSERT);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct, RowKind kind) {
+    this.type = type;
+    this.struct = struct;
+    this.kind = kind;
+  }
+
+  public StructRowData setStruct(StructLike newStruct) {
+    this.struct = newStruct;
+    return this;
+  }
+
+  @Override
+  public int getArity() {
+    return struct.size();
+  }
+
+  @Override
+  public RowKind getRowKind() {
+    return kind;
+  }
+
+  @Override
+  public void setRowKind(RowKind newKind) {
+    Preconditions.checkNotNull(newKind, "kind can not be null");
+    this.kind = newKind;
+  }
+
+  @Override
+  public boolean isNullAt(int pos) {
+    return struct.get(pos, Object.class) == null;
+  }
+
+  @Override
+  public boolean getBoolean(int pos) {
+    return struct.get(pos, Boolean.class);
+  }
+
+  @Override
+  public byte getByte(int pos) {
+    return (byte) (int) struct.get(pos, Integer.class);
+  }
+
+  @Override
+  public short getShort(int pos) {
+    return (short) (int) struct.get(pos, Integer.class);
+  }
+
+  @Override
+  public int getInt(int pos) {
+    Object integer = struct.get(pos, Object.class);
+
+    if (integer instanceof Integer) {
+      return (int) integer;
+    } else if (integer instanceof LocalDate) {
+      return (int) ((LocalDate) integer).toEpochDay();
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for int field. Type name: " + integer.getClass().getName());
+    }
+  }
+
+  @Override
+  public long getLong(int pos) {
+    Object longVal = struct.get(pos, Object.class);
+
+    if (longVal instanceof Long) {
+      return (long) longVal;
+    } else if (longVal instanceof OffsetDateTime) {
+      return Duration.between(Instant.EPOCH, (OffsetDateTime) longVal).toNanos() / 1000;
+    } else if (longVal instanceof LocalDate) {
+      return ((LocalDate) longVal).toEpochDay();
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for long field. Type name: " + longVal.getClass().getName());
+    }
+  }
+
+  @Override
+  public float getFloat(int pos) {
+    return struct.get(pos, Float.class);
+  }
+
+  @Override
+  public double getDouble(int pos) {
+    return struct.get(pos, Double.class);
+  }
+
+  @Override
+  public StringData getString(int pos) {
+    return isNullAt(pos) ? null : getStringDataInternal(pos);
+  }
+
+  private StringData getStringDataInternal(int pos) {
+    CharSequence seq = struct.get(pos, CharSequence.class);
+    return StringData.fromString(seq.toString());
+  }
+
+  @Override
+  public DecimalData getDecimal(int pos, int precision, int scale) {
+    return isNullAt(pos)
+        ? null
+        : DecimalData.fromBigDecimal(getDecimalInternal(pos), precision, scale);
+  }
+
+  private BigDecimal getDecimalInternal(int pos) {
+    return struct.get(pos, BigDecimal.class);
+  }
+
+  @Override
+  public TimestampData getTimestamp(int pos, int precision) {
+    long timeLong = getLong(pos);

Review Comment:
   you mean , we should use `TimestampData.fromEpochMillis(micros / 1000, (int) (micros % 1000) * 1000);` ?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] pvary commented on pull request #6222: Flink: Support inspecting table

Posted by GitBox <gi...@apache.org>.
pvary commented on PR #6222:
URL: https://github.com/apache/iceberg/pull/6222#issuecomment-1334856276

   @hililiwei: Hive/Spark/Impala(?) uses `SELECT * FROM prod.db.table.history` like naming convention for accessing metadata tables. See: https://iceberg.apache.org/docs/latest/spark-queries/#history
   
   Do we want to introduce a different naming convention for Flink? How widespread is the usage of the `SELECT * FROM prod.db.table$history` like SQL?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] stevenzwu commented on a diff in pull request #6222: Flink: Support inspecting table

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on code in PR #6222:
URL: https://github.com/apache/iceberg/pull/6222#discussion_r1036696525


##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java:
##########
@@ -0,0 +1,340 @@
+/*
+ * 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.Duration;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.OffsetDateTime;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.function.BiConsumer;
+import java.util.function.Function;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericMapData;
+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.flink.types.RowKind;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+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.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.ByteBuffers;
+
+public class StructRowData implements RowData {
+  private final Types.StructType type;
+  private RowKind kind;
+  private StructLike struct;
+
+  public StructRowData(Types.StructType type) {
+    this(type, RowKind.INSERT);
+  }
+
+  public StructRowData(Types.StructType type, RowKind kind) {
+    this(type, null, kind);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct) {

Review Comment:
   is this used?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] hililiwei commented on a diff in pull request #6222: Flink: Support inspecting table

Posted by GitBox <gi...@apache.org>.
hililiwei commented on code in PR #6222:
URL: https://github.com/apache/iceberg/pull/6222#discussion_r1051368174


##########
flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java:
##########
@@ -295,6 +299,161 @@ private static void assertEquals(
     }
   }
 
+  public static void assertEqualsSafe(
+      Schema schema, List<GenericData.Record> recs, List<Row> rows) {
+    Streams.forEachPair(
+        recs.stream(), rows.stream(), (rec, row) -> assertEqualsSafe(schema, rec, row));
+  }
+
+  public static void assertEqualsSafe(Schema schema, GenericData.Record rec, Row row) {
+    List<Types.NestedField> fields = schema.asStruct().fields();
+    RowType rowType = FlinkSchemaUtil.convert(schema);
+    for (int i = 0; i < fields.size(); i += 1) {
+      Type fieldType = fields.get(i).type();
+      Object expectedValue = rec.get(i);
+      Object actualValue = row.getField(i);
+      LogicalType logicalType = rowType.getTypeAt(i);
+      assertAvroEquals(fieldType, logicalType, expectedValue, actualValue);
+    }
+  }
+
+  private static void assertEqualsSafe(Types.StructType struct, GenericData.Record rec, Row row) {
+    List<Types.NestedField> fields = struct.fields();
+    for (int i = 0; i < fields.size(); i += 1) {
+      Type fieldType = fields.get(i).type();
+      Object expectedValue = rec.get(i);
+      Object actualValue = row.getField(i);
+      assertAvroEquals(fieldType, null, expectedValue, actualValue);
+    }
+  }
+
+  private static void assertAvroEquals(
+      Type type, LogicalType logicalType, Object expected, Object actual) {
+
+    if (expected == null && actual == null) {
+      return;
+    }
+
+    Assert.assertTrue(
+        "expected and actual should be both null or not null", expected != null && actual != null);
+
+    switch (type.typeId()) {
+      case BOOLEAN:

Review Comment:
   but string is a primitive type.
   
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] stevenzwu commented on a diff in pull request #6222: Flink: Support inspecting table

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on code in PR #6222:
URL: https://github.com/apache/iceberg/pull/6222#discussion_r1042761243


##########
core/src/main/java/org/apache/iceberg/BaseFilesTable.java:
##########
@@ -223,34 +225,28 @@ ManifestFile manifest() {
   static class ContentFileStructWithMetrics implements StructLike {
     private final StructLike fileAsStruct;
     private final MetricsUtil.ReadableMetricsStruct readableMetrics;
-    private final int expectedSize;
+    private final int position;
 
     ContentFileStructWithMetrics(
-        int expectedSize,
-        StructLike fileAsStruct,
-        MetricsUtil.ReadableMetricsStruct readableMetrics) {
+        int position, StructLike fileAsStruct, MetricsUtil.ReadableMetricsStruct readableMetrics) {
       this.fileAsStruct = fileAsStruct;
       this.readableMetrics = readableMetrics;
-      this.expectedSize = expectedSize;
+      this.position = position;
     }
 
     @Override
     public int size() {
-      return expectedSize;
+      return position;
     }
 
     @Override
     public <T> T get(int pos, Class<T> javaClass) {
-      int lastExpectedIndex = expectedSize - 1;
-      if (pos < lastExpectedIndex) {
+      if (pos < position) {
         return fileAsStruct.get(pos, javaClass);
-      } else if (pos == lastExpectedIndex) {
+      } else if (pos == position) {
         return javaClass.cast(readableMetrics);
       } else {
-        throw new IllegalArgumentException(

Review Comment:
   I think we should just add a Preconditions check
   ```
   Preconditions.check(pos < expectedSize, ...)
   if (pos == metricsPosition) {
     return javaClass.cast...
   } else {
    return  fileAsStruct.get(pos, javaClass);
   }
   ```
   
   not sure if the metricsPosition can be null or not. if yes, we can update the if condition a little bit.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] stevenzwu commented on a diff in pull request #6222: Flink: Support inspecting table

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on code in PR #6222:
URL: https://github.com/apache/iceberg/pull/6222#discussion_r1042723978


##########
core/src/main/java/org/apache/iceberg/BaseFilesTable.java:
##########
@@ -177,14 +177,17 @@ public CloseableIterable<StructLike> rows() {
         Set<Integer> readableMetricsIds = TypeUtil.getProjectedIds(readableMetricsField.type());
         Schema fileProjection = TypeUtil.selectNot(projection, readableMetricsIds);
 
+        int position = projection.columns().indexOf(readableMetricsField);

Review Comment:
   could it be possible that the projection schema doesn't contain readable metrics struct?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] stevenzwu commented on a diff in pull request #6222: Flink: Support inspecting table

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on code in PR #6222:
URL: https://github.com/apache/iceberg/pull/6222#discussion_r1042792168


##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java:
##########
@@ -104,11 +105,38 @@ public static Schema convert(Schema baseSchema, TableSchema flinkSchema) {
     Types.StructType struct = convert(flinkSchema).asStruct();
     // reassign ids to match the base schema
     Schema schema = TypeUtil.reassignIds(new Schema(struct.fields()), baseSchema);
+    // reassign doc to match the base schema
+    schema = reassignDoc(schema, baseSchema);
+
     // fix types that can't be represented in Flink (UUID)
     Schema fixedSchema = FlinkFixupTypes.fixup(schema, baseSchema);
     return freshIdentifierFieldIds(fixedSchema, flinkSchema);
   }
 
+  private static Schema reassignDoc(Schema schema, Schema docSourceSchema) {
+    TypeUtil.CustomOrderSchemaVisitor<Type> visitor = new FlinkFixupDoc(docSourceSchema);
+    return new Schema(
+        visitor
+            .schema(schema, new VisitFuture<>(schema.asStruct(), visitor))
+            .asStructType()
+            .fields());
+  }
+
+  private static class VisitFuture<T> implements Supplier<T> {

Review Comment:
   not sure if we need to copy this class here. can it just be a lambda function for supplier?
   ```
   () -> TypeUtil.visit(type, 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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] hililiwei commented on a diff in pull request #6222: Flink: Support inspecting table

Posted by GitBox <gi...@apache.org>.
hililiwei commented on code in PR #6222:
URL: https://github.com/apache/iceberg/pull/6222#discussion_r1043374234


##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java:
##########
@@ -140,15 +142,25 @@ public Catalog catalog() {
     return icebergCatalog;
   }
 
-  private Namespace toNamespace(String database) {
+  /** Append a new level to the base namespace */
+  private static Namespace appendLevel(Namespace baseNamespace, String newLevel) {
     String[] namespace = new String[baseNamespace.levels().length + 1];
     System.arraycopy(baseNamespace.levels(), 0, namespace, 0, baseNamespace.levels().length);
-    namespace[baseNamespace.levels().length] = database;
+    namespace[baseNamespace.levels().length] = newLevel;
     return Namespace.of(namespace);
   }
 
   TableIdentifier toIdentifier(ObjectPath path) {
-    return TableIdentifier.of(toNamespace(path.getDatabaseName()), path.getObjectName());
+    String objectName = path.getObjectName();
+    List<String> tableName = Splitter.on('$').splitToList(objectName);
+    if (tableName.size() == 2 && MetadataTableType.from(tableName.get(1)) != null) {

Review Comment:
   Thanks for your explanation, I have revised it. I think where we differ is whether the `$` can appear in the table name. In general, it is prohibited from being used in a table name. But technically, it can.
   Of course, I think it's ok to ban it.
   Thx.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] hililiwei commented on a diff in pull request #6222: Flink: Support inspecting table

Posted by GitBox <gi...@apache.org>.
hililiwei commented on code in PR #6222:
URL: https://github.com/apache/iceberg/pull/6222#discussion_r1039571147


##########
flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMetaDataTable.java:
##########
@@ -0,0 +1,713 @@
+/*
+ * 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.source;
+
+import java.io.IOException;
+import java.time.Instant;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.avro.generic.GenericData;
+import org.apache.commons.collections.ListUtils;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.CoreOptions;
+import org.apache.flink.table.api.TableEnvironment;
+import org.apache.flink.types.Row;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileContent;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Files;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.MetadataTableUtils;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.data.FileHelpers;
+import org.apache.iceberg.data.GenericRecord;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.flink.FlinkCatalogTestBase;
+import org.apache.iceberg.flink.FlinkConfigOptions;
+import org.apache.iceberg.flink.TestHelpers;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.util.SnapshotUtil;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+public class TestFlinkMetaDataTable extends FlinkCatalogTestBase {
+
+  private static final String TABLE_NAME = "test_table";
+  private final FileFormat format = FileFormat.AVRO;
+  private static TemporaryFolder temp = new TemporaryFolder();
+
+  public TestFlinkMetaDataTable(String catalogName, Namespace baseNamespace) {
+    super(catalogName, baseNamespace);
+  }
+
+  @Override
+  protected TableEnvironment getTableEnv() {
+    Configuration configuration = super.getTableEnv().getConfig().getConfiguration();
+    configuration.set(CoreOptions.DEFAULT_PARALLELISM, 1);
+    configuration.set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE, true);
+    return super.getTableEnv();
+  }
+
+  @Before
+  public void before() {
+    super.before();
+    sql("CREATE DATABASE %s", flinkDatabase);
+    sql("USE CATALOG %s", catalogName);
+    sql("USE %s", DATABASE);
+  }
+
+  @Override
+  @After
+  public void clean() {
+    sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, TABLE_NAME);
+    sql("DROP DATABASE IF EXISTS %s", flinkDatabase);
+    super.clean();
+  }
+
+  @Test
+  public void testSnapshots() {
+    sql(
+        "CREATE TABLE %s (id INT, data VARCHAR,d DOUBLE) WITH ('format-version'='2', 'write.format.default'='%s')",
+        TABLE_NAME, format.name());
+    sql(
+        "INSERT INTO %s VALUES (1,'iceberg',10),(2,'b',20),(3,CAST(NULL AS VARCHAR),30)",
+        TABLE_NAME);
+    String sql = String.format("SELECT * FROM %s$snapshots ", TABLE_NAME);
+    List<Row> result = sql(sql);
+
+    Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME));
+    Iterator<Snapshot> snapshots = table.snapshots().iterator();
+    for (Row row : result) {
+      Snapshot next = snapshots.next();
+      Assert.assertEquals(
+          "Should have expected timestamp",
+          ((Instant) row.getField(0)).toEpochMilli(),
+          next.timestampMillis());
+      Assert.assertEquals("Should have expected snapshot id", row.getField(1), next.snapshotId());

Review Comment:
   😂 It has been corrected.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] hililiwei commented on a diff in pull request #6222: Flink: Support inspecting table

Posted by GitBox <gi...@apache.org>.
hililiwei commented on code in PR #6222:
URL: https://github.com/apache/iceberg/pull/6222#discussion_r1039570176


##########
flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMetaDataTable.java:
##########
@@ -0,0 +1,713 @@
+/*
+ * 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.source;
+
+import java.io.IOException;
+import java.time.Instant;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.avro.generic.GenericData;
+import org.apache.commons.collections.ListUtils;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.CoreOptions;
+import org.apache.flink.table.api.TableEnvironment;
+import org.apache.flink.types.Row;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileContent;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Files;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.MetadataTableUtils;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.data.FileHelpers;
+import org.apache.iceberg.data.GenericRecord;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.flink.FlinkCatalogTestBase;
+import org.apache.iceberg.flink.FlinkConfigOptions;
+import org.apache.iceberg.flink.TestHelpers;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.util.SnapshotUtil;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+public class TestFlinkMetaDataTable extends FlinkCatalogTestBase {

Review Comment:
   Missed it 😂 , added.  All the types have been tested.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] hililiwei commented on a diff in pull request #6222: Flink: Support inspecting table

Posted by GitBox <gi...@apache.org>.
hililiwei commented on code in PR #6222:
URL: https://github.com/apache/iceberg/pull/6222#discussion_r1042027502


##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java:
##########
@@ -140,15 +142,25 @@ public Catalog catalog() {
     return icebergCatalog;
   }
 
-  private Namespace toNamespace(String database) {
+  /** Append a new level to the base namespace */
+  private static Namespace appendLevel(Namespace baseNamespace, String newLevel) {
     String[] namespace = new String[baseNamespace.levels().length + 1];
     System.arraycopy(baseNamespace.levels(), 0, namespace, 0, baseNamespace.levels().length);
-    namespace[baseNamespace.levels().length] = database;
+    namespace[baseNamespace.levels().length] = newLevel;
     return Namespace.of(namespace);
   }
 
   TableIdentifier toIdentifier(ObjectPath path) {
-    return TableIdentifier.of(toNamespace(path.getDatabaseName()), path.getObjectName());
+    String objectName = path.getObjectName();
+    List<String> tableName = Splitter.on('$').splitToList(objectName);
+    if (tableName.size() == 2 && MetadataTableType.from(tableName.get(1)) != null) {

Review Comment:
   `tableName.size() == 2 && MetadataTableType.from(tableName.get(1)) != null`
   This condition seems to be sufficient to pick up the metadata table, and we don't need to add additional precondition check. We just need to make sure that the non-metadata table doesn't match the if branch, and it continues with the old logic. Or have I misunderstood what you're trying to say?
   
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] stevenzwu merged pull request #6222: Flink: Support inspecting table

Posted by GitBox <gi...@apache.org>.
stevenzwu merged PR #6222:
URL: https://github.com/apache/iceberg/pull/6222


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] stevenzwu commented on a diff in pull request #6222: Flink: Support inspecting table

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on code in PR #6222:
URL: https://github.com/apache/iceberg/pull/6222#discussion_r1040360338


##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java:
##########
@@ -0,0 +1,340 @@
+/*
+ * 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.Duration;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.OffsetDateTime;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.function.BiConsumer;
+import java.util.function.Function;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericMapData;
+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.flink.types.RowKind;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+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.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.ByteBuffers;
+
+public class StructRowData implements RowData {
+  private final Types.StructType type;
+  private RowKind kind;
+  private StructLike struct;
+
+  public StructRowData(Types.StructType type) {
+    this(type, RowKind.INSERT);
+  }
+
+  public StructRowData(Types.StructType type, RowKind kind) {
+    this(type, null, kind);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct) {
+    this(type, struct, RowKind.INSERT);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct, RowKind kind) {
+    this.type = type;
+    this.struct = struct;
+    this.kind = kind;
+  }
+
+  public StructRowData setStruct(StructLike newStruct) {
+    this.struct = newStruct;
+    return this;
+  }
+
+  @Override
+  public int getArity() {
+    return struct.size();
+  }
+
+  @Override
+  public RowKind getRowKind() {
+    return kind;
+  }
+
+  @Override
+  public void setRowKind(RowKind newKind) {
+    Preconditions.checkNotNull(newKind, "kind can not be null");
+    this.kind = newKind;
+  }
+
+  @Override
+  public boolean isNullAt(int pos) {
+    return struct.get(pos, Object.class) == null;
+  }
+
+  @Override
+  public boolean getBoolean(int pos) {
+    return struct.get(pos, Boolean.class);
+  }
+
+  @Override
+  public byte getByte(int pos) {
+    return (byte) (int) struct.get(pos, Integer.class);
+  }
+
+  @Override
+  public short getShort(int pos) {
+    return (short) (int) struct.get(pos, Integer.class);
+  }
+
+  @Override
+  public int getInt(int pos) {
+    Object integer = struct.get(pos, Object.class);
+
+    if (integer instanceof Integer) {
+      return (int) integer;
+    } else if (integer instanceof LocalDate) {
+      return (int) ((LocalDate) integer).toEpochDay();
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for int field. Type name: " + integer.getClass().getName());
+    }
+  }
+
+  @Override
+  public long getLong(int pos) {
+    Object longVal = struct.get(pos, Object.class);
+
+    if (longVal instanceof Long) {
+      return (long) longVal;
+    } else if (longVal instanceof OffsetDateTime) {
+      return Duration.between(Instant.EPOCH, (OffsetDateTime) longVal).toNanos() / 1000;
+    } else if (longVal instanceof LocalDate) {
+      return ((LocalDate) longVal).toEpochDay();
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for long field. Type name: " + longVal.getClass().getName());
+    }
+  }
+
+  @Override
+  public float getFloat(int pos) {
+    return struct.get(pos, Float.class);
+  }
+
+  @Override
+  public double getDouble(int pos) {
+    return struct.get(pos, Double.class);
+  }
+
+  @Override
+  public StringData getString(int pos) {
+    return isNullAt(pos) ? null : getStringDataInternal(pos);
+  }
+
+  private StringData getStringDataInternal(int pos) {
+    CharSequence seq = struct.get(pos, CharSequence.class);
+    return StringData.fromString(seq.toString());
+  }
+
+  @Override
+  public DecimalData getDecimal(int pos, int precision, int scale) {
+    return isNullAt(pos)
+        ? null
+        : DecimalData.fromBigDecimal(getDecimalInternal(pos), precision, scale);
+  }
+
+  private BigDecimal getDecimalInternal(int pos) {
+    return struct.get(pos, BigDecimal.class);
+  }
+
+  @Override
+  public TimestampData getTimestamp(int pos, int precision) {
+    long timeLong = getLong(pos);
+    if (precision == 6) {
+      int nanosOfMillisecond = (int) (timeLong % 1000);
+      return TimestampData.fromEpochMillis(timeLong / 1000, nanosOfMillisecond);
+    } else {
+      return TimestampData.fromEpochMillis(timeLong);
+    }
+  }
+
+  @Override
+  public <T> RawValueData<T> getRawValue(int pos) {
+    return RawValueData.fromBytes(getBinary(pos));
+  }
+
+  @Override
+  public byte[] getBinary(int pos) {
+    return isNullAt(pos) ? null : getBinaryInternal(pos);
+  }
+
+  private byte[] getBinaryInternal(int pos) {
+    Object bytes = struct.get(pos, Object.class);
+
+    // should only be either ByteBuffer or byte[]
+    if (bytes instanceof ByteBuffer) {
+      return ByteBuffers.toByteArray((ByteBuffer) bytes);
+    } else if (bytes instanceof byte[]) {
+      return (byte[]) bytes;
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for binary field. Type name: " + bytes.getClass().getName());
+    }
+  }
+
+  @Override
+  public ArrayData getArray(int pos) {
+    return isNullAt(pos) ? null : getArrayInternal(pos);
+  }
+
+  private ArrayData getArrayInternal(int pos) {
+    return collectionToArrayData(
+        type.fields().get(pos).type().asListType().elementType(),
+        struct.get(pos, Collection.class));

Review Comment:
   let's do list for now. collection can be many other collection types



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] hililiwei commented on a diff in pull request #6222: Flink: Support inspecting table

Posted by GitBox <gi...@apache.org>.
hililiwei commented on code in PR #6222:
URL: https://github.com/apache/iceberg/pull/6222#discussion_r1040606681


##########
docs/flink-getting-started.md:
##########
@@ -712,9 +712,188 @@ INSERT INTO tableName /*+ OPTIONS('upsert-enabled'='true') */
 | compression-strategy   | Table write.orc.compression-strategy       | Overrides this table's compression strategy for ORC tables for this write                                  |
 
 
-## Inspecting tables.
+## Inspecting tables
 
-Iceberg does not support inspecting table in flink sql now, we need to use [iceberg's Java API](../api) to read iceberg's meta data to get those table information.
+To inspect a table's history, snapshots, and other metadata, Iceberg supports metadata tables.
+
+Metadata tables are identified by adding the metadata table name after the original table name. For example, history for `db.table` is read using `db.table$history`.
+
+### History
+
+To show table history:
+
+```sql
+SELECT * FROM prod.db.table$history;
+```
+
+| made_current_at         | snapshot_id         | parent_id           | is_current_ancestor |
+| ----------------------- | ------------------- | ------------------- | ------------------- |
+| 2019-02-08 03:29:51.215 | 5781947118336215154 | NULL                | true                |
+| 2019-02-08 03:47:55.948 | 5179299526185056830 | 5781947118336215154 | true                |
+| 2019-02-09 16:24:30.13  | 296410040247533544  | 5179299526185056830 | false               |
+| 2019-02-09 16:32:47.336 | 2999875608062437330 | 5179299526185056830 | true                |
+| 2019-02-09 19:42:03.919 | 8924558786060583479 | 2999875608062437330 | true                |
+| 2019-02-09 19:49:16.343 | 6536733823181975045 | 8924558786060583479 | true                |
+
+{{< hint info >}}
+**This shows a commit that was rolled back.** The example has two snapshots with the same parent, and one is *not* an ancestor of the current table state.

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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] hililiwei commented on a diff in pull request #6222: Flink: Support inspecting table

Posted by GitBox <gi...@apache.org>.
hililiwei commented on code in PR #6222:
URL: https://github.com/apache/iceberg/pull/6222#discussion_r1042986024


##########
core/src/main/java/org/apache/iceberg/BaseFilesTable.java:
##########
@@ -223,34 +225,28 @@ ManifestFile manifest() {
   static class ContentFileStructWithMetrics implements StructLike {
     private final StructLike fileAsStruct;
     private final MetricsUtil.ReadableMetricsStruct readableMetrics;
-    private final int expectedSize;
+    private final int position;
 
     ContentFileStructWithMetrics(
-        int expectedSize,
-        StructLike fileAsStruct,
-        MetricsUtil.ReadableMetricsStruct readableMetrics) {
+        int position, StructLike fileAsStruct, MetricsUtil.ReadableMetricsStruct readableMetrics) {
       this.fileAsStruct = fileAsStruct;
       this.readableMetrics = readableMetrics;
-      this.expectedSize = expectedSize;
+      this.position = position;
     }
 
     @Override
     public int size() {
-      return expectedSize;
+      return position;
     }
 
     @Override
     public <T> T get(int pos, Class<T> javaClass) {
-      int lastExpectedIndex = expectedSize - 1;
-      if (pos < lastExpectedIndex) {
+      if (pos < position) {
         return fileAsStruct.get(pos, javaClass);
-      } else if (pos == lastExpectedIndex) {
+      } else if (pos == position) {
         return javaClass.cast(readableMetrics);
       } else {
-        throw new IllegalArgumentException(

Review Comment:
   When pos is over the size limit, `fileAsStruct.get` method will throw an exception, we do not need to add Preconditions check, refer to the implementation of other `get` methods, I think this is the general way of it, unless we want to return a specific value, such as `null`, if it is over the limit.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] stevenzwu commented on a diff in pull request #6222: Flink: Support inspecting table

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on code in PR #6222:
URL: https://github.com/apache/iceberg/pull/6222#discussion_r1053541074


##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java:
##########
@@ -0,0 +1,300 @@
+/*
+ * 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.Duration;
+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.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericMapData;
+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.flink.types.RowKind;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.ByteBuffers;
+
+@Internal
+public class StructRowData implements RowData {
+  private final Types.StructType type;
+  private RowKind kind;
+  private StructLike struct;
+
+  public StructRowData(Types.StructType type) {
+    this(type, RowKind.INSERT);
+  }
+
+  public StructRowData(Types.StructType type, RowKind kind) {
+    this(type, null, kind);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct) {
+    this(type, struct, RowKind.INSERT);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct, RowKind kind) {
+    this.type = type;
+    this.struct = struct;
+    this.kind = kind;
+  }
+
+  public StructRowData setStruct(StructLike newStruct) {
+    this.struct = newStruct;
+    return this;
+  }
+
+  @Override
+  public int getArity() {
+    return struct.size();
+  }
+
+  @Override
+  public RowKind getRowKind() {
+    return kind;
+  }
+
+  @Override
+  public void setRowKind(RowKind newKind) {
+    Preconditions.checkNotNull(newKind, "kind can not be null");
+    this.kind = newKind;
+  }
+
+  @Override
+  public boolean isNullAt(int pos) {
+    return struct.get(pos, Object.class) == null;
+  }
+
+  @Override
+  public boolean getBoolean(int pos) {
+    return struct.get(pos, Boolean.class);
+  }
+
+  @Override
+  public byte getByte(int pos) {
+    return (byte) (int) struct.get(pos, Integer.class);
+  }
+
+  @Override
+  public short getShort(int pos) {
+    return (short) (int) struct.get(pos, Integer.class);
+  }
+
+  @Override
+  public int getInt(int pos) {
+    Object integer = struct.get(pos, Object.class);
+
+    if (integer instanceof Integer) {
+      return (int) integer;
+    } else if (integer instanceof LocalDate) {
+      return (int) ((LocalDate) integer).toEpochDay();
+    } else if (integer instanceof LocalTime) {
+      return (int) (((LocalTime) integer).toNanoOfDay() / 1000_000);
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for int field. Type name: " + integer.getClass().getName());
+    }
+  }
+
+  @Override
+  public long getLong(int pos) {
+    Object longVal = struct.get(pos, Object.class);
+
+    if (longVal instanceof Long) {
+      return (long) longVal;
+    } else if (longVal instanceof OffsetDateTime) {
+      return Duration.between(Instant.EPOCH, (OffsetDateTime) longVal).toNanos() / 1000;
+    } else if (longVal instanceof LocalDate) {
+      return ((LocalDate) longVal).toEpochDay();
+    } else if (longVal instanceof LocalTime) {
+      return ((LocalTime) longVal).toNanoOfDay();
+    } else if (longVal instanceof LocalDateTime) {
+      return Duration.between(Instant.EPOCH, ((LocalDateTime) longVal).atOffset(ZoneOffset.UTC))
+              .toNanos()
+          / 1000;
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for long field. Type name: " + longVal.getClass().getName());
+    }
+  }
+
+  @Override
+  public float getFloat(int pos) {
+    return struct.get(pos, Float.class);
+  }
+
+  @Override
+  public double getDouble(int pos) {
+    return struct.get(pos, Double.class);
+  }
+
+  @Override
+  public StringData getString(int pos) {
+    return isNullAt(pos) ? null : getStringDataInternal(pos);
+  }
+
+  private StringData getStringDataInternal(int pos) {
+    CharSequence seq = struct.get(pos, CharSequence.class);
+    return StringData.fromString(seq.toString());
+  }
+
+  @Override
+  public DecimalData getDecimal(int pos, int precision, int scale) {
+    return isNullAt(pos)
+        ? null
+        : DecimalData.fromBigDecimal(getDecimalInternal(pos), precision, scale);
+  }
+
+  private BigDecimal getDecimalInternal(int pos) {
+    return struct.get(pos, BigDecimal.class);
+  }
+
+  @Override
+  public TimestampData getTimestamp(int pos, int precision) {
+    long timeLong = getLong(pos);
+    return TimestampData.fromEpochMillis(timeLong / 1000, (int) (timeLong % 1000) * 1000);
+  }
+
+  @Override
+  public <T> RawValueData<T> getRawValue(int pos) {
+    throw new UnsupportedOperationException("Not supported yet.");
+  }
+
+  @Override
+  public byte[] getBinary(int pos) {
+    return isNullAt(pos) ? null : getBinaryInternal(pos);
+  }
+
+  private byte[] getBinaryInternal(int pos) {
+    Object bytes = struct.get(pos, Object.class);
+
+    // should only be either ByteBuffer or byte[]
+    if (bytes instanceof ByteBuffer) {
+      return ByteBuffers.toByteArray((ByteBuffer) bytes);
+    } else if (bytes instanceof byte[]) {
+      return (byte[]) bytes;
+    } else if (bytes instanceof UUID) {
+      UUID uuid = (UUID) bytes;
+      ByteBuffer bb = ByteBuffer.allocate(16);
+      bb.putLong(uuid.getMostSignificantBits());
+      bb.putLong(uuid.getLeastSignificantBits());
+      return bb.array();
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for binary field. Type name: " + bytes.getClass().getName());
+    }
+  }
+
+  @Override
+  public ArrayData getArray(int pos) {
+    return isNullAt(pos)
+        ? null
+        : (ArrayData)
+            coverValue(type.fields().get(pos).type().asListType(), struct.get(pos, List.class));
+  }
+
+  @Override
+  public MapData getMap(int pos) {
+    return isNullAt(pos)
+        ? null
+        : (MapData)
+            coverValue(type.fields().get(pos).type().asMapType(), struct.get(pos, Map.class));
+  }
+
+  @Override
+  public RowData getRow(int pos, int numFields) {
+    return isNullAt(pos) ? null : getStructRowData(pos, numFields);
+  }
+
+  private StructRowData getStructRowData(int pos, int numFields) {
+    return new StructRowData(
+        type.fields().get(pos).type().asStructType(), struct.get(pos, StructLike.class));
+  }
+
+  private <T> Object coverValue(Type elementType, Object value) {

Review Comment:
   should be `convertValue`. `cover` is a different word and meaning



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] stevenzwu commented on a diff in pull request #6222: Flink: Support inspecting table

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on code in PR #6222:
URL: https://github.com/apache/iceberg/pull/6222#discussion_r1043887176


##########
core/src/main/java/org/apache/iceberg/BaseFilesTable.java:
##########
@@ -200,13 +202,14 @@ private CloseableIterable<? extends ContentFile<?>> files(Schema fileProjection)
       }
     }
 
-    private StructLike withReadableMetrics(ContentFile<?> file) {
-      int expectedSize = projection.columns().size();
+    private StructLike withReadableMetrics(ContentFile<?> file, int metricsPosition) {
+      int columnSize = projection.columns().size();

Review Comment:
   nit: `fieldCount` or `columnCount` is probably more clear. `columnSize` can be read as the byte size of one particular column.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] hililiwei commented on a diff in pull request #6222: Flink: Support inspecting table

Posted by GitBox <gi...@apache.org>.
hililiwei commented on code in PR #6222:
URL: https://github.com/apache/iceberg/pull/6222#discussion_r1054019501


##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java:
##########
@@ -0,0 +1,300 @@
+/*
+ * 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.Duration;
+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.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericMapData;
+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.flink.types.RowKind;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.ByteBuffers;
+
+@Internal
+public class StructRowData implements RowData {
+  private final Types.StructType type;
+  private RowKind kind;
+  private StructLike struct;
+
+  public StructRowData(Types.StructType type) {
+    this(type, RowKind.INSERT);
+  }
+
+  public StructRowData(Types.StructType type, RowKind kind) {
+    this(type, null, kind);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct) {
+    this(type, struct, RowKind.INSERT);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct, RowKind kind) {
+    this.type = type;
+    this.struct = struct;
+    this.kind = kind;
+  }
+
+  public StructRowData setStruct(StructLike newStruct) {
+    this.struct = newStruct;
+    return this;
+  }
+
+  @Override
+  public int getArity() {
+    return struct.size();
+  }
+
+  @Override
+  public RowKind getRowKind() {
+    return kind;
+  }
+
+  @Override
+  public void setRowKind(RowKind newKind) {
+    Preconditions.checkNotNull(newKind, "kind can not be null");
+    this.kind = newKind;
+  }
+
+  @Override
+  public boolean isNullAt(int pos) {
+    return struct.get(pos, Object.class) == null;
+  }
+
+  @Override
+  public boolean getBoolean(int pos) {
+    return struct.get(pos, Boolean.class);
+  }
+
+  @Override
+  public byte getByte(int pos) {
+    return (byte) (int) struct.get(pos, Integer.class);
+  }
+
+  @Override
+  public short getShort(int pos) {
+    return (short) (int) struct.get(pos, Integer.class);
+  }
+
+  @Override
+  public int getInt(int pos) {
+    Object integer = struct.get(pos, Object.class);
+
+    if (integer instanceof Integer) {
+      return (int) integer;
+    } else if (integer instanceof LocalDate) {
+      return (int) ((LocalDate) integer).toEpochDay();
+    } else if (integer instanceof LocalTime) {
+      return (int) (((LocalTime) integer).toNanoOfDay() / 1000_000);
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for int field. Type name: " + integer.getClass().getName());
+    }
+  }
+
+  @Override
+  public long getLong(int pos) {
+    Object longVal = struct.get(pos, Object.class);
+
+    if (longVal instanceof Long) {
+      return (long) longVal;
+    } else if (longVal instanceof OffsetDateTime) {
+      return Duration.between(Instant.EPOCH, (OffsetDateTime) longVal).toNanos() / 1000;
+    } else if (longVal instanceof LocalDate) {
+      return ((LocalDate) longVal).toEpochDay();
+    } else if (longVal instanceof LocalTime) {
+      return ((LocalTime) longVal).toNanoOfDay();
+    } else if (longVal instanceof LocalDateTime) {
+      return Duration.between(Instant.EPOCH, ((LocalDateTime) longVal).atOffset(ZoneOffset.UTC))
+              .toNanos()
+          / 1000;
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for long field. Type name: " + longVal.getClass().getName());
+    }
+  }
+
+  @Override
+  public float getFloat(int pos) {
+    return struct.get(pos, Float.class);
+  }
+
+  @Override
+  public double getDouble(int pos) {
+    return struct.get(pos, Double.class);
+  }
+
+  @Override
+  public StringData getString(int pos) {
+    return isNullAt(pos) ? null : getStringDataInternal(pos);
+  }
+
+  private StringData getStringDataInternal(int pos) {
+    CharSequence seq = struct.get(pos, CharSequence.class);
+    return StringData.fromString(seq.toString());
+  }
+
+  @Override
+  public DecimalData getDecimal(int pos, int precision, int scale) {
+    return isNullAt(pos)
+        ? null
+        : DecimalData.fromBigDecimal(getDecimalInternal(pos), precision, scale);
+  }
+
+  private BigDecimal getDecimalInternal(int pos) {
+    return struct.get(pos, BigDecimal.class);
+  }
+
+  @Override
+  public TimestampData getTimestamp(int pos, int precision) {
+    long timeLong = getLong(pos);
+    return TimestampData.fromEpochMillis(timeLong / 1000, (int) (timeLong % 1000) * 1000);
+  }
+
+  @Override
+  public <T> RawValueData<T> getRawValue(int pos) {
+    throw new UnsupportedOperationException("Not supported yet.");
+  }
+
+  @Override
+  public byte[] getBinary(int pos) {
+    return isNullAt(pos) ? null : getBinaryInternal(pos);
+  }
+
+  private byte[] getBinaryInternal(int pos) {
+    Object bytes = struct.get(pos, Object.class);
+
+    // should only be either ByteBuffer or byte[]
+    if (bytes instanceof ByteBuffer) {
+      return ByteBuffers.toByteArray((ByteBuffer) bytes);
+    } else if (bytes instanceof byte[]) {
+      return (byte[]) bytes;
+    } else if (bytes instanceof UUID) {
+      UUID uuid = (UUID) bytes;
+      ByteBuffer bb = ByteBuffer.allocate(16);
+      bb.putLong(uuid.getMostSignificantBits());
+      bb.putLong(uuid.getLeastSignificantBits());
+      return bb.array();
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for binary field. Type name: " + bytes.getClass().getName());
+    }
+  }
+
+  @Override
+  public ArrayData getArray(int pos) {
+    return isNullAt(pos)
+        ? null
+        : (ArrayData)
+            coverValue(type.fields().get(pos).type().asListType(), struct.get(pos, List.class));
+  }
+
+  @Override
+  public MapData getMap(int pos) {
+    return isNullAt(pos)
+        ? null
+        : (MapData)
+            coverValue(type.fields().get(pos).type().asMapType(), struct.get(pos, Map.class));
+  }
+
+  @Override
+  public RowData getRow(int pos, int numFields) {
+    return isNullAt(pos) ? null : getStructRowData(pos, numFields);
+  }
+
+  private StructRowData getStructRowData(int pos, int numFields) {
+    return new StructRowData(
+        type.fields().get(pos).type().asStructType(), struct.get(pos, StructLike.class));
+  }
+
+  private <T> Object coverValue(Type elementType, Object value) {
+    switch (elementType.typeId()) {
+      case BOOLEAN:
+      case INTEGER:
+      case DATE:
+      case TIME:
+      case LONG:
+      case FLOAT:
+      case DOUBLE:
+      case DECIMAL:
+        return value;
+      case TIMESTAMP:
+        long millisecond = (long) value / 1000;
+        int nanoOfMillisecond = (int) ((Long) value % 1000) * 1000;
+        return TimestampData.fromEpochMillis(millisecond, nanoOfMillisecond);
+      case STRING:
+        return StringData.fromString(value.toString());
+      case FIXED:
+      case BINARY:
+        return ByteBuffers.toByteArray((ByteBuffer) value);
+      case STRUCT:
+        return new StructRowData(elementType.asStructType(), (StructLike) value);
+      case LIST:
+        List<Object> list = (List<Object>) value;
+        T[] array = (T[]) new Object[list.size()];

Review Comment:
   After consideration, I reversed the revision. Just adding `T` declaration doesn't seem to make much sense. What do you think? @stevenzwu 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] stevenzwu commented on a diff in pull request #6222: Flink: Support inspecting table

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on code in PR #6222:
URL: https://github.com/apache/iceberg/pull/6222#discussion_r1037760103


##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java:
##########
@@ -0,0 +1,340 @@
+/*
+ * 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.Duration;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.OffsetDateTime;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.function.BiConsumer;
+import java.util.function.Function;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericMapData;
+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.flink.types.RowKind;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+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.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.ByteBuffers;
+
+public class StructRowData implements RowData {
+  private final Types.StructType type;
+  private RowKind kind;
+  private StructLike struct;
+
+  public StructRowData(Types.StructType type) {
+    this(type, RowKind.INSERT);
+  }
+
+  public StructRowData(Types.StructType type, RowKind kind) {
+    this(type, null, kind);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct) {
+    this(type, struct, RowKind.INSERT);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct, RowKind kind) {
+    this.type = type;
+    this.struct = struct;
+    this.kind = kind;
+  }
+
+  public StructRowData setStruct(StructLike newStruct) {
+    this.struct = newStruct;
+    return this;
+  }
+
+  @Override
+  public int getArity() {
+    return struct.size();
+  }
+
+  @Override
+  public RowKind getRowKind() {
+    return kind;
+  }
+
+  @Override
+  public void setRowKind(RowKind newKind) {
+    Preconditions.checkNotNull(newKind, "kind can not be null");
+    this.kind = newKind;
+  }
+
+  @Override
+  public boolean isNullAt(int pos) {
+    return struct.get(pos, Object.class) == null;
+  }
+
+  @Override
+  public boolean getBoolean(int pos) {
+    return struct.get(pos, Boolean.class);
+  }
+
+  @Override
+  public byte getByte(int pos) {
+    return (byte) (int) struct.get(pos, Integer.class);
+  }
+
+  @Override
+  public short getShort(int pos) {
+    return (short) (int) struct.get(pos, Integer.class);
+  }
+
+  @Override
+  public int getInt(int pos) {
+    Object integer = struct.get(pos, Object.class);
+
+    if (integer instanceof Integer) {
+      return (int) integer;
+    } else if (integer instanceof LocalDate) {
+      return (int) ((LocalDate) integer).toEpochDay();
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for int field. Type name: " + integer.getClass().getName());
+    }
+  }
+
+  @Override
+  public long getLong(int pos) {
+    Object longVal = struct.get(pos, Object.class);
+
+    if (longVal instanceof Long) {
+      return (long) longVal;
+    } else if (longVal instanceof OffsetDateTime) {
+      return Duration.between(Instant.EPOCH, (OffsetDateTime) longVal).toNanos() / 1000;
+    } else if (longVal instanceof LocalDate) {
+      return ((LocalDate) longVal).toEpochDay();
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for long field. Type name: " + longVal.getClass().getName());
+    }
+  }
+
+  @Override
+  public float getFloat(int pos) {
+    return struct.get(pos, Float.class);
+  }
+
+  @Override
+  public double getDouble(int pos) {
+    return struct.get(pos, Double.class);
+  }
+
+  @Override
+  public StringData getString(int pos) {
+    return isNullAt(pos) ? null : getStringDataInternal(pos);
+  }
+
+  private StringData getStringDataInternal(int pos) {
+    CharSequence seq = struct.get(pos, CharSequence.class);
+    return StringData.fromString(seq.toString());
+  }
+
+  @Override
+  public DecimalData getDecimal(int pos, int precision, int scale) {
+    return isNullAt(pos)
+        ? null
+        : DecimalData.fromBigDecimal(getDecimalInternal(pos), precision, scale);
+  }
+
+  private BigDecimal getDecimalInternal(int pos) {
+    return struct.get(pos, BigDecimal.class);
+  }
+
+  @Override
+  public TimestampData getTimestamp(int pos, int precision) {
+    long timeLong = getLong(pos);

Review Comment:
   What is the Iceberg field type in this case? plain long or Iceberg timestamp (with or withoutTZ)?
   
   if it is Iceberg timestamp, it is always micro-second. Then `timeLong` should really be `timeMicro`.
   
   The implementation seems to assume the field is a plain long time (epoch in nanoSeconds)? is this correct?
   
   



##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java:
##########
@@ -0,0 +1,340 @@
+/*
+ * 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.Duration;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.OffsetDateTime;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.function.BiConsumer;
+import java.util.function.Function;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericMapData;
+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.flink.types.RowKind;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+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.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.ByteBuffers;
+
+public class StructRowData implements RowData {
+  private final Types.StructType type;
+  private RowKind kind;
+  private StructLike struct;
+
+  public StructRowData(Types.StructType type) {
+    this(type, RowKind.INSERT);
+  }
+
+  public StructRowData(Types.StructType type, RowKind kind) {
+    this(type, null, kind);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct) {
+    this(type, struct, RowKind.INSERT);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct, RowKind kind) {
+    this.type = type;
+    this.struct = struct;
+    this.kind = kind;
+  }
+
+  public StructRowData setStruct(StructLike newStruct) {
+    this.struct = newStruct;
+    return this;
+  }
+
+  @Override
+  public int getArity() {
+    return struct.size();
+  }
+
+  @Override
+  public RowKind getRowKind() {
+    return kind;
+  }
+
+  @Override
+  public void setRowKind(RowKind newKind) {
+    Preconditions.checkNotNull(newKind, "kind can not be null");
+    this.kind = newKind;
+  }
+
+  @Override
+  public boolean isNullAt(int pos) {
+    return struct.get(pos, Object.class) == null;
+  }
+
+  @Override
+  public boolean getBoolean(int pos) {
+    return struct.get(pos, Boolean.class);
+  }
+
+  @Override
+  public byte getByte(int pos) {
+    return (byte) (int) struct.get(pos, Integer.class);
+  }
+
+  @Override
+  public short getShort(int pos) {
+    return (short) (int) struct.get(pos, Integer.class);
+  }
+
+  @Override
+  public int getInt(int pos) {
+    Object integer = struct.get(pos, Object.class);
+
+    if (integer instanceof Integer) {
+      return (int) integer;
+    } else if (integer instanceof LocalDate) {
+      return (int) ((LocalDate) integer).toEpochDay();
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for int field. Type name: " + integer.getClass().getName());
+    }
+  }
+
+  @Override
+  public long getLong(int pos) {
+    Object longVal = struct.get(pos, Object.class);
+
+    if (longVal instanceof Long) {
+      return (long) longVal;
+    } else if (longVal instanceof OffsetDateTime) {
+      return Duration.between(Instant.EPOCH, (OffsetDateTime) longVal).toNanos() / 1000;
+    } else if (longVal instanceof LocalDate) {
+      return ((LocalDate) longVal).toEpochDay();
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for long field. Type name: " + longVal.getClass().getName());
+    }
+  }
+
+  @Override
+  public float getFloat(int pos) {
+    return struct.get(pos, Float.class);
+  }
+
+  @Override
+  public double getDouble(int pos) {
+    return struct.get(pos, Double.class);
+  }
+
+  @Override
+  public StringData getString(int pos) {
+    return isNullAt(pos) ? null : getStringDataInternal(pos);
+  }
+
+  private StringData getStringDataInternal(int pos) {
+    CharSequence seq = struct.get(pos, CharSequence.class);
+    return StringData.fromString(seq.toString());
+  }
+
+  @Override
+  public DecimalData getDecimal(int pos, int precision, int scale) {
+    return isNullAt(pos)
+        ? null
+        : DecimalData.fromBigDecimal(getDecimalInternal(pos), precision, scale);
+  }
+
+  private BigDecimal getDecimalInternal(int pos) {
+    return struct.get(pos, BigDecimal.class);
+  }
+
+  @Override
+  public TimestampData getTimestamp(int pos, int precision) {
+    long timeLong = getLong(pos);
+    if (precision == 6) {
+      int nanosOfMillisecond = (int) (timeLong % 1000);
+      return TimestampData.fromEpochMillis(timeLong / 1000, nanosOfMillisecond);
+    } else {
+      return TimestampData.fromEpochMillis(timeLong);
+    }
+  }
+
+  @Override
+  public <T> RawValueData<T> getRawValue(int pos) {
+    return RawValueData.fromBytes(getBinary(pos));
+  }
+
+  @Override
+  public byte[] getBinary(int pos) {
+    return isNullAt(pos) ? null : getBinaryInternal(pos);
+  }
+
+  private byte[] getBinaryInternal(int pos) {
+    Object bytes = struct.get(pos, Object.class);
+
+    // should only be either ByteBuffer or byte[]
+    if (bytes instanceof ByteBuffer) {
+      return ByteBuffers.toByteArray((ByteBuffer) bytes);
+    } else if (bytes instanceof byte[]) {
+      return (byte[]) bytes;
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for binary field. Type name: " + bytes.getClass().getName());
+    }
+  }
+
+  @Override
+  public ArrayData getArray(int pos) {
+    return isNullAt(pos) ? null : getArrayInternal(pos);
+  }
+
+  private ArrayData getArrayInternal(int pos) {
+    return collectionToArrayData(
+        type.fields().get(pos).type().asListType().elementType(),
+        struct.get(pos, Collection.class));

Review Comment:
   why using `Collection` here? Looking at some of the code in Iceberg, it seems that Iceberg assumes the collection to be a list.
   
   e.g. `SingleValueParser`
   ```
         case LIST:
           Preconditions.checkArgument(
               defaultValue instanceof List, "Invalid default %s value: %s", type, defaultValue);
           List<Object> defaultList = (List<Object>) defaultValue;
           Type elementType = type.asListType().elementType();
           generator.writeStartArray();
           for (Object element : defaultList) {
             toJson(elementType, element, generator);
           }
           generator.writeEndArray();
           break;
   ````
   
   `DataTestHelpers`
   ```
         case LIST:
           Assertions.assertThat(expected).as("Expected should be a List").isInstanceOf(List.class);
           Assertions.assertThat(actual).as("Actual should be a List").isInstanceOf(List.class);
           assertEquals(type.asListType(), (List) expected, (List) actual);
           break;
   ```



##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java:
##########
@@ -0,0 +1,340 @@
+/*
+ * 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.Duration;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.OffsetDateTime;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.function.BiConsumer;
+import java.util.function.Function;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericMapData;
+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.flink.types.RowKind;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+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.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.ByteBuffers;
+
+public class StructRowData implements RowData {
+  private final Types.StructType type;
+  private RowKind kind;
+  private StructLike struct;
+
+  public StructRowData(Types.StructType type) {
+    this(type, RowKind.INSERT);
+  }
+
+  public StructRowData(Types.StructType type, RowKind kind) {
+    this(type, null, kind);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct) {
+    this(type, struct, RowKind.INSERT);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct, RowKind kind) {
+    this.type = type;
+    this.struct = struct;
+    this.kind = kind;
+  }
+
+  public StructRowData setStruct(StructLike newStruct) {
+    this.struct = newStruct;
+    return this;
+  }
+
+  @Override
+  public int getArity() {
+    return struct.size();
+  }
+
+  @Override
+  public RowKind getRowKind() {
+    return kind;
+  }
+
+  @Override
+  public void setRowKind(RowKind newKind) {
+    Preconditions.checkNotNull(newKind, "kind can not be null");
+    this.kind = newKind;
+  }
+
+  @Override
+  public boolean isNullAt(int pos) {
+    return struct.get(pos, Object.class) == null;
+  }
+
+  @Override
+  public boolean getBoolean(int pos) {
+    return struct.get(pos, Boolean.class);
+  }
+
+  @Override
+  public byte getByte(int pos) {
+    return (byte) (int) struct.get(pos, Integer.class);
+  }
+
+  @Override
+  public short getShort(int pos) {
+    return (short) (int) struct.get(pos, Integer.class);
+  }
+
+  @Override
+  public int getInt(int pos) {
+    Object integer = struct.get(pos, Object.class);
+
+    if (integer instanceof Integer) {
+      return (int) integer;
+    } else if (integer instanceof LocalDate) {
+      return (int) ((LocalDate) integer).toEpochDay();
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for int field. Type name: " + integer.getClass().getName());
+    }
+  }
+
+  @Override
+  public long getLong(int pos) {
+    Object longVal = struct.get(pos, Object.class);
+
+    if (longVal instanceof Long) {
+      return (long) longVal;
+    } else if (longVal instanceof OffsetDateTime) {
+      return Duration.between(Instant.EPOCH, (OffsetDateTime) longVal).toNanos() / 1000;
+    } else if (longVal instanceof LocalDate) {
+      return ((LocalDate) longVal).toEpochDay();
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for long field. Type name: " + longVal.getClass().getName());
+    }
+  }
+
+  @Override
+  public float getFloat(int pos) {
+    return struct.get(pos, Float.class);
+  }
+
+  @Override
+  public double getDouble(int pos) {
+    return struct.get(pos, Double.class);
+  }
+
+  @Override
+  public StringData getString(int pos) {
+    return isNullAt(pos) ? null : getStringDataInternal(pos);
+  }
+
+  private StringData getStringDataInternal(int pos) {
+    CharSequence seq = struct.get(pos, CharSequence.class);
+    return StringData.fromString(seq.toString());
+  }
+
+  @Override
+  public DecimalData getDecimal(int pos, int precision, int scale) {
+    return isNullAt(pos)
+        ? null
+        : DecimalData.fromBigDecimal(getDecimalInternal(pos), precision, scale);
+  }
+
+  private BigDecimal getDecimalInternal(int pos) {
+    return struct.get(pos, BigDecimal.class);
+  }
+
+  @Override
+  public TimestampData getTimestamp(int pos, int precision) {
+    long timeLong = getLong(pos);
+    if (precision == 6) {
+      int nanosOfMillisecond = (int) (timeLong % 1000);
+      return TimestampData.fromEpochMillis(timeLong / 1000, nanosOfMillisecond);
+    } else {
+      return TimestampData.fromEpochMillis(timeLong);
+    }
+  }
+
+  @Override
+  public <T> RawValueData<T> getRawValue(int pos) {
+    return RawValueData.fromBytes(getBinary(pos));
+  }
+
+  @Override
+  public byte[] getBinary(int pos) {
+    return isNullAt(pos) ? null : getBinaryInternal(pos);
+  }
+
+  private byte[] getBinaryInternal(int pos) {
+    Object bytes = struct.get(pos, Object.class);
+
+    // should only be either ByteBuffer or byte[]
+    if (bytes instanceof ByteBuffer) {
+      return ByteBuffers.toByteArray((ByteBuffer) bytes);
+    } else if (bytes instanceof byte[]) {
+      return (byte[]) bytes;
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for binary field. Type name: " + bytes.getClass().getName());
+    }
+  }
+
+  @Override
+  public ArrayData getArray(int pos) {
+    return isNullAt(pos) ? null : getArrayInternal(pos);
+  }
+
+  private ArrayData getArrayInternal(int pos) {
+    return collectionToArrayData(
+        type.fields().get(pos).type().asListType().elementType(),
+        struct.get(pos, Collection.class));
+  }
+
+  @Override
+  public MapData getMap(int pos) {
+    return isNullAt(pos) ? null : getMapInternal(pos);
+  }
+
+  private MapData getMapInternal(int pos) {
+    return mapToMapData(type.fields().get(pos).type().asMapType(), struct.get(pos, Map.class));
+  }
+
+  @Override
+  public RowData getRow(int pos, int numFields) {
+    return isNullAt(pos) ? null : getStructRowData(pos, numFields);
+  }
+
+  private StructRowData getStructRowData(int ordinal, int numFields) {
+    return new StructRowData(
+        type.fields().get(ordinal).type().asStructType(), struct.get(ordinal, StructLike.class));
+  }
+
+  private MapData mapToMapData(Types.MapType mapType, Map<?, ?> map) {
+    // make a defensive copy to ensure entries do not change

Review Comment:
   StructRowData just apply the view of RowData on an Iceberg Struct. not sure if we should worry about the underline struct changes or not.



##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java:
##########
@@ -0,0 +1,340 @@
+/*
+ * 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.Duration;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.OffsetDateTime;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.function.BiConsumer;
+import java.util.function.Function;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericMapData;
+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.flink.types.RowKind;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+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.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.ByteBuffers;
+
+public class StructRowData implements RowData {
+  private final Types.StructType type;
+  private RowKind kind;
+  private StructLike struct;
+
+  public StructRowData(Types.StructType type) {
+    this(type, RowKind.INSERT);
+  }
+
+  public StructRowData(Types.StructType type, RowKind kind) {
+    this(type, null, kind);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct) {
+    this(type, struct, RowKind.INSERT);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct, RowKind kind) {
+    this.type = type;
+    this.struct = struct;
+    this.kind = kind;
+  }
+
+  public StructRowData setStruct(StructLike newStruct) {
+    this.struct = newStruct;
+    return this;
+  }
+
+  @Override
+  public int getArity() {
+    return struct.size();
+  }
+
+  @Override
+  public RowKind getRowKind() {
+    return kind;
+  }
+
+  @Override
+  public void setRowKind(RowKind newKind) {
+    Preconditions.checkNotNull(newKind, "kind can not be null");
+    this.kind = newKind;
+  }
+
+  @Override
+  public boolean isNullAt(int pos) {
+    return struct.get(pos, Object.class) == null;
+  }
+
+  @Override
+  public boolean getBoolean(int pos) {
+    return struct.get(pos, Boolean.class);
+  }
+
+  @Override
+  public byte getByte(int pos) {
+    return (byte) (int) struct.get(pos, Integer.class);
+  }
+
+  @Override
+  public short getShort(int pos) {
+    return (short) (int) struct.get(pos, Integer.class);
+  }
+
+  @Override
+  public int getInt(int pos) {
+    Object integer = struct.get(pos, Object.class);
+
+    if (integer instanceof Integer) {
+      return (int) integer;
+    } else if (integer instanceof LocalDate) {
+      return (int) ((LocalDate) integer).toEpochDay();
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for int field. Type name: " + integer.getClass().getName());
+    }
+  }
+
+  @Override
+  public long getLong(int pos) {
+    Object longVal = struct.get(pos, Object.class);
+
+    if (longVal instanceof Long) {
+      return (long) longVal;
+    } else if (longVal instanceof OffsetDateTime) {
+      return Duration.between(Instant.EPOCH, (OffsetDateTime) longVal).toNanos() / 1000;
+    } else if (longVal instanceof LocalDate) {
+      return ((LocalDate) longVal).toEpochDay();
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for long field. Type name: " + longVal.getClass().getName());
+    }
+  }
+
+  @Override
+  public float getFloat(int pos) {
+    return struct.get(pos, Float.class);
+  }
+
+  @Override
+  public double getDouble(int pos) {
+    return struct.get(pos, Double.class);
+  }
+
+  @Override
+  public StringData getString(int pos) {
+    return isNullAt(pos) ? null : getStringDataInternal(pos);
+  }
+
+  private StringData getStringDataInternal(int pos) {
+    CharSequence seq = struct.get(pos, CharSequence.class);
+    return StringData.fromString(seq.toString());
+  }
+
+  @Override
+  public DecimalData getDecimal(int pos, int precision, int scale) {
+    return isNullAt(pos)
+        ? null
+        : DecimalData.fromBigDecimal(getDecimalInternal(pos), precision, scale);
+  }
+
+  private BigDecimal getDecimalInternal(int pos) {
+    return struct.get(pos, BigDecimal.class);
+  }
+
+  @Override
+  public TimestampData getTimestamp(int pos, int precision) {
+    long timeLong = getLong(pos);
+    if (precision == 6) {
+      int nanosOfMillisecond = (int) (timeLong % 1000);
+      return TimestampData.fromEpochMillis(timeLong / 1000, nanosOfMillisecond);
+    } else {
+      return TimestampData.fromEpochMillis(timeLong);
+    }
+  }
+
+  @Override
+  public <T> RawValueData<T> getRawValue(int pos) {
+    return RawValueData.fromBytes(getBinary(pos));
+  }
+
+  @Override
+  public byte[] getBinary(int pos) {
+    return isNullAt(pos) ? null : getBinaryInternal(pos);
+  }
+
+  private byte[] getBinaryInternal(int pos) {
+    Object bytes = struct.get(pos, Object.class);
+
+    // should only be either ByteBuffer or byte[]
+    if (bytes instanceof ByteBuffer) {
+      return ByteBuffers.toByteArray((ByteBuffer) bytes);
+    } else if (bytes instanceof byte[]) {
+      return (byte[]) bytes;
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for binary field. Type name: " + bytes.getClass().getName());
+    }
+  }
+
+  @Override
+  public ArrayData getArray(int pos) {
+    return isNullAt(pos) ? null : getArrayInternal(pos);
+  }
+
+  private ArrayData getArrayInternal(int pos) {
+    return collectionToArrayData(
+        type.fields().get(pos).type().asListType().elementType(),
+        struct.get(pos, Collection.class));
+  }
+
+  @Override
+  public MapData getMap(int pos) {
+    return isNullAt(pos) ? null : getMapInternal(pos);
+  }
+
+  private MapData getMapInternal(int pos) {
+    return mapToMapData(type.fields().get(pos).type().asMapType(), struct.get(pos, Map.class));

Review Comment:
   nit: `mapToMapData ` seems only used by this private method. merge it in this method?



##########
flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMetaDataTable.java:
##########
@@ -0,0 +1,713 @@
+/*
+ * 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.source;
+
+import java.io.IOException;
+import java.time.Instant;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.avro.generic.GenericData;
+import org.apache.commons.collections.ListUtils;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.CoreOptions;
+import org.apache.flink.table.api.TableEnvironment;
+import org.apache.flink.types.Row;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileContent;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Files;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.MetadataTableUtils;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.data.FileHelpers;
+import org.apache.iceberg.data.GenericRecord;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.flink.FlinkCatalogTestBase;
+import org.apache.iceberg.flink.FlinkConfigOptions;
+import org.apache.iceberg.flink.TestHelpers;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.util.SnapshotUtil;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+public class TestFlinkMetaDataTable extends FlinkCatalogTestBase {
+
+  private static final String TABLE_NAME = "test_table";
+  private final FileFormat format = FileFormat.AVRO;
+  private static TemporaryFolder temp = new TemporaryFolder();
+
+  public TestFlinkMetaDataTable(String catalogName, Namespace baseNamespace) {
+    super(catalogName, baseNamespace);
+  }
+
+  @Override
+  protected TableEnvironment getTableEnv() {
+    Configuration configuration = super.getTableEnv().getConfig().getConfiguration();
+    configuration.set(CoreOptions.DEFAULT_PARALLELISM, 1);
+    configuration.set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE, true);
+    return super.getTableEnv();
+  }
+
+  @Before
+  public void before() {
+    super.before();
+    sql("CREATE DATABASE %s", flinkDatabase);
+    sql("USE CATALOG %s", catalogName);
+    sql("USE %s", DATABASE);
+  }
+
+  @Override
+  @After
+  public void clean() {
+    sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, TABLE_NAME);
+    sql("DROP DATABASE IF EXISTS %s", flinkDatabase);
+    super.clean();
+  }
+
+  @Test
+  public void testSnapshots() {
+    sql(

Review Comment:
   should `CREATE TABLE` be done in the `before` method to pair with the `DROP TABLE` in the `after` method?



##########
flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMetaDataTable.java:
##########
@@ -0,0 +1,713 @@
+/*
+ * 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.source;
+
+import java.io.IOException;
+import java.time.Instant;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.avro.generic.GenericData;
+import org.apache.commons.collections.ListUtils;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.CoreOptions;
+import org.apache.flink.table.api.TableEnvironment;
+import org.apache.flink.types.Row;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileContent;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Files;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.MetadataTableUtils;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.data.FileHelpers;
+import org.apache.iceberg.data.GenericRecord;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.flink.FlinkCatalogTestBase;
+import org.apache.iceberg.flink.FlinkConfigOptions;
+import org.apache.iceberg.flink.TestHelpers;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.util.SnapshotUtil;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+public class TestFlinkMetaDataTable extends FlinkCatalogTestBase {
+
+  private static final String TABLE_NAME = "test_table";
+  private final FileFormat format = FileFormat.AVRO;
+  private static TemporaryFolder temp = new TemporaryFolder();
+
+  public TestFlinkMetaDataTable(String catalogName, Namespace baseNamespace) {
+    super(catalogName, baseNamespace);
+  }
+
+  @Override
+  protected TableEnvironment getTableEnv() {
+    Configuration configuration = super.getTableEnv().getConfig().getConfiguration();
+    configuration.set(CoreOptions.DEFAULT_PARALLELISM, 1);
+    configuration.set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE, true);
+    return super.getTableEnv();
+  }
+
+  @Before
+  public void before() {
+    super.before();
+    sql("CREATE DATABASE %s", flinkDatabase);
+    sql("USE CATALOG %s", catalogName);
+    sql("USE %s", DATABASE);
+  }
+
+  @Override
+  @After
+  public void clean() {
+    sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, TABLE_NAME);
+    sql("DROP DATABASE IF EXISTS %s", flinkDatabase);
+    super.clean();
+  }
+
+  @Test
+  public void testSnapshots() {
+    sql(
+        "CREATE TABLE %s (id INT, data VARCHAR,d DOUBLE) WITH ('format-version'='2', 'write.format.default'='%s')",
+        TABLE_NAME, format.name());
+    sql(
+        "INSERT INTO %s VALUES (1,'iceberg',10),(2,'b',20),(3,CAST(NULL AS VARCHAR),30)",
+        TABLE_NAME);
+    String sql = String.format("SELECT * FROM %s$snapshots ", TABLE_NAME);
+    List<Row> result = sql(sql);
+
+    Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME));
+    Iterator<Snapshot> snapshots = table.snapshots().iterator();
+    for (Row row : result) {
+      Snapshot next = snapshots.next();
+      Assert.assertEquals(
+          "Should have expected timestamp",
+          ((Instant) row.getField(0)).toEpochMilli(),
+          next.timestampMillis());
+      Assert.assertEquals("Should have expected snapshot id", row.getField(1), next.snapshotId());

Review Comment:
   I think we got the expected and actual reversed in the assertion.



##########
flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMetaDataTable.java:
##########
@@ -0,0 +1,713 @@
+/*
+ * 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.source;
+
+import java.io.IOException;
+import java.time.Instant;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.avro.generic.GenericData;
+import org.apache.commons.collections.ListUtils;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.CoreOptions;
+import org.apache.flink.table.api.TableEnvironment;
+import org.apache.flink.types.Row;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileContent;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Files;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.MetadataTableUtils;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.data.FileHelpers;
+import org.apache.iceberg.data.GenericRecord;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.flink.FlinkCatalogTestBase;
+import org.apache.iceberg.flink.FlinkConfigOptions;
+import org.apache.iceberg.flink.TestHelpers;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.util.SnapshotUtil;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+public class TestFlinkMetaDataTable extends FlinkCatalogTestBase {
+
+  private static final String TABLE_NAME = "test_table";
+  private final FileFormat format = FileFormat.AVRO;
+  private static TemporaryFolder temp = new TemporaryFolder();
+
+  public TestFlinkMetaDataTable(String catalogName, Namespace baseNamespace) {
+    super(catalogName, baseNamespace);
+  }
+
+  @Override
+  protected TableEnvironment getTableEnv() {
+    Configuration configuration = super.getTableEnv().getConfig().getConfiguration();
+    configuration.set(CoreOptions.DEFAULT_PARALLELISM, 1);
+    configuration.set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE, true);
+    return super.getTableEnv();
+  }
+
+  @Before
+  public void before() {
+    super.before();
+    sql("CREATE DATABASE %s", flinkDatabase);
+    sql("USE CATALOG %s", catalogName);
+    sql("USE %s", DATABASE);
+  }
+
+  @Override
+  @After
+  public void clean() {
+    sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, TABLE_NAME);
+    sql("DROP DATABASE IF EXISTS %s", flinkDatabase);
+    super.clean();
+  }
+
+  @Test
+  public void testSnapshots() {
+    sql(
+        "CREATE TABLE %s (id INT, data VARCHAR,d DOUBLE) WITH ('format-version'='2', 'write.format.default'='%s')",
+        TABLE_NAME, format.name());
+    sql(
+        "INSERT INTO %s VALUES (1,'iceberg',10),(2,'b',20),(3,CAST(NULL AS VARCHAR),30)",
+        TABLE_NAME);
+    String sql = String.format("SELECT * FROM %s$snapshots ", TABLE_NAME);
+    List<Row> result = sql(sql);
+
+    Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME));
+    Iterator<Snapshot> snapshots = table.snapshots().iterator();
+    for (Row row : result) {
+      Snapshot next = snapshots.next();
+      Assert.assertEquals(
+          "Should have expected timestamp",
+          ((Instant) row.getField(0)).toEpochMilli(),
+          next.timestampMillis());
+      Assert.assertEquals("Should have expected snapshot id", row.getField(1), next.snapshotId());
+      Assert.assertEquals("Should have expected parent id", row.getField(2), next.parentId());
+      Assert.assertEquals("Should have expected operation", row.getField(3), next.operation());
+      Assert.assertEquals(
+          "Should have expected manifest list location",
+          row.getField(4),
+          next.manifestListLocation());
+      Assert.assertEquals("Should have expected summary", row.getField(5), next.summary());
+    }
+  }
+
+  @Test
+  public void testHistory() {
+    sql(
+        "CREATE TABLE %s (id INT, data VARCHAR,d DOUBLE) WITH ('format-version'='2', 'write.format.default'='%s')",
+        TABLE_NAME, format.name());
+    sql(
+        "INSERT INTO %s VALUES (1,'iceberg',10),(2,'b',20),(3,CAST(NULL AS VARCHAR),30)",
+        TABLE_NAME);
+    String sql = String.format("SELECT * FROM %s$history ", TABLE_NAME);
+    List<Row> result = sql(sql);
+
+    Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME));
+    Iterator<Snapshot> snapshots = table.snapshots().iterator();
+    for (Row row : result) {
+      Snapshot next = snapshots.next();
+      Assert.assertEquals(
+          "Should have expected made_current_at",
+          ((Instant) row.getField(0)).toEpochMilli(),
+          next.timestampMillis());
+      Assert.assertEquals("Should have expected snapshot id", row.getField(1), next.snapshotId());
+      Assert.assertEquals("Should have expected parent id", row.getField(2), next.parentId());
+      Assert.assertEquals(

Review Comment:
   this is where having multiple snapshots can make the verification more comprehensive. E.g., it wasn't immediately clear to me what `is_current_ancestor` means. is it only for the table's current snapshot? 



##########
flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMetaDataTable.java:
##########
@@ -0,0 +1,713 @@
+/*
+ * 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.source;
+
+import java.io.IOException;
+import java.time.Instant;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.avro.generic.GenericData;
+import org.apache.commons.collections.ListUtils;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.CoreOptions;
+import org.apache.flink.table.api.TableEnvironment;
+import org.apache.flink.types.Row;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileContent;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Files;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.MetadataTableUtils;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.data.FileHelpers;
+import org.apache.iceberg.data.GenericRecord;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.flink.FlinkCatalogTestBase;
+import org.apache.iceberg.flink.FlinkConfigOptions;
+import org.apache.iceberg.flink.TestHelpers;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.util.SnapshotUtil;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+public class TestFlinkMetaDataTable extends FlinkCatalogTestBase {
+
+  private static final String TABLE_NAME = "test_table";
+  private final FileFormat format = FileFormat.AVRO;
+  private static TemporaryFolder temp = new TemporaryFolder();
+
+  public TestFlinkMetaDataTable(String catalogName, Namespace baseNamespace) {
+    super(catalogName, baseNamespace);
+  }
+
+  @Override
+  protected TableEnvironment getTableEnv() {
+    Configuration configuration = super.getTableEnv().getConfig().getConfiguration();
+    configuration.set(CoreOptions.DEFAULT_PARALLELISM, 1);
+    configuration.set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE, true);
+    return super.getTableEnv();
+  }
+
+  @Before
+  public void before() {
+    super.before();
+    sql("CREATE DATABASE %s", flinkDatabase);
+    sql("USE CATALOG %s", catalogName);
+    sql("USE %s", DATABASE);
+  }
+
+  @Override
+  @After
+  public void clean() {
+    sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, TABLE_NAME);
+    sql("DROP DATABASE IF EXISTS %s", flinkDatabase);
+    super.clean();
+  }
+
+  @Test
+  public void testSnapshots() {
+    sql(
+        "CREATE TABLE %s (id INT, data VARCHAR,d DOUBLE) WITH ('format-version'='2', 'write.format.default'='%s')",
+        TABLE_NAME, format.name());
+    sql(
+        "INSERT INTO %s VALUES (1,'iceberg',10),(2,'b',20),(3,CAST(NULL AS VARCHAR),30)",
+        TABLE_NAME);
+    String sql = String.format("SELECT * FROM %s$snapshots ", TABLE_NAME);
+    List<Row> result = sql(sql);
+
+    Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME));
+    Iterator<Snapshot> snapshots = table.snapshots().iterator();
+    for (Row row : result) {
+      Snapshot next = snapshots.next();
+      Assert.assertEquals(
+          "Should have expected timestamp",
+          ((Instant) row.getField(0)).toEpochMilli(),
+          next.timestampMillis());
+      Assert.assertEquals("Should have expected snapshot id", row.getField(1), next.snapshotId());
+      Assert.assertEquals("Should have expected parent id", row.getField(2), next.parentId());
+      Assert.assertEquals("Should have expected operation", row.getField(3), next.operation());
+      Assert.assertEquals(
+          "Should have expected manifest list location",
+          row.getField(4),
+          next.manifestListLocation());
+      Assert.assertEquals("Should have expected summary", row.getField(5), next.summary());
+    }
+  }
+
+  @Test
+  public void testHistory() {
+    sql(
+        "CREATE TABLE %s (id INT, data VARCHAR,d DOUBLE) WITH ('format-version'='2', 'write.format.default'='%s')",
+        TABLE_NAME, format.name());
+    sql(
+        "INSERT INTO %s VALUES (1,'iceberg',10),(2,'b',20),(3,CAST(NULL AS VARCHAR),30)",
+        TABLE_NAME);
+    String sql = String.format("SELECT * FROM %s$history ", TABLE_NAME);
+    List<Row> result = sql(sql);
+
+    Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME));
+    Iterator<Snapshot> snapshots = table.snapshots().iterator();
+    for (Row row : result) {
+      Snapshot next = snapshots.next();
+      Assert.assertEquals(
+          "Should have expected made_current_at",
+          ((Instant) row.getField(0)).toEpochMilli(),
+          next.timestampMillis());
+      Assert.assertEquals("Should have expected snapshot id", row.getField(1), next.snapshotId());
+      Assert.assertEquals("Should have expected parent id", row.getField(2), next.parentId());
+      Assert.assertEquals(
+          "Should have expected is current ancestor",
+          row.getField(3),
+          SnapshotUtil.isAncestorOf(
+              table, next.snapshotId(), table.currentSnapshot().snapshotId()));
+    }
+  }
+
+  @Test
+  public void testManifests() {
+    sql(
+        "CREATE TABLE %s (id INT, data VARCHAR,d DOUBLE) WITH ('format-version'='2', 'write.format.default'='%s')",
+        TABLE_NAME, format.name());
+    sql(
+        "INSERT INTO %s VALUES (1,'iceberg',10),(2,'b',20),(3,CAST(NULL AS VARCHAR),30)",
+        TABLE_NAME);
+    String sql = String.format("SELECT * FROM %s$manifests ", TABLE_NAME);
+    List<Row> result = sql(sql);
+
+    Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME));
+    List<ManifestFile> expectedDataManifests = dataManifests(table);
+
+    for (int i = 0; i < result.size(); i++) {
+      Row row = result.get(i);
+      ManifestFile manifestFile = expectedDataManifests.get(i);
+      Assert.assertEquals(
+          "Should have expected content", row.getField(0), manifestFile.content().id());
+      Assert.assertEquals("Should have expected path", row.getField(1), manifestFile.path());
+      Assert.assertEquals("Should have expected length", row.getField(2), manifestFile.length());
+      Assert.assertEquals(
+          "Should have expected partition_spec_id",
+          row.getField(3),
+          manifestFile.partitionSpecId());
+      Assert.assertEquals(
+          "Should have expected added_snapshot_id", row.getField(4), manifestFile.snapshotId());
+      Assert.assertEquals(
+          "Should have expected added_data_files_count",
+          row.getField(5),
+          manifestFile.addedFilesCount());
+      Assert.assertEquals(
+          "Should have expected existing_data_files_count",
+          row.getField(6),
+          manifestFile.existingFilesCount());
+      Assert.assertEquals(
+          "Should have expected deleted_data_files_count",
+          row.getField(7),
+          manifestFile.deletedFilesCount());
+    }
+  }
+
+  @Test
+  public void testAllManifests() {

Review Comment:
   we should create the scenario where `all_manifests` test returns different results as the `manifests` test above.



##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java:
##########
@@ -0,0 +1,340 @@
+/*
+ * 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.Duration;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.OffsetDateTime;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.function.BiConsumer;
+import java.util.function.Function;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericMapData;
+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.flink.types.RowKind;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+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.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.ByteBuffers;
+
+public class StructRowData implements RowData {
+  private final Types.StructType type;
+  private RowKind kind;
+  private StructLike struct;
+
+  public StructRowData(Types.StructType type) {
+    this(type, RowKind.INSERT);
+  }
+
+  public StructRowData(Types.StructType type, RowKind kind) {
+    this(type, null, kind);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct) {
+    this(type, struct, RowKind.INSERT);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct, RowKind kind) {
+    this.type = type;
+    this.struct = struct;
+    this.kind = kind;
+  }
+
+  public StructRowData setStruct(StructLike newStruct) {
+    this.struct = newStruct;
+    return this;
+  }
+
+  @Override
+  public int getArity() {
+    return struct.size();
+  }
+
+  @Override
+  public RowKind getRowKind() {
+    return kind;
+  }
+
+  @Override
+  public void setRowKind(RowKind newKind) {
+    Preconditions.checkNotNull(newKind, "kind can not be null");
+    this.kind = newKind;
+  }
+
+  @Override
+  public boolean isNullAt(int pos) {
+    return struct.get(pos, Object.class) == null;
+  }
+
+  @Override
+  public boolean getBoolean(int pos) {
+    return struct.get(pos, Boolean.class);
+  }
+
+  @Override
+  public byte getByte(int pos) {
+    return (byte) (int) struct.get(pos, Integer.class);
+  }
+
+  @Override
+  public short getShort(int pos) {
+    return (short) (int) struct.get(pos, Integer.class);
+  }
+
+  @Override
+  public int getInt(int pos) {
+    Object integer = struct.get(pos, Object.class);
+
+    if (integer instanceof Integer) {
+      return (int) integer;
+    } else if (integer instanceof LocalDate) {
+      return (int) ((LocalDate) integer).toEpochDay();
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for int field. Type name: " + integer.getClass().getName());
+    }
+  }
+
+  @Override
+  public long getLong(int pos) {
+    Object longVal = struct.get(pos, Object.class);
+
+    if (longVal instanceof Long) {
+      return (long) longVal;
+    } else if (longVal instanceof OffsetDateTime) {
+      return Duration.between(Instant.EPOCH, (OffsetDateTime) longVal).toNanos() / 1000;
+    } else if (longVal instanceof LocalDate) {
+      return ((LocalDate) longVal).toEpochDay();
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for long field. Type name: " + longVal.getClass().getName());
+    }
+  }
+
+  @Override
+  public float getFloat(int pos) {
+    return struct.get(pos, Float.class);
+  }
+
+  @Override
+  public double getDouble(int pos) {
+    return struct.get(pos, Double.class);
+  }
+
+  @Override
+  public StringData getString(int pos) {
+    return isNullAt(pos) ? null : getStringDataInternal(pos);
+  }
+
+  private StringData getStringDataInternal(int pos) {
+    CharSequence seq = struct.get(pos, CharSequence.class);
+    return StringData.fromString(seq.toString());
+  }
+
+  @Override
+  public DecimalData getDecimal(int pos, int precision, int scale) {
+    return isNullAt(pos)
+        ? null
+        : DecimalData.fromBigDecimal(getDecimalInternal(pos), precision, scale);
+  }
+
+  private BigDecimal getDecimalInternal(int pos) {
+    return struct.get(pos, BigDecimal.class);
+  }
+
+  @Override
+  public TimestampData getTimestamp(int pos, int precision) {
+    long timeLong = getLong(pos);
+    if (precision == 6) {
+      int nanosOfMillisecond = (int) (timeLong % 1000);
+      return TimestampData.fromEpochMillis(timeLong / 1000, nanosOfMillisecond);
+    } else {
+      return TimestampData.fromEpochMillis(timeLong);
+    }
+  }
+
+  @Override
+  public <T> RawValueData<T> getRawValue(int pos) {
+    return RawValueData.fromBytes(getBinary(pos));
+  }
+
+  @Override
+  public byte[] getBinary(int pos) {
+    return isNullAt(pos) ? null : getBinaryInternal(pos);
+  }
+
+  private byte[] getBinaryInternal(int pos) {
+    Object bytes = struct.get(pos, Object.class);
+
+    // should only be either ByteBuffer or byte[]
+    if (bytes instanceof ByteBuffer) {
+      return ByteBuffers.toByteArray((ByteBuffer) bytes);
+    } else if (bytes instanceof byte[]) {
+      return (byte[]) bytes;
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for binary field. Type name: " + bytes.getClass().getName());
+    }
+  }
+
+  @Override
+  public ArrayData getArray(int pos) {
+    return isNullAt(pos) ? null : getArrayInternal(pos);
+  }
+
+  private ArrayData getArrayInternal(int pos) {
+    return collectionToArrayData(
+        type.fields().get(pos).type().asListType().elementType(),
+        struct.get(pos, Collection.class));
+  }
+
+  @Override
+  public MapData getMap(int pos) {
+    return isNullAt(pos) ? null : getMapInternal(pos);
+  }
+
+  private MapData getMapInternal(int pos) {
+    return mapToMapData(type.fields().get(pos).type().asMapType(), struct.get(pos, Map.class));
+  }
+
+  @Override
+  public RowData getRow(int pos, int numFields) {
+    return isNullAt(pos) ? null : getStructRowData(pos, numFields);
+  }
+
+  private StructRowData getStructRowData(int ordinal, int numFields) {

Review Comment:
   nit: `ordinal` -> `pos` for consistency



##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java:
##########
@@ -140,15 +142,25 @@ public Catalog catalog() {
     return icebergCatalog;
   }
 
-  private Namespace toNamespace(String database) {
+  /** Append a new level to the base namespace */
+  private static Namespace appendLevel(Namespace baseNamespace, String newLevel) {
     String[] namespace = new String[baseNamespace.levels().length + 1];
     System.arraycopy(baseNamespace.levels(), 0, namespace, 0, baseNamespace.levels().length);
-    namespace[baseNamespace.levels().length] = database;
+    namespace[baseNamespace.levels().length] = newLevel;
     return Namespace.of(namespace);
   }
 
   TableIdentifier toIdentifier(ObjectPath path) {
-    return TableIdentifier.of(toNamespace(path.getDatabaseName()), path.getObjectName());
+    String objectName = path.getObjectName();
+    List<String> tableName = Splitter.on('$').splitToList(objectName);
+    if (tableName.size() == 2 && MetadataTableType.from(tableName.get(1)) != null) {

Review Comment:
   This check is still not quite right. the size should be either 0 or 2. 
   
   Should this be a precondition check?
   ```
   MetadataTableType.from(tableName.get(1)) != null
   ```



##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java:
##########
@@ -0,0 +1,340 @@
+/*
+ * 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.Duration;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.OffsetDateTime;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.function.BiConsumer;
+import java.util.function.Function;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericMapData;
+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.flink.types.RowKind;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+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.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.ByteBuffers;
+
+public class StructRowData implements RowData {
+  private final Types.StructType type;
+  private RowKind kind;
+  private StructLike struct;
+
+  public StructRowData(Types.StructType type) {
+    this(type, RowKind.INSERT);
+  }
+
+  public StructRowData(Types.StructType type, RowKind kind) {
+    this(type, null, kind);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct) {
+    this(type, struct, RowKind.INSERT);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct, RowKind kind) {
+    this.type = type;
+    this.struct = struct;
+    this.kind = kind;
+  }
+
+  public StructRowData setStruct(StructLike newStruct) {
+    this.struct = newStruct;
+    return this;
+  }
+
+  @Override
+  public int getArity() {
+    return struct.size();
+  }
+
+  @Override
+  public RowKind getRowKind() {
+    return kind;
+  }
+
+  @Override
+  public void setRowKind(RowKind newKind) {
+    Preconditions.checkNotNull(newKind, "kind can not be null");
+    this.kind = newKind;
+  }
+
+  @Override
+  public boolean isNullAt(int pos) {
+    return struct.get(pos, Object.class) == null;
+  }
+
+  @Override
+  public boolean getBoolean(int pos) {
+    return struct.get(pos, Boolean.class);
+  }
+
+  @Override
+  public byte getByte(int pos) {
+    return (byte) (int) struct.get(pos, Integer.class);
+  }
+
+  @Override
+  public short getShort(int pos) {
+    return (short) (int) struct.get(pos, Integer.class);
+  }
+
+  @Override
+  public int getInt(int pos) {
+    Object integer = struct.get(pos, Object.class);
+
+    if (integer instanceof Integer) {
+      return (int) integer;
+    } else if (integer instanceof LocalDate) {
+      return (int) ((LocalDate) integer).toEpochDay();
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for int field. Type name: " + integer.getClass().getName());
+    }
+  }
+
+  @Override
+  public long getLong(int pos) {
+    Object longVal = struct.get(pos, Object.class);
+
+    if (longVal instanceof Long) {
+      return (long) longVal;
+    } else if (longVal instanceof OffsetDateTime) {
+      return Duration.between(Instant.EPOCH, (OffsetDateTime) longVal).toNanos() / 1000;
+    } else if (longVal instanceof LocalDate) {
+      return ((LocalDate) longVal).toEpochDay();
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for long field. Type name: " + longVal.getClass().getName());
+    }
+  }
+
+  @Override
+  public float getFloat(int pos) {
+    return struct.get(pos, Float.class);
+  }
+
+  @Override
+  public double getDouble(int pos) {
+    return struct.get(pos, Double.class);
+  }
+
+  @Override
+  public StringData getString(int pos) {
+    return isNullAt(pos) ? null : getStringDataInternal(pos);
+  }
+
+  private StringData getStringDataInternal(int pos) {
+    CharSequence seq = struct.get(pos, CharSequence.class);
+    return StringData.fromString(seq.toString());
+  }
+
+  @Override
+  public DecimalData getDecimal(int pos, int precision, int scale) {
+    return isNullAt(pos)
+        ? null
+        : DecimalData.fromBigDecimal(getDecimalInternal(pos), precision, scale);
+  }
+
+  private BigDecimal getDecimalInternal(int pos) {
+    return struct.get(pos, BigDecimal.class);
+  }
+
+  @Override
+  public TimestampData getTimestamp(int pos, int precision) {
+    long timeLong = getLong(pos);
+    if (precision == 6) {
+      int nanosOfMillisecond = (int) (timeLong % 1000);
+      return TimestampData.fromEpochMillis(timeLong / 1000, nanosOfMillisecond);
+    } else {
+      return TimestampData.fromEpochMillis(timeLong);
+    }
+  }
+
+  @Override
+  public <T> RawValueData<T> getRawValue(int pos) {
+    return RawValueData.fromBytes(getBinary(pos));

Review Comment:
   Looking at the impl from Flink `GenericRowData`, I am wondering if we should throw `UnsupportedOperationException` here, as Iceberg `StructLike` can't contain `RawValueData` object internally.
   
   ```
       @Override
       public <T> RawValueData<T> getRawValue(int pos) {
           return (RawValueData<T>) this.fields[pos];
       }
   ```
   
   if the underline field is a binary field, caller can just call the `getBinary` method below, right?



##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java:
##########
@@ -0,0 +1,340 @@
+/*
+ * 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.Duration;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.OffsetDateTime;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.function.BiConsumer;
+import java.util.function.Function;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericMapData;
+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.flink.types.RowKind;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+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.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.ByteBuffers;
+
+public class StructRowData implements RowData {

Review Comment:
   nit: add `@Internal` annotation



##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/MetaDataReaderFunction.java:
##########
@@ -0,0 +1,62 @@
+/*
+ * 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.source.reader;
+
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.table.data.RowData;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.encryption.EncryptionManager;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.flink.source.DataIterator;
+import org.apache.iceberg.flink.source.DataTaskReader;
+import org.apache.iceberg.flink.source.split.IcebergSourceSplit;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class MetaDataReaderFunction extends DataIteratorReaderFunction<RowData> {

Review Comment:
   nit: add `@Internal` annotation



##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java:
##########
@@ -0,0 +1,340 @@
+/*
+ * 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.Duration;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.OffsetDateTime;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.function.BiConsumer;
+import java.util.function.Function;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericMapData;
+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.flink.types.RowKind;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+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.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.ByteBuffers;
+
+public class StructRowData implements RowData {
+  private final Types.StructType type;
+  private RowKind kind;
+  private StructLike struct;
+
+  public StructRowData(Types.StructType type) {
+    this(type, RowKind.INSERT);
+  }
+
+  public StructRowData(Types.StructType type, RowKind kind) {
+    this(type, null, kind);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct) {
+    this(type, struct, RowKind.INSERT);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct, RowKind kind) {
+    this.type = type;
+    this.struct = struct;
+    this.kind = kind;
+  }
+
+  public StructRowData setStruct(StructLike newStruct) {
+    this.struct = newStruct;
+    return this;
+  }
+
+  @Override
+  public int getArity() {
+    return struct.size();
+  }
+
+  @Override
+  public RowKind getRowKind() {
+    return kind;
+  }
+
+  @Override
+  public void setRowKind(RowKind newKind) {
+    Preconditions.checkNotNull(newKind, "kind can not be null");
+    this.kind = newKind;
+  }
+
+  @Override
+  public boolean isNullAt(int pos) {
+    return struct.get(pos, Object.class) == null;
+  }
+
+  @Override
+  public boolean getBoolean(int pos) {
+    return struct.get(pos, Boolean.class);
+  }
+
+  @Override
+  public byte getByte(int pos) {
+    return (byte) (int) struct.get(pos, Integer.class);
+  }
+
+  @Override
+  public short getShort(int pos) {
+    return (short) (int) struct.get(pos, Integer.class);
+  }
+
+  @Override
+  public int getInt(int pos) {
+    Object integer = struct.get(pos, Object.class);
+
+    if (integer instanceof Integer) {
+      return (int) integer;
+    } else if (integer instanceof LocalDate) {
+      return (int) ((LocalDate) integer).toEpochDay();
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for int field. Type name: " + integer.getClass().getName());
+    }
+  }
+
+  @Override
+  public long getLong(int pos) {
+    Object longVal = struct.get(pos, Object.class);
+
+    if (longVal instanceof Long) {
+      return (long) longVal;
+    } else if (longVal instanceof OffsetDateTime) {
+      return Duration.between(Instant.EPOCH, (OffsetDateTime) longVal).toNanos() / 1000;
+    } else if (longVal instanceof LocalDate) {
+      return ((LocalDate) longVal).toEpochDay();
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for long field. Type name: " + longVal.getClass().getName());
+    }
+  }
+
+  @Override
+  public float getFloat(int pos) {
+    return struct.get(pos, Float.class);
+  }
+
+  @Override
+  public double getDouble(int pos) {
+    return struct.get(pos, Double.class);
+  }
+
+  @Override
+  public StringData getString(int pos) {
+    return isNullAt(pos) ? null : getStringDataInternal(pos);
+  }
+
+  private StringData getStringDataInternal(int pos) {
+    CharSequence seq = struct.get(pos, CharSequence.class);
+    return StringData.fromString(seq.toString());
+  }
+
+  @Override
+  public DecimalData getDecimal(int pos, int precision, int scale) {
+    return isNullAt(pos)
+        ? null
+        : DecimalData.fromBigDecimal(getDecimalInternal(pos), precision, scale);
+  }
+
+  private BigDecimal getDecimalInternal(int pos) {
+    return struct.get(pos, BigDecimal.class);
+  }
+
+  @Override
+  public TimestampData getTimestamp(int pos, int precision) {
+    long timeLong = getLong(pos);
+    if (precision == 6) {
+      int nanosOfMillisecond = (int) (timeLong % 1000);
+      return TimestampData.fromEpochMillis(timeLong / 1000, nanosOfMillisecond);
+    } else {
+      return TimestampData.fromEpochMillis(timeLong);
+    }
+  }
+
+  @Override
+  public <T> RawValueData<T> getRawValue(int pos) {
+    return RawValueData.fromBytes(getBinary(pos));
+  }
+
+  @Override
+  public byte[] getBinary(int pos) {
+    return isNullAt(pos) ? null : getBinaryInternal(pos);
+  }
+
+  private byte[] getBinaryInternal(int pos) {
+    Object bytes = struct.get(pos, Object.class);
+
+    // should only be either ByteBuffer or byte[]
+    if (bytes instanceof ByteBuffer) {
+      return ByteBuffers.toByteArray((ByteBuffer) bytes);
+    } else if (bytes instanceof byte[]) {
+      return (byte[]) bytes;
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for binary field. Type name: " + bytes.getClass().getName());
+    }
+  }
+
+  @Override
+  public ArrayData getArray(int pos) {
+    return isNullAt(pos) ? null : getArrayInternal(pos);
+  }
+
+  private ArrayData getArrayInternal(int pos) {
+    return collectionToArrayData(
+        type.fields().get(pos).type().asListType().elementType(),
+        struct.get(pos, Collection.class));
+  }
+
+  @Override
+  public MapData getMap(int pos) {
+    return isNullAt(pos) ? null : getMapInternal(pos);
+  }
+
+  private MapData getMapInternal(int pos) {
+    return mapToMapData(type.fields().get(pos).type().asMapType(), struct.get(pos, Map.class));
+  }
+
+  @Override
+  public RowData getRow(int pos, int numFields) {
+    return isNullAt(pos) ? null : getStructRowData(pos, numFields);
+  }
+
+  private StructRowData getStructRowData(int ordinal, int numFields) {
+    return new StructRowData(
+        type.fields().get(ordinal).type().asStructType(), struct.get(ordinal, StructLike.class));
+  }
+
+  private MapData mapToMapData(Types.MapType mapType, Map<?, ?> map) {
+    // make a defensive copy to ensure entries do not change
+    List<Map.Entry<?, ?>> entries = ImmutableList.copyOf(map.entrySet());
+
+    ArrayData keyArray =

Review Comment:
   why do we need the complex logic from converting map to two lists (key and value) and convert back to a map? what's the problem of simply returning `new GenericMapData(map)`? 



##########
flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMetaDataTable.java:
##########
@@ -0,0 +1,713 @@
+/*
+ * 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.source;
+
+import java.io.IOException;
+import java.time.Instant;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.avro.generic.GenericData;
+import org.apache.commons.collections.ListUtils;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.CoreOptions;
+import org.apache.flink.table.api.TableEnvironment;
+import org.apache.flink.types.Row;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileContent;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Files;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.MetadataTableUtils;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.data.FileHelpers;
+import org.apache.iceberg.data.GenericRecord;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.flink.FlinkCatalogTestBase;
+import org.apache.iceberg.flink.FlinkConfigOptions;
+import org.apache.iceberg.flink.TestHelpers;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.util.SnapshotUtil;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+public class TestFlinkMetaDataTable extends FlinkCatalogTestBase {
+
+  private static final String TABLE_NAME = "test_table";
+  private final FileFormat format = FileFormat.AVRO;
+  private static TemporaryFolder temp = new TemporaryFolder();
+
+  public TestFlinkMetaDataTable(String catalogName, Namespace baseNamespace) {
+    super(catalogName, baseNamespace);
+  }
+
+  @Override
+  protected TableEnvironment getTableEnv() {
+    Configuration configuration = super.getTableEnv().getConfig().getConfiguration();
+    configuration.set(CoreOptions.DEFAULT_PARALLELISM, 1);
+    configuration.set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE, true);
+    return super.getTableEnv();
+  }
+
+  @Before
+  public void before() {
+    super.before();
+    sql("CREATE DATABASE %s", flinkDatabase);
+    sql("USE CATALOG %s", catalogName);

Review Comment:
   shouldn't `USE CATALOG` happen before `CREATE DATABASE`?



##########
flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMetaDataTable.java:
##########
@@ -0,0 +1,713 @@
+/*
+ * 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.source;
+
+import java.io.IOException;
+import java.time.Instant;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.avro.generic.GenericData;
+import org.apache.commons.collections.ListUtils;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.CoreOptions;
+import org.apache.flink.table.api.TableEnvironment;
+import org.apache.flink.types.Row;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileContent;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Files;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.MetadataTableUtils;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.data.FileHelpers;
+import org.apache.iceberg.data.GenericRecord;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.flink.FlinkCatalogTestBase;
+import org.apache.iceberg.flink.FlinkConfigOptions;
+import org.apache.iceberg.flink.TestHelpers;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.util.SnapshotUtil;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+public class TestFlinkMetaDataTable extends FlinkCatalogTestBase {
+
+  private static final String TABLE_NAME = "test_table";
+  private final FileFormat format = FileFormat.AVRO;
+  private static TemporaryFolder temp = new TemporaryFolder();
+
+  public TestFlinkMetaDataTable(String catalogName, Namespace baseNamespace) {
+    super(catalogName, baseNamespace);
+  }
+
+  @Override
+  protected TableEnvironment getTableEnv() {
+    Configuration configuration = super.getTableEnv().getConfig().getConfiguration();
+    configuration.set(CoreOptions.DEFAULT_PARALLELISM, 1);
+    configuration.set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE, true);
+    return super.getTableEnv();
+  }
+
+  @Before
+  public void before() {
+    super.before();
+    sql("CREATE DATABASE %s", flinkDatabase);
+    sql("USE CATALOG %s", catalogName);
+    sql("USE %s", DATABASE);
+  }
+
+  @Override
+  @After
+  public void clean() {
+    sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, TABLE_NAME);
+    sql("DROP DATABASE IF EXISTS %s", flinkDatabase);
+    super.clean();
+  }
+
+  @Test
+  public void testSnapshots() {
+    sql(
+        "CREATE TABLE %s (id INT, data VARCHAR,d DOUBLE) WITH ('format-version'='2', 'write.format.default'='%s')",
+        TABLE_NAME, format.name());
+    sql(

Review Comment:
   `INSERT INTO`  can probably also be put into `before` method to reduce duplications. also I am wondering if we can have 2 insertions/snapshots (basically more than 1)



##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java:
##########
@@ -0,0 +1,340 @@
+/*
+ * 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.Duration;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.OffsetDateTime;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.function.BiConsumer;
+import java.util.function.Function;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericMapData;
+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.flink.types.RowKind;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+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.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.ByteBuffers;
+
+public class StructRowData implements RowData {
+  private final Types.StructType type;
+  private RowKind kind;
+  private StructLike struct;
+
+  public StructRowData(Types.StructType type) {
+    this(type, RowKind.INSERT);
+  }
+
+  public StructRowData(Types.StructType type, RowKind kind) {
+    this(type, null, kind);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct) {
+    this(type, struct, RowKind.INSERT);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct, RowKind kind) {
+    this.type = type;
+    this.struct = struct;
+    this.kind = kind;
+  }
+
+  public StructRowData setStruct(StructLike newStruct) {
+    this.struct = newStruct;
+    return this;
+  }
+
+  @Override
+  public int getArity() {
+    return struct.size();
+  }
+
+  @Override
+  public RowKind getRowKind() {
+    return kind;
+  }
+
+  @Override
+  public void setRowKind(RowKind newKind) {
+    Preconditions.checkNotNull(newKind, "kind can not be null");
+    this.kind = newKind;
+  }
+
+  @Override
+  public boolean isNullAt(int pos) {
+    return struct.get(pos, Object.class) == null;
+  }
+
+  @Override
+  public boolean getBoolean(int pos) {
+    return struct.get(pos, Boolean.class);
+  }
+
+  @Override
+  public byte getByte(int pos) {
+    return (byte) (int) struct.get(pos, Integer.class);
+  }
+
+  @Override
+  public short getShort(int pos) {
+    return (short) (int) struct.get(pos, Integer.class);
+  }
+
+  @Override
+  public int getInt(int pos) {
+    Object integer = struct.get(pos, Object.class);
+
+    if (integer instanceof Integer) {
+      return (int) integer;
+    } else if (integer instanceof LocalDate) {
+      return (int) ((LocalDate) integer).toEpochDay();
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for int field. Type name: " + integer.getClass().getName());
+    }
+  }
+
+  @Override
+  public long getLong(int pos) {
+    Object longVal = struct.get(pos, Object.class);
+
+    if (longVal instanceof Long) {
+      return (long) longVal;
+    } else if (longVal instanceof OffsetDateTime) {
+      return Duration.between(Instant.EPOCH, (OffsetDateTime) longVal).toNanos() / 1000;
+    } else if (longVal instanceof LocalDate) {
+      return ((LocalDate) longVal).toEpochDay();
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for long field. Type name: " + longVal.getClass().getName());
+    }
+  }
+
+  @Override
+  public float getFloat(int pos) {
+    return struct.get(pos, Float.class);
+  }
+
+  @Override
+  public double getDouble(int pos) {
+    return struct.get(pos, Double.class);
+  }
+
+  @Override
+  public StringData getString(int pos) {
+    return isNullAt(pos) ? null : getStringDataInternal(pos);
+  }
+
+  private StringData getStringDataInternal(int pos) {
+    CharSequence seq = struct.get(pos, CharSequence.class);
+    return StringData.fromString(seq.toString());
+  }
+
+  @Override
+  public DecimalData getDecimal(int pos, int precision, int scale) {
+    return isNullAt(pos)
+        ? null
+        : DecimalData.fromBigDecimal(getDecimalInternal(pos), precision, scale);
+  }
+
+  private BigDecimal getDecimalInternal(int pos) {
+    return struct.get(pos, BigDecimal.class);
+  }
+
+  @Override
+  public TimestampData getTimestamp(int pos, int precision) {
+    long timeLong = getLong(pos);
+    if (precision == 6) {
+      int nanosOfMillisecond = (int) (timeLong % 1000);
+      return TimestampData.fromEpochMillis(timeLong / 1000, nanosOfMillisecond);
+    } else {
+      return TimestampData.fromEpochMillis(timeLong);
+    }
+  }
+
+  @Override
+  public <T> RawValueData<T> getRawValue(int pos) {
+    return RawValueData.fromBytes(getBinary(pos));
+  }
+
+  @Override
+  public byte[] getBinary(int pos) {
+    return isNullAt(pos) ? null : getBinaryInternal(pos);
+  }
+
+  private byte[] getBinaryInternal(int pos) {
+    Object bytes = struct.get(pos, Object.class);
+
+    // should only be either ByteBuffer or byte[]
+    if (bytes instanceof ByteBuffer) {
+      return ByteBuffers.toByteArray((ByteBuffer) bytes);
+    } else if (bytes instanceof byte[]) {
+      return (byte[]) bytes;
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for binary field. Type name: " + bytes.getClass().getName());
+    }
+  }
+
+  @Override
+  public ArrayData getArray(int pos) {
+    return isNullAt(pos) ? null : getArrayInternal(pos);
+  }
+
+  private ArrayData getArrayInternal(int pos) {
+    return collectionToArrayData(
+        type.fields().get(pos).type().asListType().elementType(),
+        struct.get(pos, Collection.class));
+  }
+
+  @Override
+  public MapData getMap(int pos) {
+    return isNullAt(pos) ? null : getMapInternal(pos);
+  }
+
+  private MapData getMapInternal(int pos) {
+    return mapToMapData(type.fields().get(pos).type().asMapType(), struct.get(pos, Map.class));
+  }
+
+  @Override
+  public RowData getRow(int pos, int numFields) {
+    return isNullAt(pos) ? null : getStructRowData(pos, numFields);
+  }
+
+  private StructRowData getStructRowData(int ordinal, int numFields) {
+    return new StructRowData(
+        type.fields().get(ordinal).type().asStructType(), struct.get(ordinal, StructLike.class));
+  }
+
+  private MapData mapToMapData(Types.MapType mapType, Map<?, ?> map) {
+    // make a defensive copy to ensure entries do not change
+    List<Map.Entry<?, ?>> entries = ImmutableList.copyOf(map.entrySet());
+
+    ArrayData keyArray =
+        collectionToArrayData(mapType.keyType(), Lists.transform(entries, Map.Entry::getKey));
+
+    ArrayData valueArray =
+        collectionToArrayData(mapType.valueType(), Lists.transform(entries, Map.Entry::getValue));
+
+    ArrayData.ElementGetter keyGetter =
+        ArrayData.createElementGetter(FlinkSchemaUtil.convert(mapType.keyType()));
+    ArrayData.ElementGetter valueGetter =
+        ArrayData.createElementGetter(FlinkSchemaUtil.convert(mapType.valueType()));
+
+    int length = keyArray.size();
+    Map<Object, Object> result = Maps.newHashMap();
+    for (int pos = 0; pos < length; pos++) {
+      final Object keyValue = keyGetter.getElementOrNull(keyArray, pos);
+      final Object valueValue = valueGetter.getElementOrNull(valueArray, pos);
+
+      result.put(keyValue, valueValue);
+    }
+    return new GenericMapData(result);
+  }
+
+  private ArrayData collectionToArrayData(Type elementType, Collection<?> values) {
+    switch (elementType.typeId()) {
+      case BOOLEAN:
+      case INTEGER:
+      case DATE:
+      case TIME:
+      case LONG:
+      case TIMESTAMP:
+      case FLOAT:
+      case DOUBLE:
+        return fillArray(values, array -> (pos, value) -> array[pos] = value);
+      case STRING:
+        return fillArray(
+            values,
+            array ->
+                (BiConsumer<Integer, CharSequence>)
+                    (pos, seq) -> array[pos] = StringData.fromString(seq.toString()));
+      case FIXED:
+      case BINARY:
+        return fillArray(
+            values,
+            array ->
+                (BiConsumer<Integer, ByteBuffer>)
+                    (pos, buf) -> array[pos] = ByteBuffers.toByteArray(buf));
+      case DECIMAL:
+        return fillArray(
+            values, array -> (BiConsumer<Integer, BigDecimal>) (pos, dec) -> array[pos] = dec);
+      case STRUCT:
+        return fillArray(
+            values,
+            array ->
+                (BiConsumer<Integer, StructLike>)
+                    (pos, tuple) ->
+                        array[pos] = new StructRowData(elementType.asStructType(), tuple));
+      case LIST:
+        return fillArray(
+            values,
+            array ->
+                (BiConsumer<Integer, Collection<?>>)
+                    (pos, list) ->
+                        array[pos] =
+                            collectionToArrayData(elementType.asListType().elementType(), list));
+      case MAP:
+        return fillArray(
+            values,
+            array ->
+                (BiConsumer<Integer, Map<?, ?>>)
+                    (pos, map) -> array[pos] = new GenericMapData(map));
+      default:
+        throw new UnsupportedOperationException("Unsupported array element type: " + elementType);
+    }
+  }
+
+  @SuppressWarnings("unchecked")
+  private <T> GenericArrayData fillArray(
+      Collection<?> values, Function<Object[], BiConsumer<Integer, T>> makeSetter) {
+    Object[] array = new Object[values.size()];

Review Comment:
   is this correct? should the array corresponds to the element type of the Iceberg list? 
   
   e.g., FlinkOrcReaders
   ```
       @Override
       public ArrayData nonNullRead(ColumnVector vector, int row) {
         ListColumnVector listVector = (ListColumnVector) vector;
         int offset = (int) listVector.offsets[row];
         int length = (int) listVector.lengths[row];
         List<T> elements = Lists.newArrayListWithExpectedSize(length);
         for (int c = 0; c < length; ++c) {
           elements.add(elementReader.read(listVector.child, offset + c));
         }
         return new GenericArrayData(elements.toArray());
       }
   ```
   
   or TestRowProjection
   ```
     @Test
     public void testListProjection() throws IOException {
       Schema writeSchema =
           new Schema(
               Types.NestedField.required(0, "id", Types.LongType.get()),
               Types.NestedField.optional(
                   10, "values", Types.ListType.ofOptional(11, Types.LongType.get())));
   
       GenericArrayData values = new GenericArrayData(new Long[] {56L, 57L, 58L});
   ```



##########
flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMetaDataTable.java:
##########
@@ -0,0 +1,713 @@
+/*
+ * 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.source;
+
+import java.io.IOException;
+import java.time.Instant;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.avro.generic.GenericData;
+import org.apache.commons.collections.ListUtils;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.CoreOptions;
+import org.apache.flink.table.api.TableEnvironment;
+import org.apache.flink.types.Row;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileContent;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Files;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.MetadataTableUtils;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.data.FileHelpers;
+import org.apache.iceberg.data.GenericRecord;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.flink.FlinkCatalogTestBase;
+import org.apache.iceberg.flink.FlinkConfigOptions;
+import org.apache.iceberg.flink.TestHelpers;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.util.SnapshotUtil;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+public class TestFlinkMetaDataTable extends FlinkCatalogTestBase {

Review Comment:
   does this cover all the types of metadata tables? e.g. is `ENTRIES` supported/tested?



##########
flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMetaDataTable.java:
##########
@@ -0,0 +1,713 @@
+/*
+ * 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.source;
+
+import java.io.IOException;
+import java.time.Instant;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.avro.generic.GenericData;
+import org.apache.commons.collections.ListUtils;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.CoreOptions;
+import org.apache.flink.table.api.TableEnvironment;
+import org.apache.flink.types.Row;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileContent;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Files;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.MetadataTableUtils;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.data.FileHelpers;
+import org.apache.iceberg.data.GenericRecord;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.flink.FlinkCatalogTestBase;
+import org.apache.iceberg.flink.FlinkConfigOptions;
+import org.apache.iceberg.flink.TestHelpers;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.util.SnapshotUtil;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+public class TestFlinkMetaDataTable extends FlinkCatalogTestBase {
+
+  private static final String TABLE_NAME = "test_table";
+  private final FileFormat format = FileFormat.AVRO;
+  private static TemporaryFolder temp = new TemporaryFolder();
+
+  public TestFlinkMetaDataTable(String catalogName, Namespace baseNamespace) {
+    super(catalogName, baseNamespace);
+  }
+
+  @Override
+  protected TableEnvironment getTableEnv() {
+    Configuration configuration = super.getTableEnv().getConfig().getConfiguration();
+    configuration.set(CoreOptions.DEFAULT_PARALLELISM, 1);
+    configuration.set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE, true);
+    return super.getTableEnv();
+  }
+
+  @Before
+  public void before() {
+    super.before();
+    sql("CREATE DATABASE %s", flinkDatabase);
+    sql("USE CATALOG %s", catalogName);
+    sql("USE %s", DATABASE);
+  }
+
+  @Override
+  @After
+  public void clean() {
+    sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, TABLE_NAME);
+    sql("DROP DATABASE IF EXISTS %s", flinkDatabase);
+    super.clean();
+  }
+
+  @Test
+  public void testSnapshots() {
+    sql(

Review Comment:
   I understand why this is not done in the before method, because we want to be able to create partitioned and unpartitioned table in some methods. that also applies to the insert comment below. 
   
   I think for testing metadata tables, we don't need to test all the catalogs from `FlinkCatalogTestBase`. We just want to test one catalog. But we may want to use test parameters on other dimensions (`v1 and v2`, `partitioned and unpartitioned`) for all the supported metadata tables. Then we can move the `CREATE TABLE` and `INSERT INTO` into the `before` method.



##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/MetaDataReaderFunction.java:
##########
@@ -0,0 +1,62 @@
+/*
+ * 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.source.reader;
+
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.table.data.RowData;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.encryption.EncryptionManager;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.flink.source.DataIterator;
+import org.apache.iceberg.flink.source.DataTaskReader;
+import org.apache.iceberg.flink.source.split.IcebergSourceSplit;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class MetaDataReaderFunction extends DataIteratorReaderFunction<RowData> {

Review Comment:
   add maybe add some Javadoc that this for reading metadata tables (like snapshots, manifests, etc.)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] hililiwei commented on a diff in pull request #6222: Flink: Support inspecting table

Posted by GitBox <gi...@apache.org>.
hililiwei commented on code in PR #6222:
URL: https://github.com/apache/iceberg/pull/6222#discussion_r1037252208


##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java:
##########
@@ -0,0 +1,340 @@
+/*
+ * 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.Duration;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.OffsetDateTime;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.function.BiConsumer;
+import java.util.function.Function;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericMapData;
+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.flink.types.RowKind;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+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.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.ByteBuffers;
+
+public class StructRowData implements RowData {
+  private final Types.StructType type;
+  private RowKind kind;
+  private StructLike struct;
+
+  public StructRowData(Types.StructType type) {
+    this(type, RowKind.INSERT);
+  }
+
+  public StructRowData(Types.StructType type, RowKind kind) {
+    this(type, null, kind);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct) {

Review Comment:
   Yes. It used to handle the STRUCT/ROW type.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] stevenzwu commented on a diff in pull request #6222: Flink: Support inspecting table

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on code in PR #6222:
URL: https://github.com/apache/iceberg/pull/6222#discussion_r1036699624


##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java:
##########
@@ -0,0 +1,340 @@
+/*
+ * 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.Duration;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.OffsetDateTime;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.function.BiConsumer;
+import java.util.function.Function;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericMapData;
+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.flink.types.RowKind;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+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.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.ByteBuffers;
+
+public class StructRowData implements RowData {
+  private final Types.StructType type;
+  private RowKind kind;
+  private StructLike struct;
+
+  public StructRowData(Types.StructType type) {
+    this(type, RowKind.INSERT);
+  }
+
+  public StructRowData(Types.StructType type, RowKind kind) {
+    this(type, null, kind);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct) {
+    this(type, struct, RowKind.INSERT);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct, RowKind kind) {
+    this.type = type;
+    this.struct = struct;
+    this.kind = kind;
+  }
+
+  public StructRowData setStruct(StructLike newStruct) {
+    this.struct = newStruct;
+    return this;
+  }
+
+  @Override
+  public int getArity() {
+    return struct.size();
+  }
+
+  @Override
+  public RowKind getRowKind() {
+    return kind;
+  }
+
+  @Override
+  public void setRowKind(RowKind newKind) {
+    Preconditions.checkNotNull(newKind, "kind can not be null");
+    this.kind = newKind;
+  }
+
+  @Override
+  public boolean isNullAt(int pos) {
+    return struct.get(pos, Object.class) == null;
+  }
+
+  @Override
+  public boolean getBoolean(int pos) {
+    return struct.get(pos, Boolean.class);
+  }
+
+  @Override
+  public byte getByte(int pos) {
+    return (byte) (int) struct.get(pos, Integer.class);

Review Comment:
   why using `Integer` here? Java has `Byte` type. Does `StructLike` only support integer?



##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java:
##########
@@ -0,0 +1,340 @@
+/*
+ * 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.Duration;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.OffsetDateTime;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.function.BiConsumer;
+import java.util.function.Function;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericMapData;
+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.flink.types.RowKind;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+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.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.ByteBuffers;
+
+public class StructRowData implements RowData {
+  private final Types.StructType type;
+  private RowKind kind;
+  private StructLike struct;
+
+  public StructRowData(Types.StructType type) {
+    this(type, RowKind.INSERT);
+  }
+
+  public StructRowData(Types.StructType type, RowKind kind) {
+    this(type, null, kind);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct) {
+    this(type, struct, RowKind.INSERT);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct, RowKind kind) {
+    this.type = type;
+    this.struct = struct;
+    this.kind = kind;
+  }
+
+  public StructRowData setStruct(StructLike newStruct) {
+    this.struct = newStruct;
+    return this;
+  }
+
+  @Override
+  public int getArity() {
+    return struct.size();
+  }
+
+  @Override
+  public RowKind getRowKind() {
+    return kind;
+  }
+
+  @Override
+  public void setRowKind(RowKind newKind) {
+    Preconditions.checkNotNull(newKind, "kind can not be null");
+    this.kind = newKind;
+  }
+
+  @Override
+  public boolean isNullAt(int pos) {
+    return struct.get(pos, Object.class) == null;
+  }
+
+  @Override
+  public boolean getBoolean(int pos) {
+    return struct.get(pos, Boolean.class);
+  }
+
+  @Override
+  public byte getByte(int pos) {
+    return (byte) (int) struct.get(pos, Integer.class);
+  }
+
+  @Override
+  public short getShort(int pos) {
+    return (short) (int) struct.get(pos, Integer.class);

Review Comment:
   why using `Integer` here? Java has `Short` type. Does `StructLike` only support integer?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] hililiwei commented on a diff in pull request #6222: Flink: Support inspecting table

Posted by GitBox <gi...@apache.org>.
hililiwei commented on code in PR #6222:
URL: https://github.com/apache/iceberg/pull/6222#discussion_r1039789754


##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java:
##########
@@ -0,0 +1,340 @@
+/*
+ * 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.Duration;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.OffsetDateTime;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.function.BiConsumer;
+import java.util.function.Function;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericMapData;
+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.flink.types.RowKind;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+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.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.ByteBuffers;
+
+public class StructRowData implements RowData {
+  private final Types.StructType type;
+  private RowKind kind;
+  private StructLike struct;
+
+  public StructRowData(Types.StructType type) {
+    this(type, RowKind.INSERT);
+  }
+
+  public StructRowData(Types.StructType type, RowKind kind) {
+    this(type, null, kind);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct) {
+    this(type, struct, RowKind.INSERT);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct, RowKind kind) {
+    this.type = type;
+    this.struct = struct;
+    this.kind = kind;
+  }
+
+  public StructRowData setStruct(StructLike newStruct) {
+    this.struct = newStruct;
+    return this;
+  }
+
+  @Override
+  public int getArity() {
+    return struct.size();
+  }
+
+  @Override
+  public RowKind getRowKind() {
+    return kind;
+  }
+
+  @Override
+  public void setRowKind(RowKind newKind) {
+    Preconditions.checkNotNull(newKind, "kind can not be null");
+    this.kind = newKind;
+  }
+
+  @Override
+  public boolean isNullAt(int pos) {
+    return struct.get(pos, Object.class) == null;
+  }
+
+  @Override
+  public boolean getBoolean(int pos) {
+    return struct.get(pos, Boolean.class);
+  }
+
+  @Override
+  public byte getByte(int pos) {
+    return (byte) (int) struct.get(pos, Integer.class);
+  }
+
+  @Override
+  public short getShort(int pos) {
+    return (short) (int) struct.get(pos, Integer.class);
+  }
+
+  @Override
+  public int getInt(int pos) {
+    Object integer = struct.get(pos, Object.class);
+
+    if (integer instanceof Integer) {
+      return (int) integer;
+    } else if (integer instanceof LocalDate) {
+      return (int) ((LocalDate) integer).toEpochDay();
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for int field. Type name: " + integer.getClass().getName());
+    }
+  }
+
+  @Override
+  public long getLong(int pos) {
+    Object longVal = struct.get(pos, Object.class);
+
+    if (longVal instanceof Long) {
+      return (long) longVal;
+    } else if (longVal instanceof OffsetDateTime) {
+      return Duration.between(Instant.EPOCH, (OffsetDateTime) longVal).toNanos() / 1000;
+    } else if (longVal instanceof LocalDate) {
+      return ((LocalDate) longVal).toEpochDay();
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for long field. Type name: " + longVal.getClass().getName());
+    }
+  }
+
+  @Override
+  public float getFloat(int pos) {
+    return struct.get(pos, Float.class);
+  }
+
+  @Override
+  public double getDouble(int pos) {
+    return struct.get(pos, Double.class);
+  }
+
+  @Override
+  public StringData getString(int pos) {
+    return isNullAt(pos) ? null : getStringDataInternal(pos);
+  }
+
+  private StringData getStringDataInternal(int pos) {
+    CharSequence seq = struct.get(pos, CharSequence.class);
+    return StringData.fromString(seq.toString());
+  }
+
+  @Override
+  public DecimalData getDecimal(int pos, int precision, int scale) {
+    return isNullAt(pos)
+        ? null
+        : DecimalData.fromBigDecimal(getDecimalInternal(pos), precision, scale);
+  }
+
+  private BigDecimal getDecimalInternal(int pos) {
+    return struct.get(pos, BigDecimal.class);
+  }
+
+  @Override
+  public TimestampData getTimestamp(int pos, int precision) {
+    long timeLong = getLong(pos);
+    if (precision == 6) {
+      int nanosOfMillisecond = (int) (timeLong % 1000);
+      return TimestampData.fromEpochMillis(timeLong / 1000, nanosOfMillisecond);
+    } else {
+      return TimestampData.fromEpochMillis(timeLong);
+    }
+  }
+
+  @Override
+  public <T> RawValueData<T> getRawValue(int pos) {
+    return RawValueData.fromBytes(getBinary(pos));
+  }
+
+  @Override
+  public byte[] getBinary(int pos) {
+    return isNullAt(pos) ? null : getBinaryInternal(pos);
+  }
+
+  private byte[] getBinaryInternal(int pos) {
+    Object bytes = struct.get(pos, Object.class);
+
+    // should only be either ByteBuffer or byte[]
+    if (bytes instanceof ByteBuffer) {
+      return ByteBuffers.toByteArray((ByteBuffer) bytes);
+    } else if (bytes instanceof byte[]) {
+      return (byte[]) bytes;
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for binary field. Type name: " + bytes.getClass().getName());
+    }
+  }
+
+  @Override
+  public ArrayData getArray(int pos) {
+    return isNullAt(pos) ? null : getArrayInternal(pos);
+  }
+
+  private ArrayData getArrayInternal(int pos) {
+    return collectionToArrayData(
+        type.fields().get(pos).type().asListType().elementType(),
+        struct.get(pos, Collection.class));

Review Comment:
   Yeah, in practice, right now, it's just going to be List. Do we need to change it to List?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] stevenzwu commented on a diff in pull request #6222: Flink: Support inspecting table

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on code in PR #6222:
URL: https://github.com/apache/iceberg/pull/6222#discussion_r1042799420


##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java:
##########
@@ -0,0 +1,340 @@
+/*
+ * 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.Duration;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.OffsetDateTime;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.function.BiConsumer;
+import java.util.function.Function;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericMapData;
+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.flink.types.RowKind;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+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.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.ByteBuffers;
+
+public class StructRowData implements RowData {
+  private final Types.StructType type;
+  private RowKind kind;
+  private StructLike struct;
+
+  public StructRowData(Types.StructType type) {
+    this(type, RowKind.INSERT);
+  }
+
+  public StructRowData(Types.StructType type, RowKind kind) {
+    this(type, null, kind);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct) {
+    this(type, struct, RowKind.INSERT);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct, RowKind kind) {
+    this.type = type;
+    this.struct = struct;
+    this.kind = kind;
+  }
+
+  public StructRowData setStruct(StructLike newStruct) {
+    this.struct = newStruct;
+    return this;
+  }
+
+  @Override
+  public int getArity() {
+    return struct.size();
+  }
+
+  @Override
+  public RowKind getRowKind() {
+    return kind;
+  }
+
+  @Override
+  public void setRowKind(RowKind newKind) {
+    Preconditions.checkNotNull(newKind, "kind can not be null");
+    this.kind = newKind;
+  }
+
+  @Override
+  public boolean isNullAt(int pos) {
+    return struct.get(pos, Object.class) == null;
+  }
+
+  @Override
+  public boolean getBoolean(int pos) {
+    return struct.get(pos, Boolean.class);
+  }
+
+  @Override
+  public byte getByte(int pos) {
+    return (byte) (int) struct.get(pos, Integer.class);
+  }
+
+  @Override
+  public short getShort(int pos) {
+    return (short) (int) struct.get(pos, Integer.class);
+  }
+
+  @Override
+  public int getInt(int pos) {
+    Object integer = struct.get(pos, Object.class);
+
+    if (integer instanceof Integer) {
+      return (int) integer;
+    } else if (integer instanceof LocalDate) {
+      return (int) ((LocalDate) integer).toEpochDay();
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for int field. Type name: " + integer.getClass().getName());
+    }
+  }
+
+  @Override
+  public long getLong(int pos) {
+    Object longVal = struct.get(pos, Object.class);
+
+    if (longVal instanceof Long) {
+      return (long) longVal;
+    } else if (longVal instanceof OffsetDateTime) {
+      return Duration.between(Instant.EPOCH, (OffsetDateTime) longVal).toNanos() / 1000;
+    } else if (longVal instanceof LocalDate) {
+      return ((LocalDate) longVal).toEpochDay();
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for long field. Type name: " + longVal.getClass().getName());
+    }
+  }
+
+  @Override
+  public float getFloat(int pos) {
+    return struct.get(pos, Float.class);
+  }
+
+  @Override
+  public double getDouble(int pos) {
+    return struct.get(pos, Double.class);
+  }
+
+  @Override
+  public StringData getString(int pos) {
+    return isNullAt(pos) ? null : getStringDataInternal(pos);
+  }
+
+  private StringData getStringDataInternal(int pos) {
+    CharSequence seq = struct.get(pos, CharSequence.class);
+    return StringData.fromString(seq.toString());
+  }
+
+  @Override
+  public DecimalData getDecimal(int pos, int precision, int scale) {
+    return isNullAt(pos)
+        ? null
+        : DecimalData.fromBigDecimal(getDecimalInternal(pos), precision, scale);
+  }
+
+  private BigDecimal getDecimalInternal(int pos) {
+    return struct.get(pos, BigDecimal.class);
+  }
+
+  @Override
+  public TimestampData getTimestamp(int pos, int precision) {
+    long timeLong = getLong(pos);

Review Comment:
   > you mean , we should use `TimestampData.fromEpochMillis(micros / 1000, (int) (micros % 1000) * 1000);` ?
   
   if the underneath field is an Iceberg `Timestamp` type, I would say yes as it is always micro second precision.
   
   can the underneath field be an Iceberg `Long` type? is that valid? if it is a long with milli or nano, then we need additional logic to handle it.
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] szehon-ho commented on a diff in pull request #6222: Flink: Support inspecting table

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on code in PR #6222:
URL: https://github.com/apache/iceberg/pull/6222#discussion_r1042776458


##########
core/src/main/java/org/apache/iceberg/BaseFilesTable.java:
##########
@@ -223,34 +225,28 @@ ManifestFile manifest() {
   static class ContentFileStructWithMetrics implements StructLike {
     private final StructLike fileAsStruct;
     private final MetricsUtil.ReadableMetricsStruct readableMetrics;
-    private final int expectedSize;
+    private final int position;
 
     ContentFileStructWithMetrics(
-        int expectedSize,
-        StructLike fileAsStruct,
-        MetricsUtil.ReadableMetricsStruct readableMetrics) {
+        int position, StructLike fileAsStruct, MetricsUtil.ReadableMetricsStruct readableMetrics) {
       this.fileAsStruct = fileAsStruct;
       this.readableMetrics = readableMetrics;
-      this.expectedSize = expectedSize;
+      this.position = position;
     }
 
     @Override
     public int size() {
-      return expectedSize;
+      return position;

Review Comment:
   This looks a bit wrong though, should we have expectedSize (projection size) as another field than position (if you are saying the two are not always the same in the Flink case).



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] hililiwei commented on a diff in pull request #6222: Flink: Support inspecting table

Posted by GitBox <gi...@apache.org>.
hililiwei commented on code in PR #6222:
URL: https://github.com/apache/iceberg/pull/6222#discussion_r1051382070


##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java:
##########
@@ -0,0 +1,286 @@
+/*
+ * 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.Duration;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.OffsetDateTime;
+import java.util.List;
+import java.util.Map;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericMapData;
+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.flink.types.RowKind;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.ByteBuffers;
+
+@Internal
+public class StructRowData implements RowData {
+  private final Types.StructType type;
+  private RowKind kind;
+  private StructLike struct;
+
+  public StructRowData(Types.StructType type) {
+    this(type, RowKind.INSERT);
+  }
+
+  public StructRowData(Types.StructType type, RowKind kind) {
+    this(type, null, kind);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct) {
+    this(type, struct, RowKind.INSERT);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct, RowKind kind) {
+    this.type = type;
+    this.struct = struct;
+    this.kind = kind;
+  }
+
+  public StructRowData setStruct(StructLike newStruct) {
+    this.struct = newStruct;
+    return this;
+  }
+
+  @Override
+  public int getArity() {
+    return struct.size();
+  }
+
+  @Override
+  public RowKind getRowKind() {
+    return kind;
+  }
+
+  @Override
+  public void setRowKind(RowKind newKind) {
+    Preconditions.checkNotNull(newKind, "kind can not be null");
+    this.kind = newKind;
+  }
+
+  @Override
+  public boolean isNullAt(int pos) {
+    return struct.get(pos, Object.class) == null;
+  }
+
+  @Override
+  public boolean getBoolean(int pos) {
+    return struct.get(pos, Boolean.class);
+  }
+
+  @Override
+  public byte getByte(int pos) {
+    return (byte) (int) struct.get(pos, Integer.class);
+  }
+
+  @Override
+  public short getShort(int pos) {
+    return (short) (int) struct.get(pos, Integer.class);
+  }
+
+  @Override
+  public int getInt(int pos) {
+    Object integer = struct.get(pos, Object.class);
+
+    if (integer instanceof Integer) {
+      return (int) integer;
+    } else if (integer instanceof LocalDate) {
+      return (int) ((LocalDate) integer).toEpochDay();
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for int field. Type name: " + integer.getClass().getName());
+    }
+  }
+
+  @Override
+  public long getLong(int pos) {
+    Object longVal = struct.get(pos, Object.class);
+
+    if (longVal instanceof Long) {
+      return (long) longVal;
+    } else if (longVal instanceof OffsetDateTime) {
+      return Duration.between(Instant.EPOCH, (OffsetDateTime) longVal).toNanos() / 1000;
+    } else if (longVal instanceof LocalDate) {
+      return ((LocalDate) longVal).toEpochDay();
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for long field. Type name: " + longVal.getClass().getName());
+    }
+  }
+
+  @Override
+  public float getFloat(int pos) {
+    return struct.get(pos, Float.class);
+  }
+
+  @Override
+  public double getDouble(int pos) {
+    return struct.get(pos, Double.class);
+  }
+
+  @Override
+  public StringData getString(int pos) {
+    return isNullAt(pos) ? null : getStringDataInternal(pos);
+  }
+
+  private StringData getStringDataInternal(int pos) {
+    CharSequence seq = struct.get(pos, CharSequence.class);
+    return StringData.fromString(seq.toString());
+  }
+
+  @Override
+  public DecimalData getDecimal(int pos, int precision, int scale) {
+    return isNullAt(pos)
+        ? null
+        : DecimalData.fromBigDecimal(getDecimalInternal(pos), precision, scale);
+  }
+
+  private BigDecimal getDecimalInternal(int pos) {
+    return struct.get(pos, BigDecimal.class);
+  }
+
+  @Override
+  public TimestampData getTimestamp(int pos, int precision) {
+    long timeLong = getLong(pos);
+    return TimestampData.fromEpochMillis(timeLong / 1000, (int) (timeLong % 1000) * 1000);
+  }
+
+  @Override
+  public <T> RawValueData<T> getRawValue(int pos) {
+    throw new UnsupportedOperationException("Not supported yet.");
+  }
+
+  @Override
+  public byte[] getBinary(int pos) {
+    return isNullAt(pos) ? null : getBinaryInternal(pos);
+  }
+
+  private byte[] getBinaryInternal(int pos) {
+    Object bytes = struct.get(pos, Object.class);
+
+    // should only be either ByteBuffer or byte[]
+    if (bytes instanceof ByteBuffer) {
+      return ByteBuffers.toByteArray((ByteBuffer) bytes);
+    } else if (bytes instanceof byte[]) {
+      return (byte[]) bytes;
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for binary field. Type name: " + bytes.getClass().getName());
+    }
+  }
+
+  @Override
+  public ArrayData getArray(int pos) {
+    return isNullAt(pos)
+        ? null
+        : (ArrayData)
+            covertFromStructToRowData(
+                type.fields().get(pos).type().asListType(), struct.get(pos, List.class));
+  }
+
+  @Override
+  public MapData getMap(int pos) {
+    return isNullAt(pos)
+        ? null
+        : (MapData)
+            covertFromStructToRowData(
+                type.fields().get(pos).type().asMapType(), struct.get(pos, Map.class));
+  }
+
+  @Override
+  public RowData getRow(int pos, int numFields) {
+    return isNullAt(pos) ? null : getStructRowData(pos, numFields);
+  }
+
+  private StructRowData getStructRowData(int pos, int numFields) {
+    return new StructRowData(
+        type.fields().get(pos).type().asStructType(), struct.get(pos, StructLike.class));
+  }
+
+  private Object covertFromStructToRowData(Type elementType, Object value) {
+    switch (elementType.typeId()) {
+      case BOOLEAN:
+      case INTEGER:
+      case DATE:
+      case TIME:
+      case LONG:
+      case FLOAT:
+      case DOUBLE:
+      case DECIMAL:
+        return value;
+      case TIMESTAMP:
+        return TimestampData.fromEpochMillis(
+            (Long) value / 1000, (int) ((Long) value % 1000) * 1000);
+      case STRING:
+        return StringData.fromString(value.toString());
+      case FIXED:
+      case BINARY:
+        return ByteBuffers.toByteArray((ByteBuffer) value);
+      case STRUCT:
+        return new StructRowData(elementType.asStructType(), (StructLike) value);
+      case LIST:
+        List<?> list = (List<?>) value;
+        Object[] array = new Object[list.size()];
+
+        int index = 0;
+        for (Object element : list) {
+          if (element == null) {
+            array[index] = null;
+          } else {
+            array[index] =
+                covertFromStructToRowData(elementType.asListType().elementType(), element);
+          }
+
+          index += 1;
+        }
+        return new GenericArrayData(array);
+      case MAP:
+        Types.MapType mapType = elementType.asMapType();
+        // make a defensive copy to ensure entries do not change
+        List<Map.Entry<?, ?>> entries = ImmutableList.copyOf(((Map<?, ?>) value).entrySet());
+
+        Map<Object, Object> result = Maps.newHashMap();
+        for (Map.Entry<?, ?> entry : entries) {
+          final Object keyValue = covertFromStructToRowData(mapType.keyType(), entry.getKey());
+          final Object valueValue =
+              covertFromStructToRowData(mapType.valueType(), entry.getValue());
+          result.put(keyValue, valueValue);
+        }
+        return new GenericMapData(result);
+      default:
+        throw new UnsupportedOperationException("Unsupported array element type: " + elementType);

Review Comment:
   change go `"Unsupported element type: " + elementType` ?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] stevenzwu commented on a diff in pull request #6222: Flink: Support inspecting table

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on code in PR #6222:
URL: https://github.com/apache/iceberg/pull/6222#discussion_r1053536659


##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java:
##########
@@ -0,0 +1,300 @@
+/*
+ * 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.Duration;
+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.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericMapData;
+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.flink.types.RowKind;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.ByteBuffers;
+
+@Internal
+public class StructRowData implements RowData {
+  private final Types.StructType type;
+  private RowKind kind;
+  private StructLike struct;
+
+  public StructRowData(Types.StructType type) {
+    this(type, RowKind.INSERT);
+  }
+
+  public StructRowData(Types.StructType type, RowKind kind) {
+    this(type, null, kind);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct) {
+    this(type, struct, RowKind.INSERT);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct, RowKind kind) {
+    this.type = type;
+    this.struct = struct;
+    this.kind = kind;
+  }
+
+  public StructRowData setStruct(StructLike newStruct) {
+    this.struct = newStruct;
+    return this;
+  }
+
+  @Override
+  public int getArity() {
+    return struct.size();
+  }
+
+  @Override
+  public RowKind getRowKind() {
+    return kind;
+  }
+
+  @Override
+  public void setRowKind(RowKind newKind) {
+    Preconditions.checkNotNull(newKind, "kind can not be null");
+    this.kind = newKind;
+  }
+
+  @Override
+  public boolean isNullAt(int pos) {
+    return struct.get(pos, Object.class) == null;
+  }
+
+  @Override
+  public boolean getBoolean(int pos) {
+    return struct.get(pos, Boolean.class);
+  }
+
+  @Override
+  public byte getByte(int pos) {
+    return (byte) (int) struct.get(pos, Integer.class);
+  }
+
+  @Override
+  public short getShort(int pos) {
+    return (short) (int) struct.get(pos, Integer.class);
+  }
+
+  @Override
+  public int getInt(int pos) {
+    Object integer = struct.get(pos, Object.class);
+
+    if (integer instanceof Integer) {
+      return (int) integer;
+    } else if (integer instanceof LocalDate) {
+      return (int) ((LocalDate) integer).toEpochDay();
+    } else if (integer instanceof LocalTime) {
+      return (int) (((LocalTime) integer).toNanoOfDay() / 1000_000);
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for int field. Type name: " + integer.getClass().getName());
+    }
+  }
+
+  @Override
+  public long getLong(int pos) {
+    Object longVal = struct.get(pos, Object.class);
+
+    if (longVal instanceof Long) {
+      return (long) longVal;
+    } else if (longVal instanceof OffsetDateTime) {
+      return Duration.between(Instant.EPOCH, (OffsetDateTime) longVal).toNanos() / 1000;
+    } else if (longVal instanceof LocalDate) {
+      return ((LocalDate) longVal).toEpochDay();
+    } else if (longVal instanceof LocalTime) {
+      return ((LocalTime) longVal).toNanoOfDay();
+    } else if (longVal instanceof LocalDateTime) {
+      return Duration.between(Instant.EPOCH, ((LocalDateTime) longVal).atOffset(ZoneOffset.UTC))
+              .toNanos()
+          / 1000;
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for long field. Type name: " + longVal.getClass().getName());
+    }
+  }
+
+  @Override
+  public float getFloat(int pos) {
+    return struct.get(pos, Float.class);
+  }
+
+  @Override
+  public double getDouble(int pos) {
+    return struct.get(pos, Double.class);
+  }
+
+  @Override
+  public StringData getString(int pos) {
+    return isNullAt(pos) ? null : getStringDataInternal(pos);
+  }
+
+  private StringData getStringDataInternal(int pos) {
+    CharSequence seq = struct.get(pos, CharSequence.class);
+    return StringData.fromString(seq.toString());
+  }
+
+  @Override
+  public DecimalData getDecimal(int pos, int precision, int scale) {
+    return isNullAt(pos)
+        ? null
+        : DecimalData.fromBigDecimal(getDecimalInternal(pos), precision, scale);
+  }
+
+  private BigDecimal getDecimalInternal(int pos) {
+    return struct.get(pos, BigDecimal.class);
+  }
+
+  @Override
+  public TimestampData getTimestamp(int pos, int precision) {
+    long timeLong = getLong(pos);
+    return TimestampData.fromEpochMillis(timeLong / 1000, (int) (timeLong % 1000) * 1000);
+  }
+
+  @Override
+  public <T> RawValueData<T> getRawValue(int pos) {
+    throw new UnsupportedOperationException("Not supported yet.");
+  }
+
+  @Override
+  public byte[] getBinary(int pos) {
+    return isNullAt(pos) ? null : getBinaryInternal(pos);
+  }
+
+  private byte[] getBinaryInternal(int pos) {
+    Object bytes = struct.get(pos, Object.class);
+
+    // should only be either ByteBuffer or byte[]
+    if (bytes instanceof ByteBuffer) {
+      return ByteBuffers.toByteArray((ByteBuffer) bytes);
+    } else if (bytes instanceof byte[]) {
+      return (byte[]) bytes;
+    } else if (bytes instanceof UUID) {
+      UUID uuid = (UUID) bytes;
+      ByteBuffer bb = ByteBuffer.allocate(16);
+      bb.putLong(uuid.getMostSignificantBits());
+      bb.putLong(uuid.getLeastSignificantBits());
+      return bb.array();
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for binary field. Type name: " + bytes.getClass().getName());
+    }
+  }
+
+  @Override
+  public ArrayData getArray(int pos) {
+    return isNullAt(pos)
+        ? null
+        : (ArrayData)
+            coverValue(type.fields().get(pos).type().asListType(), struct.get(pos, List.class));
+  }
+
+  @Override
+  public MapData getMap(int pos) {
+    return isNullAt(pos)
+        ? null
+        : (MapData)
+            coverValue(type.fields().get(pos).type().asMapType(), struct.get(pos, Map.class));
+  }
+
+  @Override
+  public RowData getRow(int pos, int numFields) {
+    return isNullAt(pos) ? null : getStructRowData(pos, numFields);
+  }
+
+  private StructRowData getStructRowData(int pos, int numFields) {
+    return new StructRowData(
+        type.fields().get(pos).type().asStructType(), struct.get(pos, StructLike.class));
+  }
+
+  private <T> Object coverValue(Type elementType, Object value) {
+    switch (elementType.typeId()) {
+      case BOOLEAN:
+      case INTEGER:
+      case DATE:
+      case TIME:
+      case LONG:
+      case FLOAT:
+      case DOUBLE:
+      case DECIMAL:
+        return value;
+      case TIMESTAMP:
+        long millisecond = (long) value / 1000;
+        int nanoOfMillisecond = (int) ((Long) value % 1000) * 1000;
+        return TimestampData.fromEpochMillis(millisecond, nanoOfMillisecond);
+      case STRING:
+        return StringData.fromString(value.toString());
+      case FIXED:
+      case BINARY:
+        return ByteBuffers.toByteArray((ByteBuffer) value);
+      case STRUCT:
+        return new StructRowData(elementType.asStructType(), (StructLike) value);
+      case LIST:
+        List<Object> list = (List<Object>) value;
+        T[] array = (T[]) new Object[list.size()];

Review Comment:
   should this be `new T[...]`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] stevenzwu commented on a diff in pull request #6222: Flink: Support inspecting table

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on code in PR #6222:
URL: https://github.com/apache/iceberg/pull/6222#discussion_r1066140558


##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java:
##########
@@ -0,0 +1,286 @@
+/*
+ * 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.Duration;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.OffsetDateTime;
+import java.util.List;
+import java.util.Map;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericMapData;
+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.flink.types.RowKind;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.ByteBuffers;
+
+@Internal
+public class StructRowData implements RowData {
+  private final Types.StructType type;
+  private RowKind kind;
+  private StructLike struct;
+
+  public StructRowData(Types.StructType type) {
+    this(type, RowKind.INSERT);
+  }
+
+  public StructRowData(Types.StructType type, RowKind kind) {
+    this(type, null, kind);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct) {
+    this(type, struct, RowKind.INSERT);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct, RowKind kind) {
+    this.type = type;
+    this.struct = struct;
+    this.kind = kind;
+  }
+
+  public StructRowData setStruct(StructLike newStruct) {
+    this.struct = newStruct;
+    return this;
+  }
+
+  @Override
+  public int getArity() {
+    return struct.size();
+  }
+
+  @Override
+  public RowKind getRowKind() {
+    return kind;
+  }
+
+  @Override
+  public void setRowKind(RowKind newKind) {
+    Preconditions.checkNotNull(newKind, "kind can not be null");
+    this.kind = newKind;
+  }
+
+  @Override
+  public boolean isNullAt(int pos) {
+    return struct.get(pos, Object.class) == null;
+  }
+
+  @Override
+  public boolean getBoolean(int pos) {
+    return struct.get(pos, Boolean.class);
+  }
+
+  @Override
+  public byte getByte(int pos) {
+    return (byte) (int) struct.get(pos, Integer.class);
+  }
+
+  @Override
+  public short getShort(int pos) {
+    return (short) (int) struct.get(pos, Integer.class);
+  }
+
+  @Override
+  public int getInt(int pos) {
+    Object integer = struct.get(pos, Object.class);
+
+    if (integer instanceof Integer) {
+      return (int) integer;
+    } else if (integer instanceof LocalDate) {
+      return (int) ((LocalDate) integer).toEpochDay();
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for int field. Type name: " + integer.getClass().getName());
+    }
+  }
+
+  @Override
+  public long getLong(int pos) {
+    Object longVal = struct.get(pos, Object.class);
+
+    if (longVal instanceof Long) {
+      return (long) longVal;
+    } else if (longVal instanceof OffsetDateTime) {
+      return Duration.between(Instant.EPOCH, (OffsetDateTime) longVal).toNanos() / 1000;
+    } else if (longVal instanceof LocalDate) {
+      return ((LocalDate) longVal).toEpochDay();
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for long field. Type name: " + longVal.getClass().getName());
+    }
+  }
+
+  @Override
+  public float getFloat(int pos) {
+    return struct.get(pos, Float.class);
+  }
+
+  @Override
+  public double getDouble(int pos) {
+    return struct.get(pos, Double.class);
+  }
+
+  @Override
+  public StringData getString(int pos) {
+    return isNullAt(pos) ? null : getStringDataInternal(pos);
+  }
+
+  private StringData getStringDataInternal(int pos) {
+    CharSequence seq = struct.get(pos, CharSequence.class);
+    return StringData.fromString(seq.toString());
+  }
+
+  @Override
+  public DecimalData getDecimal(int pos, int precision, int scale) {
+    return isNullAt(pos)
+        ? null
+        : DecimalData.fromBigDecimal(getDecimalInternal(pos), precision, scale);
+  }
+
+  private BigDecimal getDecimalInternal(int pos) {
+    return struct.get(pos, BigDecimal.class);
+  }
+
+  @Override
+  public TimestampData getTimestamp(int pos, int precision) {
+    long timeLong = getLong(pos);
+    return TimestampData.fromEpochMillis(timeLong / 1000, (int) (timeLong % 1000) * 1000);
+  }
+
+  @Override
+  public <T> RawValueData<T> getRawValue(int pos) {
+    throw new UnsupportedOperationException("Not supported yet.");
+  }
+
+  @Override
+  public byte[] getBinary(int pos) {
+    return isNullAt(pos) ? null : getBinaryInternal(pos);
+  }
+
+  private byte[] getBinaryInternal(int pos) {
+    Object bytes = struct.get(pos, Object.class);
+
+    // should only be either ByteBuffer or byte[]
+    if (bytes instanceof ByteBuffer) {
+      return ByteBuffers.toByteArray((ByteBuffer) bytes);
+    } else if (bytes instanceof byte[]) {
+      return (byte[]) bytes;
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for binary field. Type name: " + bytes.getClass().getName());
+    }
+  }
+
+  @Override
+  public ArrayData getArray(int pos) {
+    return isNullAt(pos)
+        ? null
+        : (ArrayData)
+            covertFromStructToRowData(
+                type.fields().get(pos).type().asListType(), struct.get(pos, List.class));
+  }
+
+  @Override
+  public MapData getMap(int pos) {
+    return isNullAt(pos)
+        ? null
+        : (MapData)
+            covertFromStructToRowData(
+                type.fields().get(pos).type().asMapType(), struct.get(pos, Map.class));
+  }
+
+  @Override
+  public RowData getRow(int pos, int numFields) {
+    return isNullAt(pos) ? null : getStructRowData(pos, numFields);
+  }
+
+  private StructRowData getStructRowData(int pos, int numFields) {
+    return new StructRowData(
+        type.fields().get(pos).type().asStructType(), struct.get(pos, StructLike.class));
+  }
+
+  private Object covertFromStructToRowData(Type elementType, Object value) {
+    switch (elementType.typeId()) {
+      case BOOLEAN:
+      case INTEGER:
+      case DATE:
+      case TIME:
+      case LONG:
+      case FLOAT:
+      case DOUBLE:
+      case DECIMAL:
+        return value;
+      case TIMESTAMP:
+        return TimestampData.fromEpochMillis(
+            (Long) value / 1000, (int) ((Long) value % 1000) * 1000);
+      case STRING:
+        return StringData.fromString(value.toString());
+      case FIXED:
+      case BINARY:
+        return ByteBuffers.toByteArray((ByteBuffer) value);
+      case STRUCT:
+        return new StructRowData(elementType.asStructType(), (StructLike) value);
+      case LIST:
+        List<?> list = (List<?>) value;
+        Object[] array = new Object[list.size()];

Review Comment:
   one related question is for the primitive types like int. E.g., should the array be  `int[]` or `Integer[]`?
   
   `GenericArrayData` supports primitive array.
   ```
       public GenericArrayData(int[] primitiveArray) {
           this(primitiveArray, primitiveArray.length, true);
       }
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] stevenzwu commented on a diff in pull request #6222: Flink: Support inspecting table

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on code in PR #6222:
URL: https://github.com/apache/iceberg/pull/6222#discussion_r1040359570


##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java:
##########
@@ -0,0 +1,340 @@
+/*
+ * 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.Duration;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.OffsetDateTime;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.function.BiConsumer;
+import java.util.function.Function;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericMapData;
+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.flink.types.RowKind;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+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.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.ByteBuffers;
+
+public class StructRowData implements RowData {
+  private final Types.StructType type;
+  private RowKind kind;
+  private StructLike struct;
+
+  public StructRowData(Types.StructType type) {
+    this(type, RowKind.INSERT);
+  }
+
+  public StructRowData(Types.StructType type, RowKind kind) {
+    this(type, null, kind);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct) {
+    this(type, struct, RowKind.INSERT);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct, RowKind kind) {
+    this.type = type;
+    this.struct = struct;
+    this.kind = kind;
+  }
+
+  public StructRowData setStruct(StructLike newStruct) {
+    this.struct = newStruct;
+    return this;
+  }
+
+  @Override
+  public int getArity() {
+    return struct.size();
+  }
+
+  @Override
+  public RowKind getRowKind() {
+    return kind;
+  }
+
+  @Override
+  public void setRowKind(RowKind newKind) {
+    Preconditions.checkNotNull(newKind, "kind can not be null");
+    this.kind = newKind;
+  }
+
+  @Override
+  public boolean isNullAt(int pos) {
+    return struct.get(pos, Object.class) == null;
+  }
+
+  @Override
+  public boolean getBoolean(int pos) {
+    return struct.get(pos, Boolean.class);
+  }
+
+  @Override
+  public byte getByte(int pos) {
+    return (byte) (int) struct.get(pos, Integer.class);
+  }
+
+  @Override
+  public short getShort(int pos) {
+    return (short) (int) struct.get(pos, Integer.class);
+  }
+
+  @Override
+  public int getInt(int pos) {
+    Object integer = struct.get(pos, Object.class);
+
+    if (integer instanceof Integer) {
+      return (int) integer;
+    } else if (integer instanceof LocalDate) {
+      return (int) ((LocalDate) integer).toEpochDay();
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for int field. Type name: " + integer.getClass().getName());
+    }
+  }
+
+  @Override
+  public long getLong(int pos) {
+    Object longVal = struct.get(pos, Object.class);
+
+    if (longVal instanceof Long) {
+      return (long) longVal;
+    } else if (longVal instanceof OffsetDateTime) {
+      return Duration.between(Instant.EPOCH, (OffsetDateTime) longVal).toNanos() / 1000;
+    } else if (longVal instanceof LocalDate) {
+      return ((LocalDate) longVal).toEpochDay();
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for long field. Type name: " + longVal.getClass().getName());
+    }
+  }
+
+  @Override
+  public float getFloat(int pos) {
+    return struct.get(pos, Float.class);
+  }
+
+  @Override
+  public double getDouble(int pos) {
+    return struct.get(pos, Double.class);
+  }
+
+  @Override
+  public StringData getString(int pos) {
+    return isNullAt(pos) ? null : getStringDataInternal(pos);
+  }
+
+  private StringData getStringDataInternal(int pos) {
+    CharSequence seq = struct.get(pos, CharSequence.class);
+    return StringData.fromString(seq.toString());
+  }
+
+  @Override
+  public DecimalData getDecimal(int pos, int precision, int scale) {
+    return isNullAt(pos)
+        ? null
+        : DecimalData.fromBigDecimal(getDecimalInternal(pos), precision, scale);
+  }
+
+  private BigDecimal getDecimalInternal(int pos) {
+    return struct.get(pos, BigDecimal.class);
+  }
+
+  @Override
+  public TimestampData getTimestamp(int pos, int precision) {
+    long timeLong = getLong(pos);

Review Comment:
   it is `millis * 1000`. so it should be `micro` seconds, which is consistent with Iceberg timestamp type.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] hililiwei commented on a diff in pull request #6222: Flink: Support inspecting table

Posted by GitBox <gi...@apache.org>.
hililiwei commented on code in PR #6222:
URL: https://github.com/apache/iceberg/pull/6222#discussion_r1040596892


##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java:
##########
@@ -0,0 +1,340 @@
+/*
+ * 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.Duration;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.OffsetDateTime;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.function.BiConsumer;
+import java.util.function.Function;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericMapData;
+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.flink.types.RowKind;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+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.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.ByteBuffers;
+
+public class StructRowData implements RowData {
+  private final Types.StructType type;
+  private RowKind kind;
+  private StructLike struct;
+
+  public StructRowData(Types.StructType type) {
+    this(type, RowKind.INSERT);
+  }
+
+  public StructRowData(Types.StructType type, RowKind kind) {
+    this(type, null, kind);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct) {
+    this(type, struct, RowKind.INSERT);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct, RowKind kind) {
+    this.type = type;
+    this.struct = struct;
+    this.kind = kind;
+  }
+
+  public StructRowData setStruct(StructLike newStruct) {
+    this.struct = newStruct;
+    return this;
+  }
+
+  @Override
+  public int getArity() {
+    return struct.size();
+  }
+
+  @Override
+  public RowKind getRowKind() {
+    return kind;
+  }
+
+  @Override
+  public void setRowKind(RowKind newKind) {
+    Preconditions.checkNotNull(newKind, "kind can not be null");
+    this.kind = newKind;
+  }
+
+  @Override
+  public boolean isNullAt(int pos) {
+    return struct.get(pos, Object.class) == null;
+  }
+
+  @Override
+  public boolean getBoolean(int pos) {
+    return struct.get(pos, Boolean.class);
+  }
+
+  @Override
+  public byte getByte(int pos) {
+    return (byte) (int) struct.get(pos, Integer.class);
+  }
+
+  @Override
+  public short getShort(int pos) {
+    return (short) (int) struct.get(pos, Integer.class);
+  }
+
+  @Override
+  public int getInt(int pos) {
+    Object integer = struct.get(pos, Object.class);
+
+    if (integer instanceof Integer) {
+      return (int) integer;
+    } else if (integer instanceof LocalDate) {
+      return (int) ((LocalDate) integer).toEpochDay();
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for int field. Type name: " + integer.getClass().getName());
+    }
+  }
+
+  @Override
+  public long getLong(int pos) {
+    Object longVal = struct.get(pos, Object.class);
+
+    if (longVal instanceof Long) {
+      return (long) longVal;
+    } else if (longVal instanceof OffsetDateTime) {
+      return Duration.between(Instant.EPOCH, (OffsetDateTime) longVal).toNanos() / 1000;
+    } else if (longVal instanceof LocalDate) {
+      return ((LocalDate) longVal).toEpochDay();
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for long field. Type name: " + longVal.getClass().getName());
+    }
+  }
+
+  @Override
+  public float getFloat(int pos) {
+    return struct.get(pos, Float.class);
+  }
+
+  @Override
+  public double getDouble(int pos) {
+    return struct.get(pos, Double.class);
+  }
+
+  @Override
+  public StringData getString(int pos) {
+    return isNullAt(pos) ? null : getStringDataInternal(pos);
+  }
+
+  private StringData getStringDataInternal(int pos) {
+    CharSequence seq = struct.get(pos, CharSequence.class);
+    return StringData.fromString(seq.toString());
+  }
+
+  @Override
+  public DecimalData getDecimal(int pos, int precision, int scale) {
+    return isNullAt(pos)
+        ? null
+        : DecimalData.fromBigDecimal(getDecimalInternal(pos), precision, scale);
+  }
+
+  private BigDecimal getDecimalInternal(int pos) {
+    return struct.get(pos, BigDecimal.class);
+  }
+
+  @Override
+  public TimestampData getTimestamp(int pos, int precision) {
+    long timeLong = getLong(pos);
+    if (precision == 6) {
+      int nanosOfMillisecond = (int) (timeLong % 1000);
+      return TimestampData.fromEpochMillis(timeLong / 1000, nanosOfMillisecond);
+    } else {
+      return TimestampData.fromEpochMillis(timeLong);
+    }
+  }
+
+  @Override
+  public <T> RawValueData<T> getRawValue(int pos) {
+    return RawValueData.fromBytes(getBinary(pos));
+  }
+
+  @Override
+  public byte[] getBinary(int pos) {
+    return isNullAt(pos) ? null : getBinaryInternal(pos);
+  }
+
+  private byte[] getBinaryInternal(int pos) {
+    Object bytes = struct.get(pos, Object.class);
+
+    // should only be either ByteBuffer or byte[]
+    if (bytes instanceof ByteBuffer) {
+      return ByteBuffers.toByteArray((ByteBuffer) bytes);
+    } else if (bytes instanceof byte[]) {
+      return (byte[]) bytes;
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for binary field. Type name: " + bytes.getClass().getName());
+    }
+  }
+
+  @Override
+  public ArrayData getArray(int pos) {
+    return isNullAt(pos) ? null : getArrayInternal(pos);
+  }
+
+  private ArrayData getArrayInternal(int pos) {
+    return collectionToArrayData(
+        type.fields().get(pos).type().asListType().elementType(),
+        struct.get(pos, Collection.class));
+  }
+
+  @Override
+  public MapData getMap(int pos) {
+    return isNullAt(pos) ? null : getMapInternal(pos);
+  }
+
+  private MapData getMapInternal(int pos) {
+    return mapToMapData(type.fields().get(pos).type().asMapType(), struct.get(pos, Map.class));
+  }
+
+  @Override
+  public RowData getRow(int pos, int numFields) {
+    return isNullAt(pos) ? null : getStructRowData(pos, numFields);
+  }
+
+  private StructRowData getStructRowData(int ordinal, int numFields) {
+    return new StructRowData(
+        type.fields().get(ordinal).type().asStructType(), struct.get(ordinal, StructLike.class));
+  }
+
+  private MapData mapToMapData(Types.MapType mapType, Map<?, ?> map) {
+    // make a defensive copy to ensure entries do not change
+    List<Map.Entry<?, ?>> entries = ImmutableList.copyOf(map.entrySet());
+
+    ArrayData keyArray =

Review Comment:
   Done, please take a look. Thx.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] stevenzwu commented on a diff in pull request #6222: Flink: Support inspecting table

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on code in PR #6222:
URL: https://github.com/apache/iceberg/pull/6222#discussion_r1036699624


##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java:
##########
@@ -0,0 +1,340 @@
+/*
+ * 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.Duration;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.OffsetDateTime;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.function.BiConsumer;
+import java.util.function.Function;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericMapData;
+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.flink.types.RowKind;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+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.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.ByteBuffers;
+
+public class StructRowData implements RowData {
+  private final Types.StructType type;
+  private RowKind kind;
+  private StructLike struct;
+
+  public StructRowData(Types.StructType type) {
+    this(type, RowKind.INSERT);
+  }
+
+  public StructRowData(Types.StructType type, RowKind kind) {
+    this(type, null, kind);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct) {
+    this(type, struct, RowKind.INSERT);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct, RowKind kind) {
+    this.type = type;
+    this.struct = struct;
+    this.kind = kind;
+  }
+
+  public StructRowData setStruct(StructLike newStruct) {
+    this.struct = newStruct;
+    return this;
+  }
+
+  @Override
+  public int getArity() {
+    return struct.size();
+  }
+
+  @Override
+  public RowKind getRowKind() {
+    return kind;
+  }
+
+  @Override
+  public void setRowKind(RowKind newKind) {
+    Preconditions.checkNotNull(newKind, "kind can not be null");
+    this.kind = newKind;
+  }
+
+  @Override
+  public boolean isNullAt(int pos) {
+    return struct.get(pos, Object.class) == null;
+  }
+
+  @Override
+  public boolean getBoolean(int pos) {
+    return struct.get(pos, Boolean.class);
+  }
+
+  @Override
+  public byte getByte(int pos) {
+    return (byte) (int) struct.get(pos, Integer.class);

Review Comment:
   why using `Integer` here? Java has `Byte` type.



##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java:
##########
@@ -0,0 +1,340 @@
+/*
+ * 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.Duration;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.OffsetDateTime;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.function.BiConsumer;
+import java.util.function.Function;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericMapData;
+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.flink.types.RowKind;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+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.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.ByteBuffers;
+
+public class StructRowData implements RowData {
+  private final Types.StructType type;
+  private RowKind kind;
+  private StructLike struct;
+
+  public StructRowData(Types.StructType type) {
+    this(type, RowKind.INSERT);
+  }
+
+  public StructRowData(Types.StructType type, RowKind kind) {
+    this(type, null, kind);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct) {
+    this(type, struct, RowKind.INSERT);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct, RowKind kind) {
+    this.type = type;
+    this.struct = struct;
+    this.kind = kind;
+  }
+
+  public StructRowData setStruct(StructLike newStruct) {
+    this.struct = newStruct;
+    return this;
+  }
+
+  @Override
+  public int getArity() {
+    return struct.size();
+  }
+
+  @Override
+  public RowKind getRowKind() {
+    return kind;
+  }
+
+  @Override
+  public void setRowKind(RowKind newKind) {
+    Preconditions.checkNotNull(newKind, "kind can not be null");
+    this.kind = newKind;
+  }
+
+  @Override
+  public boolean isNullAt(int pos) {
+    return struct.get(pos, Object.class) == null;
+  }
+
+  @Override
+  public boolean getBoolean(int pos) {
+    return struct.get(pos, Boolean.class);
+  }
+
+  @Override
+  public byte getByte(int pos) {
+    return (byte) (int) struct.get(pos, Integer.class);
+  }
+
+  @Override
+  public short getShort(int pos) {
+    return (short) (int) struct.get(pos, Integer.class);

Review Comment:
   why using `Integer` here? Java has `Short` type.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] hililiwei commented on a diff in pull request #6222: Flink: Support inspecting table

Posted by GitBox <gi...@apache.org>.
hililiwei commented on code in PR #6222:
URL: https://github.com/apache/iceberg/pull/6222#discussion_r1037191643


##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java:
##########
@@ -0,0 +1,340 @@
+/*
+ * 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.Duration;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.OffsetDateTime;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.function.BiConsumer;
+import java.util.function.Function;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericMapData;
+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.flink.types.RowKind;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+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.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.ByteBuffers;
+
+public class StructRowData implements RowData {
+  private final Types.StructType type;
+  private RowKind kind;
+  private StructLike struct;
+
+  public StructRowData(Types.StructType type) {
+    this(type, RowKind.INSERT);
+  }
+
+  public StructRowData(Types.StructType type, RowKind kind) {
+    this(type, null, kind);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct) {
+    this(type, struct, RowKind.INSERT);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct, RowKind kind) {
+    this.type = type;
+    this.struct = struct;
+    this.kind = kind;
+  }
+
+  public StructRowData setStruct(StructLike newStruct) {
+    this.struct = newStruct;
+    return this;
+  }
+
+  @Override
+  public int getArity() {
+    return struct.size();
+  }
+
+  @Override
+  public RowKind getRowKind() {
+    return kind;
+  }
+
+  @Override
+  public void setRowKind(RowKind newKind) {
+    Preconditions.checkNotNull(newKind, "kind can not be null");
+    this.kind = newKind;
+  }
+
+  @Override
+  public boolean isNullAt(int pos) {
+    return struct.get(pos, Object.class) == null;
+  }
+
+  @Override
+  public boolean getBoolean(int pos) {
+    return struct.get(pos, Boolean.class);
+  }
+
+  @Override
+  public byte getByte(int pos) {
+    return (byte) (int) struct.get(pos, Integer.class);

Review Comment:
   All types here are derived from 
   https://github.com/apache/iceberg/blob/49a0ea956e3a4b9979754c887d803d4ab51131ae/api/src/main/java/org/apache/iceberg/types/Types.java#L40-L54
   
   It doesn't has `byte` and `short`
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] stevenzwu commented on a diff in pull request #6222: Flink: Support inspecting table

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on code in PR #6222:
URL: https://github.com/apache/iceberg/pull/6222#discussion_r1042787664


##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java:
##########
@@ -104,11 +105,38 @@ public static Schema convert(Schema baseSchema, TableSchema flinkSchema) {
     Types.StructType struct = convert(flinkSchema).asStruct();
     // reassign ids to match the base schema
     Schema schema = TypeUtil.reassignIds(new Schema(struct.fields()), baseSchema);
+    // reassign doc to match the base schema
+    schema = reassignDoc(schema, baseSchema);
+
     // fix types that can't be represented in Flink (UUID)
     Schema fixedSchema = FlinkFixupTypes.fixup(schema, baseSchema);
     return freshIdentifierFieldIds(fixedSchema, flinkSchema);
   }
 
+  private static Schema reassignDoc(Schema schema, Schema docSourceSchema) {
+    TypeUtil.CustomOrderSchemaVisitor<Type> visitor = new FlinkFixupDoc(docSourceSchema);

Review Comment:
   `FlinkFixupDoc` doesn't seem like the right name. it is not fixing up sth (like `FlinkFixupTypes` does). Maybe call it `ReassignDoc` or `CopyDoc`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] hililiwei commented on a diff in pull request #6222: Flink: Support inspecting table

Posted by GitBox <gi...@apache.org>.
hililiwei commented on code in PR #6222:
URL: https://github.com/apache/iceberg/pull/6222#discussion_r1051382070


##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java:
##########
@@ -0,0 +1,286 @@
+/*
+ * 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.Duration;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.OffsetDateTime;
+import java.util.List;
+import java.util.Map;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericMapData;
+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.flink.types.RowKind;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.ByteBuffers;
+
+@Internal
+public class StructRowData implements RowData {
+  private final Types.StructType type;
+  private RowKind kind;
+  private StructLike struct;
+
+  public StructRowData(Types.StructType type) {
+    this(type, RowKind.INSERT);
+  }
+
+  public StructRowData(Types.StructType type, RowKind kind) {
+    this(type, null, kind);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct) {
+    this(type, struct, RowKind.INSERT);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct, RowKind kind) {
+    this.type = type;
+    this.struct = struct;
+    this.kind = kind;
+  }
+
+  public StructRowData setStruct(StructLike newStruct) {
+    this.struct = newStruct;
+    return this;
+  }
+
+  @Override
+  public int getArity() {
+    return struct.size();
+  }
+
+  @Override
+  public RowKind getRowKind() {
+    return kind;
+  }
+
+  @Override
+  public void setRowKind(RowKind newKind) {
+    Preconditions.checkNotNull(newKind, "kind can not be null");
+    this.kind = newKind;
+  }
+
+  @Override
+  public boolean isNullAt(int pos) {
+    return struct.get(pos, Object.class) == null;
+  }
+
+  @Override
+  public boolean getBoolean(int pos) {
+    return struct.get(pos, Boolean.class);
+  }
+
+  @Override
+  public byte getByte(int pos) {
+    return (byte) (int) struct.get(pos, Integer.class);
+  }
+
+  @Override
+  public short getShort(int pos) {
+    return (short) (int) struct.get(pos, Integer.class);
+  }
+
+  @Override
+  public int getInt(int pos) {
+    Object integer = struct.get(pos, Object.class);
+
+    if (integer instanceof Integer) {
+      return (int) integer;
+    } else if (integer instanceof LocalDate) {
+      return (int) ((LocalDate) integer).toEpochDay();
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for int field. Type name: " + integer.getClass().getName());
+    }
+  }
+
+  @Override
+  public long getLong(int pos) {
+    Object longVal = struct.get(pos, Object.class);
+
+    if (longVal instanceof Long) {
+      return (long) longVal;
+    } else if (longVal instanceof OffsetDateTime) {
+      return Duration.between(Instant.EPOCH, (OffsetDateTime) longVal).toNanos() / 1000;
+    } else if (longVal instanceof LocalDate) {
+      return ((LocalDate) longVal).toEpochDay();
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for long field. Type name: " + longVal.getClass().getName());
+    }
+  }
+
+  @Override
+  public float getFloat(int pos) {
+    return struct.get(pos, Float.class);
+  }
+
+  @Override
+  public double getDouble(int pos) {
+    return struct.get(pos, Double.class);
+  }
+
+  @Override
+  public StringData getString(int pos) {
+    return isNullAt(pos) ? null : getStringDataInternal(pos);
+  }
+
+  private StringData getStringDataInternal(int pos) {
+    CharSequence seq = struct.get(pos, CharSequence.class);
+    return StringData.fromString(seq.toString());
+  }
+
+  @Override
+  public DecimalData getDecimal(int pos, int precision, int scale) {
+    return isNullAt(pos)
+        ? null
+        : DecimalData.fromBigDecimal(getDecimalInternal(pos), precision, scale);
+  }
+
+  private BigDecimal getDecimalInternal(int pos) {
+    return struct.get(pos, BigDecimal.class);
+  }
+
+  @Override
+  public TimestampData getTimestamp(int pos, int precision) {
+    long timeLong = getLong(pos);
+    return TimestampData.fromEpochMillis(timeLong / 1000, (int) (timeLong % 1000) * 1000);
+  }
+
+  @Override
+  public <T> RawValueData<T> getRawValue(int pos) {
+    throw new UnsupportedOperationException("Not supported yet.");
+  }
+
+  @Override
+  public byte[] getBinary(int pos) {
+    return isNullAt(pos) ? null : getBinaryInternal(pos);
+  }
+
+  private byte[] getBinaryInternal(int pos) {
+    Object bytes = struct.get(pos, Object.class);
+
+    // should only be either ByteBuffer or byte[]
+    if (bytes instanceof ByteBuffer) {
+      return ByteBuffers.toByteArray((ByteBuffer) bytes);
+    } else if (bytes instanceof byte[]) {
+      return (byte[]) bytes;
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for binary field. Type name: " + bytes.getClass().getName());
+    }
+  }
+
+  @Override
+  public ArrayData getArray(int pos) {
+    return isNullAt(pos)
+        ? null
+        : (ArrayData)
+            covertFromStructToRowData(
+                type.fields().get(pos).type().asListType(), struct.get(pos, List.class));
+  }
+
+  @Override
+  public MapData getMap(int pos) {
+    return isNullAt(pos)
+        ? null
+        : (MapData)
+            covertFromStructToRowData(
+                type.fields().get(pos).type().asMapType(), struct.get(pos, Map.class));
+  }
+
+  @Override
+  public RowData getRow(int pos, int numFields) {
+    return isNullAt(pos) ? null : getStructRowData(pos, numFields);
+  }
+
+  private StructRowData getStructRowData(int pos, int numFields) {
+    return new StructRowData(
+        type.fields().get(pos).type().asStructType(), struct.get(pos, StructLike.class));
+  }
+
+  private Object covertFromStructToRowData(Type elementType, Object value) {
+    switch (elementType.typeId()) {
+      case BOOLEAN:
+      case INTEGER:
+      case DATE:
+      case TIME:
+      case LONG:
+      case FLOAT:
+      case DOUBLE:
+      case DECIMAL:
+        return value;
+      case TIMESTAMP:
+        return TimestampData.fromEpochMillis(
+            (Long) value / 1000, (int) ((Long) value % 1000) * 1000);
+      case STRING:
+        return StringData.fromString(value.toString());
+      case FIXED:
+      case BINARY:
+        return ByteBuffers.toByteArray((ByteBuffer) value);
+      case STRUCT:
+        return new StructRowData(elementType.asStructType(), (StructLike) value);
+      case LIST:
+        List<?> list = (List<?>) value;
+        Object[] array = new Object[list.size()];
+
+        int index = 0;
+        for (Object element : list) {
+          if (element == null) {
+            array[index] = null;
+          } else {
+            array[index] =
+                covertFromStructToRowData(elementType.asListType().elementType(), element);
+          }
+
+          index += 1;
+        }
+        return new GenericArrayData(array);
+      case MAP:
+        Types.MapType mapType = elementType.asMapType();
+        // make a defensive copy to ensure entries do not change
+        List<Map.Entry<?, ?>> entries = ImmutableList.copyOf(((Map<?, ?>) value).entrySet());
+
+        Map<Object, Object> result = Maps.newHashMap();
+        for (Map.Entry<?, ?> entry : entries) {
+          final Object keyValue = covertFromStructToRowData(mapType.keyType(), entry.getKey());
+          final Object valueValue =
+              covertFromStructToRowData(mapType.valueType(), entry.getValue());
+          result.put(keyValue, valueValue);
+        }
+        return new GenericMapData(result);
+      default:
+        throw new UnsupportedOperationException("Unsupported array element type: " + elementType);

Review Comment:
   "Unsupported element type: " + elementType



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] hililiwei commented on a diff in pull request #6222: Flink: Support inspecting table

Posted by GitBox <gi...@apache.org>.
hililiwei commented on code in PR #6222:
URL: https://github.com/apache/iceberg/pull/6222#discussion_r1051380948


##########
core/src/main/java/org/apache/iceberg/BaseFilesTable.java:
##########
@@ -223,34 +225,28 @@ ManifestFile manifest() {
   static class ContentFileStructWithMetrics implements StructLike {
     private final StructLike fileAsStruct;
     private final MetricsUtil.ReadableMetricsStruct readableMetrics;
-    private final int expectedSize;
+    private final int position;
 
     ContentFileStructWithMetrics(
-        int expectedSize,
-        StructLike fileAsStruct,
-        MetricsUtil.ReadableMetricsStruct readableMetrics) {
+        int position, StructLike fileAsStruct, MetricsUtil.ReadableMetricsStruct readableMetrics) {
       this.fileAsStruct = fileAsStruct;
       this.readableMetrics = readableMetrics;
-      this.expectedSize = expectedSize;
+      this.position = position;
     }
 
     @Override
     public int size() {
-      return expectedSize;
+      return position;
     }
 
     @Override
     public <T> T get(int pos, Class<T> javaClass) {
-      int lastExpectedIndex = expectedSize - 1;
-      if (pos < lastExpectedIndex) {
+      if (pos < position) {
         return fileAsStruct.get(pos, javaClass);
-      } else if (pos == lastExpectedIndex) {
+      } else if (pos == position) {
         return javaClass.cast(readableMetrics);
       } else {
-        throw new IllegalArgumentException(

Review Comment:
   metrics occupies a position.  `fileAsStruct` size + `Metrics` size(is 1) is equal to the size of `projection`. 
   
   When pos is greater than metricsPosition, the actual position of the field in `fileAsStruct` should be subtracted by 1.
   For instance:
   ```
   fileAsStruct :[0->c1, 1->c2]
   metrics :[0->c3]
   project:[c1,c3,c2]
   ```
   When we want to get `c2` (its pos is 2), but actually 1 in `fileAsStruct`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] hililiwei commented on a diff in pull request #6222: Flink: Support inspecting table

Posted by GitBox <gi...@apache.org>.
hililiwei commented on code in PR #6222:
URL: https://github.com/apache/iceberg/pull/6222#discussion_r1053955391


##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java:
##########
@@ -0,0 +1,300 @@
+/*
+ * 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.Duration;
+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.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericMapData;
+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.flink.types.RowKind;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.ByteBuffers;
+
+@Internal
+public class StructRowData implements RowData {
+  private final Types.StructType type;
+  private RowKind kind;
+  private StructLike struct;
+
+  public StructRowData(Types.StructType type) {
+    this(type, RowKind.INSERT);
+  }
+
+  public StructRowData(Types.StructType type, RowKind kind) {
+    this(type, null, kind);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct) {
+    this(type, struct, RowKind.INSERT);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct, RowKind kind) {
+    this.type = type;
+    this.struct = struct;
+    this.kind = kind;
+  }
+
+  public StructRowData setStruct(StructLike newStruct) {
+    this.struct = newStruct;
+    return this;
+  }
+
+  @Override
+  public int getArity() {
+    return struct.size();
+  }
+
+  @Override
+  public RowKind getRowKind() {
+    return kind;
+  }
+
+  @Override
+  public void setRowKind(RowKind newKind) {
+    Preconditions.checkNotNull(newKind, "kind can not be null");
+    this.kind = newKind;
+  }
+
+  @Override
+  public boolean isNullAt(int pos) {
+    return struct.get(pos, Object.class) == null;
+  }
+
+  @Override
+  public boolean getBoolean(int pos) {
+    return struct.get(pos, Boolean.class);
+  }
+
+  @Override
+  public byte getByte(int pos) {
+    return (byte) (int) struct.get(pos, Integer.class);
+  }
+
+  @Override
+  public short getShort(int pos) {
+    return (short) (int) struct.get(pos, Integer.class);
+  }
+
+  @Override
+  public int getInt(int pos) {
+    Object integer = struct.get(pos, Object.class);
+
+    if (integer instanceof Integer) {
+      return (int) integer;
+    } else if (integer instanceof LocalDate) {
+      return (int) ((LocalDate) integer).toEpochDay();
+    } else if (integer instanceof LocalTime) {
+      return (int) (((LocalTime) integer).toNanoOfDay() / 1000_000);
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for int field. Type name: " + integer.getClass().getName());
+    }
+  }
+
+  @Override
+  public long getLong(int pos) {
+    Object longVal = struct.get(pos, Object.class);
+
+    if (longVal instanceof Long) {
+      return (long) longVal;
+    } else if (longVal instanceof OffsetDateTime) {
+      return Duration.between(Instant.EPOCH, (OffsetDateTime) longVal).toNanos() / 1000;
+    } else if (longVal instanceof LocalDate) {
+      return ((LocalDate) longVal).toEpochDay();
+    } else if (longVal instanceof LocalTime) {
+      return ((LocalTime) longVal).toNanoOfDay();
+    } else if (longVal instanceof LocalDateTime) {
+      return Duration.between(Instant.EPOCH, ((LocalDateTime) longVal).atOffset(ZoneOffset.UTC))
+              .toNanos()
+          / 1000;
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for long field. Type name: " + longVal.getClass().getName());
+    }
+  }
+
+  @Override
+  public float getFloat(int pos) {
+    return struct.get(pos, Float.class);
+  }
+
+  @Override
+  public double getDouble(int pos) {
+    return struct.get(pos, Double.class);
+  }
+
+  @Override
+  public StringData getString(int pos) {
+    return isNullAt(pos) ? null : getStringDataInternal(pos);
+  }
+
+  private StringData getStringDataInternal(int pos) {
+    CharSequence seq = struct.get(pos, CharSequence.class);
+    return StringData.fromString(seq.toString());
+  }
+
+  @Override
+  public DecimalData getDecimal(int pos, int precision, int scale) {
+    return isNullAt(pos)
+        ? null
+        : DecimalData.fromBigDecimal(getDecimalInternal(pos), precision, scale);
+  }
+
+  private BigDecimal getDecimalInternal(int pos) {
+    return struct.get(pos, BigDecimal.class);
+  }
+
+  @Override
+  public TimestampData getTimestamp(int pos, int precision) {
+    long timeLong = getLong(pos);
+    return TimestampData.fromEpochMillis(timeLong / 1000, (int) (timeLong % 1000) * 1000);
+  }
+
+  @Override
+  public <T> RawValueData<T> getRawValue(int pos) {
+    throw new UnsupportedOperationException("Not supported yet.");
+  }
+
+  @Override
+  public byte[] getBinary(int pos) {
+    return isNullAt(pos) ? null : getBinaryInternal(pos);
+  }
+
+  private byte[] getBinaryInternal(int pos) {
+    Object bytes = struct.get(pos, Object.class);
+
+    // should only be either ByteBuffer or byte[]
+    if (bytes instanceof ByteBuffer) {
+      return ByteBuffers.toByteArray((ByteBuffer) bytes);
+    } else if (bytes instanceof byte[]) {
+      return (byte[]) bytes;
+    } else if (bytes instanceof UUID) {
+      UUID uuid = (UUID) bytes;
+      ByteBuffer bb = ByteBuffer.allocate(16);
+      bb.putLong(uuid.getMostSignificantBits());
+      bb.putLong(uuid.getLeastSignificantBits());
+      return bb.array();
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for binary field. Type name: " + bytes.getClass().getName());
+    }
+  }
+
+  @Override
+  public ArrayData getArray(int pos) {
+    return isNullAt(pos)
+        ? null
+        : (ArrayData)
+            coverValue(type.fields().get(pos).type().asListType(), struct.get(pos, List.class));
+  }
+
+  @Override
+  public MapData getMap(int pos) {
+    return isNullAt(pos)
+        ? null
+        : (MapData)
+            coverValue(type.fields().get(pos).type().asMapType(), struct.get(pos, Map.class));
+  }
+
+  @Override
+  public RowData getRow(int pos, int numFields) {
+    return isNullAt(pos) ? null : getStructRowData(pos, numFields);
+  }
+
+  private StructRowData getStructRowData(int pos, int numFields) {
+    return new StructRowData(
+        type.fields().get(pos).type().asStructType(), struct.get(pos, StructLike.class));
+  }
+
+  private <T> Object coverValue(Type elementType, Object value) {
+    switch (elementType.typeId()) {
+      case BOOLEAN:
+      case INTEGER:
+      case DATE:
+      case TIME:
+      case LONG:
+      case FLOAT:
+      case DOUBLE:
+      case DECIMAL:
+        return value;
+      case TIMESTAMP:
+        long millisecond = (long) value / 1000;
+        int nanoOfMillisecond = (int) ((Long) value % 1000) * 1000;
+        return TimestampData.fromEpochMillis(millisecond, nanoOfMillisecond);
+      case STRING:
+        return StringData.fromString(value.toString());
+      case FIXED:
+      case BINARY:
+        return ByteBuffers.toByteArray((ByteBuffer) value);
+      case STRUCT:
+        return new StructRowData(elementType.asStructType(), (StructLike) value);
+      case LIST:
+        List<Object> list = (List<Object>) value;
+        T[] array = (T[]) new Object[list.size()];

Review Comment:
   It won't compile.
   `Type parameter 'T' cannot be instantiated directly`
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] hililiwei commented on a diff in pull request #6222: Flink: Support inspecting table

Posted by GitBox <gi...@apache.org>.
hililiwei commented on code in PR #6222:
URL: https://github.com/apache/iceberg/pull/6222#discussion_r1053955391


##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java:
##########
@@ -0,0 +1,300 @@
+/*
+ * 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.Duration;
+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.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericMapData;
+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.flink.types.RowKind;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.ByteBuffers;
+
+@Internal
+public class StructRowData implements RowData {
+  private final Types.StructType type;
+  private RowKind kind;
+  private StructLike struct;
+
+  public StructRowData(Types.StructType type) {
+    this(type, RowKind.INSERT);
+  }
+
+  public StructRowData(Types.StructType type, RowKind kind) {
+    this(type, null, kind);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct) {
+    this(type, struct, RowKind.INSERT);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct, RowKind kind) {
+    this.type = type;
+    this.struct = struct;
+    this.kind = kind;
+  }
+
+  public StructRowData setStruct(StructLike newStruct) {
+    this.struct = newStruct;
+    return this;
+  }
+
+  @Override
+  public int getArity() {
+    return struct.size();
+  }
+
+  @Override
+  public RowKind getRowKind() {
+    return kind;
+  }
+
+  @Override
+  public void setRowKind(RowKind newKind) {
+    Preconditions.checkNotNull(newKind, "kind can not be null");
+    this.kind = newKind;
+  }
+
+  @Override
+  public boolean isNullAt(int pos) {
+    return struct.get(pos, Object.class) == null;
+  }
+
+  @Override
+  public boolean getBoolean(int pos) {
+    return struct.get(pos, Boolean.class);
+  }
+
+  @Override
+  public byte getByte(int pos) {
+    return (byte) (int) struct.get(pos, Integer.class);
+  }
+
+  @Override
+  public short getShort(int pos) {
+    return (short) (int) struct.get(pos, Integer.class);
+  }
+
+  @Override
+  public int getInt(int pos) {
+    Object integer = struct.get(pos, Object.class);
+
+    if (integer instanceof Integer) {
+      return (int) integer;
+    } else if (integer instanceof LocalDate) {
+      return (int) ((LocalDate) integer).toEpochDay();
+    } else if (integer instanceof LocalTime) {
+      return (int) (((LocalTime) integer).toNanoOfDay() / 1000_000);
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for int field. Type name: " + integer.getClass().getName());
+    }
+  }
+
+  @Override
+  public long getLong(int pos) {
+    Object longVal = struct.get(pos, Object.class);
+
+    if (longVal instanceof Long) {
+      return (long) longVal;
+    } else if (longVal instanceof OffsetDateTime) {
+      return Duration.between(Instant.EPOCH, (OffsetDateTime) longVal).toNanos() / 1000;
+    } else if (longVal instanceof LocalDate) {
+      return ((LocalDate) longVal).toEpochDay();
+    } else if (longVal instanceof LocalTime) {
+      return ((LocalTime) longVal).toNanoOfDay();
+    } else if (longVal instanceof LocalDateTime) {
+      return Duration.between(Instant.EPOCH, ((LocalDateTime) longVal).atOffset(ZoneOffset.UTC))
+              .toNanos()
+          / 1000;
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for long field. Type name: " + longVal.getClass().getName());
+    }
+  }
+
+  @Override
+  public float getFloat(int pos) {
+    return struct.get(pos, Float.class);
+  }
+
+  @Override
+  public double getDouble(int pos) {
+    return struct.get(pos, Double.class);
+  }
+
+  @Override
+  public StringData getString(int pos) {
+    return isNullAt(pos) ? null : getStringDataInternal(pos);
+  }
+
+  private StringData getStringDataInternal(int pos) {
+    CharSequence seq = struct.get(pos, CharSequence.class);
+    return StringData.fromString(seq.toString());
+  }
+
+  @Override
+  public DecimalData getDecimal(int pos, int precision, int scale) {
+    return isNullAt(pos)
+        ? null
+        : DecimalData.fromBigDecimal(getDecimalInternal(pos), precision, scale);
+  }
+
+  private BigDecimal getDecimalInternal(int pos) {
+    return struct.get(pos, BigDecimal.class);
+  }
+
+  @Override
+  public TimestampData getTimestamp(int pos, int precision) {
+    long timeLong = getLong(pos);
+    return TimestampData.fromEpochMillis(timeLong / 1000, (int) (timeLong % 1000) * 1000);
+  }
+
+  @Override
+  public <T> RawValueData<T> getRawValue(int pos) {
+    throw new UnsupportedOperationException("Not supported yet.");
+  }
+
+  @Override
+  public byte[] getBinary(int pos) {
+    return isNullAt(pos) ? null : getBinaryInternal(pos);
+  }
+
+  private byte[] getBinaryInternal(int pos) {
+    Object bytes = struct.get(pos, Object.class);
+
+    // should only be either ByteBuffer or byte[]
+    if (bytes instanceof ByteBuffer) {
+      return ByteBuffers.toByteArray((ByteBuffer) bytes);
+    } else if (bytes instanceof byte[]) {
+      return (byte[]) bytes;
+    } else if (bytes instanceof UUID) {
+      UUID uuid = (UUID) bytes;
+      ByteBuffer bb = ByteBuffer.allocate(16);
+      bb.putLong(uuid.getMostSignificantBits());
+      bb.putLong(uuid.getLeastSignificantBits());
+      return bb.array();
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for binary field. Type name: " + bytes.getClass().getName());
+    }
+  }
+
+  @Override
+  public ArrayData getArray(int pos) {
+    return isNullAt(pos)
+        ? null
+        : (ArrayData)
+            coverValue(type.fields().get(pos).type().asListType(), struct.get(pos, List.class));
+  }
+
+  @Override
+  public MapData getMap(int pos) {
+    return isNullAt(pos)
+        ? null
+        : (MapData)
+            coverValue(type.fields().get(pos).type().asMapType(), struct.get(pos, Map.class));
+  }
+
+  @Override
+  public RowData getRow(int pos, int numFields) {
+    return isNullAt(pos) ? null : getStructRowData(pos, numFields);
+  }
+
+  private StructRowData getStructRowData(int pos, int numFields) {
+    return new StructRowData(
+        type.fields().get(pos).type().asStructType(), struct.get(pos, StructLike.class));
+  }
+
+  private <T> Object coverValue(Type elementType, Object value) {
+    switch (elementType.typeId()) {
+      case BOOLEAN:
+      case INTEGER:
+      case DATE:
+      case TIME:
+      case LONG:
+      case FLOAT:
+      case DOUBLE:
+      case DECIMAL:
+        return value;
+      case TIMESTAMP:
+        long millisecond = (long) value / 1000;
+        int nanoOfMillisecond = (int) ((Long) value % 1000) * 1000;
+        return TimestampData.fromEpochMillis(millisecond, nanoOfMillisecond);
+      case STRING:
+        return StringData.fromString(value.toString());
+      case FIXED:
+      case BINARY:
+        return ByteBuffers.toByteArray((ByteBuffer) value);
+      case STRUCT:
+        return new StructRowData(elementType.asStructType(), (StructLike) value);
+      case LIST:
+        List<Object> list = (List<Object>) value;
+        T[] array = (T[]) new Object[list.size()];

Review Comment:
   compile error:
   `Type parameter 'T' cannot be instantiated directly`
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] stevenzwu commented on a diff in pull request #6222: Flink: Support inspecting table

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on code in PR #6222:
URL: https://github.com/apache/iceberg/pull/6222#discussion_r1066110588


##########
flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java:
##########
@@ -295,6 +299,161 @@ private static void assertEquals(
     }
   }
 
+  public static void assertEqualsSafe(
+      Schema schema, List<GenericData.Record> recs, List<Row> rows) {
+    Streams.forEachPair(
+        recs.stream(), rows.stream(), (rec, row) -> assertEqualsSafe(schema, rec, row));
+  }
+
+  public static void assertEqualsSafe(Schema schema, GenericData.Record rec, Row row) {
+    List<Types.NestedField> fields = schema.asStruct().fields();
+    RowType rowType = FlinkSchemaUtil.convert(schema);
+    for (int i = 0; i < fields.size(); i += 1) {
+      Type fieldType = fields.get(i).type();
+      Object expectedValue = rec.get(i);
+      Object actualValue = row.getField(i);
+      LogicalType logicalType = rowType.getTypeAt(i);
+      assertAvroEquals(fieldType, logicalType, expectedValue, actualValue);
+    }
+  }
+
+  private static void assertEqualsSafe(Types.StructType struct, GenericData.Record rec, Row row) {
+    List<Types.NestedField> fields = struct.fields();
+    for (int i = 0; i < fields.size(); i += 1) {
+      Type fieldType = fields.get(i).type();
+      Object expectedValue = rec.get(i);
+      Object actualValue = row.getField(i);
+      assertAvroEquals(fieldType, null, expectedValue, actualValue);
+    }
+  }
+
+  private static void assertAvroEquals(
+      Type type, LogicalType logicalType, Object expected, Object actual) {
+
+    if (expected == null && actual == null) {
+      return;
+    }
+
+    Assert.assertTrue(
+        "expected and actual should be both null or not null", expected != null && actual != null);
+
+    switch (type.typeId()) {
+      case BOOLEAN:

Review Comment:
   BOOLEAN to DOUBLE can be collapsed to avoid repetition. The error msg can include the type name. Also we should check the type of the expected and actual values. E.g., for INTEGER type, if both expected and actual are string values. the test will pass incorrectly.



##########
flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java:
##########
@@ -295,6 +299,161 @@ private static void assertEquals(
     }
   }
 
+  public static void assertEqualsSafe(
+      Schema schema, List<GenericData.Record> recs, List<Row> rows) {
+    Streams.forEachPair(
+        recs.stream(), rows.stream(), (rec, row) -> assertEqualsSafe(schema, rec, row));
+  }
+
+  public static void assertEqualsSafe(Schema schema, GenericData.Record rec, Row row) {
+    List<Types.NestedField> fields = schema.asStruct().fields();
+    RowType rowType = FlinkSchemaUtil.convert(schema);
+    for (int i = 0; i < fields.size(); i += 1) {
+      Type fieldType = fields.get(i).type();
+      Object expectedValue = rec.get(i);
+      Object actualValue = row.getField(i);
+      LogicalType logicalType = rowType.getTypeAt(i);
+      assertAvroEquals(fieldType, logicalType, expectedValue, actualValue);
+    }
+  }
+
+  private static void assertEqualsSafe(Types.StructType struct, GenericData.Record rec, Row row) {
+    List<Types.NestedField> fields = struct.fields();
+    for (int i = 0; i < fields.size(); i += 1) {
+      Type fieldType = fields.get(i).type();
+      Object expectedValue = rec.get(i);
+      Object actualValue = row.getField(i);
+      assertAvroEquals(fieldType, null, expectedValue, actualValue);
+    }
+  }
+
+  private static void assertAvroEquals(
+      Type type, LogicalType logicalType, Object expected, Object actual) {
+
+    if (expected == null && actual == null) {
+      return;
+    }
+
+    Assert.assertTrue(
+        "expected and actual should be both null or not null", expected != null && actual != null);
+
+    switch (type.typeId()) {
+      case BOOLEAN:

Review Comment:
   BOOLEAN to DOUBLE can be collapsed to avoid repetition. The error msg can include the type name. 
   
   Also we should check the type of the expected and actual values. E.g., for INTEGER type, if both expected and actual are string values. the test will pass incorrectly.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] hililiwei commented on a diff in pull request #6222: Flink: Support inspecting table

Posted by GitBox <gi...@apache.org>.
hililiwei commented on code in PR #6222:
URL: https://github.com/apache/iceberg/pull/6222#discussion_r1027062483


##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java:
##########
@@ -148,6 +150,17 @@ private Namespace toNamespace(String database) {
   }
 
   TableIdentifier toIdentifier(ObjectPath path) {
+    String objectName = path.getObjectName();
+    List<String> tableName = Splitter.on('$').splitToList(objectName);
+    if (tableName.size() > 1 && MetadataTableType.from(tableName.get(1)) != null) {

Review Comment:
   Check whether the name is points to a metadata table.
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] hililiwei commented on a diff in pull request #6222: Flink: Support inspecting table

Posted by GitBox <gi...@apache.org>.
hililiwei commented on code in PR #6222:
URL: https://github.com/apache/iceberg/pull/6222#discussion_r1037253800


##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java:
##########
@@ -148,6 +150,17 @@ private Namespace toNamespace(String database) {
   }
 
   TableIdentifier toIdentifier(ObjectPath path) {
+    String objectName = path.getObjectName();
+    List<String> tableName = Splitter.on('$').splitToList(objectName);
+    if (tableName.size() > 1 && MetadataTableType.from(tableName.get(1)) != null) {

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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] hililiwei commented on pull request #6222: Flink: Support inspecting table

Posted by GitBox <gi...@apache.org>.
hililiwei commented on PR #6222:
URL: https://github.com/apache/iceberg/pull/6222#issuecomment-1334989732

   > Do we want to introduce a different naming convention for Flink? How widespread is the usage of the SELECT * FROM prod.db.table$history like SQL?
   
   @pvary, thank you for your feedback.
   I personally like `SELECT * FROM prod.db.table.history` too, but unfortunately, flink does not support this syntax, and it discards `.history`. I tried for a long time and couldn't get it to work properly. Eventually, I had to choose the same way as flink-table-store.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] stevenzwu commented on a diff in pull request #6222: Flink: Support inspecting table

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on code in PR #6222:
URL: https://github.com/apache/iceberg/pull/6222#discussion_r1066087638


##########
core/src/main/java/org/apache/iceberg/BaseFilesTable.java:
##########
@@ -223,34 +226,37 @@ ManifestFile manifest() {
   static class ContentFileStructWithMetrics implements StructLike {
     private final StructLike fileAsStruct;
     private final MetricsUtil.ReadableMetricsStruct readableMetrics;
-    private final int expectedSize;
+    private final int columnCount;
+    private final int metricsPosition;
 
     ContentFileStructWithMetrics(
-        int expectedSize,
+        int columnCount,
+        int metricsPosition,
         StructLike fileAsStruct,
         MetricsUtil.ReadableMetricsStruct readableMetrics) {
       this.fileAsStruct = fileAsStruct;
       this.readableMetrics = readableMetrics;
-      this.expectedSize = expectedSize;
+      this.columnCount = columnCount;
+      this.metricsPosition = metricsPosition;
     }
 
     @Override
     public int size() {
-      return expectedSize;
+      return columnCount;
     }
 
     @Override
     public <T> T get(int pos, Class<T> javaClass) {
-      int lastExpectedIndex = expectedSize - 1;
-      if (pos < lastExpectedIndex) {
+      if (pos < metricsPosition) {
         return fileAsStruct.get(pos, javaClass);
-      } else if (pos == lastExpectedIndex) {
+      } else if (pos == metricsPosition) {
         return javaClass.cast(readableMetrics);
       } else {
-        throw new IllegalArgumentException(
-            String.format(
-                "Illegal position access for ContentFileStructWithMetrics: %d, max allowed is %d",
-                pos, lastExpectedIndex));
+        // columnCount = fileAsStruct column count + the readable metrics field.
+        // When pos is greater than metricsPosition, the actual position of the field in
+        // fileAsStruct should be
+        // subtracted by 1.

Review Comment:
   nit: merge this line with the line above



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] stevenzwu commented on a diff in pull request #6222: Flink: Support inspecting table

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on code in PR #6222:
URL: https://github.com/apache/iceberg/pull/6222#discussion_r1066088176


##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java:
##########
@@ -149,15 +151,28 @@ public Catalog catalog() {
     return icebergCatalog;
   }
 
-  private Namespace toNamespace(String database) {
+  /** Append a new level to the base namespace */
+  private static Namespace appendLevel(Namespace baseNamespace, String newLevel) {
     String[] namespace = new String[baseNamespace.levels().length + 1];
     System.arraycopy(baseNamespace.levels(), 0, namespace, 0, baseNamespace.levels().length);
-    namespace[baseNamespace.levels().length] = database;
+    namespace[baseNamespace.levels().length] = newLevel;
     return Namespace.of(namespace);
   }
 
   TableIdentifier toIdentifier(ObjectPath path) {
-    return TableIdentifier.of(toNamespace(path.getDatabaseName()), path.getObjectName());
+    String objectName = path.getObjectName();
+    List<String> tableName = Splitter.on('$').splitToList(objectName);
+
+    if (tableName.size() == 1) {
+      return TableIdentifier.of(
+          appendLevel(baseNamespace, path.getDatabaseName()), path.getObjectName());
+    } else if (tableName.size() == 2 && MetadataTableType.from(tableName.get(1)) != null) {
+      return TableIdentifier.of(
+          appendLevel(appendLevel(baseNamespace, path.getDatabaseName()), tableName.get(0)),
+          tableName.get(1));
+    } else {
+      throw new IllegalArgumentException("Illegal table name");

Review Comment:
   nit: include the `objectName` in the error msg



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] stevenzwu commented on a diff in pull request #6222: Flink: Support inspecting table

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on code in PR #6222:
URL: https://github.com/apache/iceberg/pull/6222#discussion_r1066146119


##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java:
##########
@@ -0,0 +1,286 @@
+/*
+ * 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.Duration;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.OffsetDateTime;
+import java.util.List;
+import java.util.Map;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericMapData;
+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.flink.types.RowKind;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.ByteBuffers;
+
+@Internal
+public class StructRowData implements RowData {
+  private final Types.StructType type;
+  private RowKind kind;
+  private StructLike struct;
+
+  public StructRowData(Types.StructType type) {
+    this(type, RowKind.INSERT);
+  }
+
+  public StructRowData(Types.StructType type, RowKind kind) {
+    this(type, null, kind);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct) {
+    this(type, struct, RowKind.INSERT);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct, RowKind kind) {
+    this.type = type;
+    this.struct = struct;
+    this.kind = kind;
+  }
+
+  public StructRowData setStruct(StructLike newStruct) {
+    this.struct = newStruct;
+    return this;
+  }
+
+  @Override
+  public int getArity() {
+    return struct.size();
+  }
+
+  @Override
+  public RowKind getRowKind() {
+    return kind;
+  }
+
+  @Override
+  public void setRowKind(RowKind newKind) {
+    Preconditions.checkNotNull(newKind, "kind can not be null");
+    this.kind = newKind;
+  }
+
+  @Override
+  public boolean isNullAt(int pos) {
+    return struct.get(pos, Object.class) == null;
+  }
+
+  @Override
+  public boolean getBoolean(int pos) {
+    return struct.get(pos, Boolean.class);
+  }
+
+  @Override
+  public byte getByte(int pos) {
+    return (byte) (int) struct.get(pos, Integer.class);
+  }
+
+  @Override
+  public short getShort(int pos) {
+    return (short) (int) struct.get(pos, Integer.class);
+  }
+
+  @Override
+  public int getInt(int pos) {
+    Object integer = struct.get(pos, Object.class);
+
+    if (integer instanceof Integer) {
+      return (int) integer;
+    } else if (integer instanceof LocalDate) {
+      return (int) ((LocalDate) integer).toEpochDay();
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for int field. Type name: " + integer.getClass().getName());
+    }
+  }
+
+  @Override
+  public long getLong(int pos) {
+    Object longVal = struct.get(pos, Object.class);
+
+    if (longVal instanceof Long) {
+      return (long) longVal;
+    } else if (longVal instanceof OffsetDateTime) {
+      return Duration.between(Instant.EPOCH, (OffsetDateTime) longVal).toNanos() / 1000;
+    } else if (longVal instanceof LocalDate) {
+      return ((LocalDate) longVal).toEpochDay();
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for long field. Type name: " + longVal.getClass().getName());
+    }
+  }
+
+  @Override
+  public float getFloat(int pos) {
+    return struct.get(pos, Float.class);
+  }
+
+  @Override
+  public double getDouble(int pos) {
+    return struct.get(pos, Double.class);
+  }
+
+  @Override
+  public StringData getString(int pos) {
+    return isNullAt(pos) ? null : getStringDataInternal(pos);
+  }
+
+  private StringData getStringDataInternal(int pos) {
+    CharSequence seq = struct.get(pos, CharSequence.class);
+    return StringData.fromString(seq.toString());
+  }
+
+  @Override
+  public DecimalData getDecimal(int pos, int precision, int scale) {
+    return isNullAt(pos)
+        ? null
+        : DecimalData.fromBigDecimal(getDecimalInternal(pos), precision, scale);
+  }
+
+  private BigDecimal getDecimalInternal(int pos) {
+    return struct.get(pos, BigDecimal.class);
+  }
+
+  @Override
+  public TimestampData getTimestamp(int pos, int precision) {
+    long timeLong = getLong(pos);
+    return TimestampData.fromEpochMillis(timeLong / 1000, (int) (timeLong % 1000) * 1000);
+  }
+
+  @Override
+  public <T> RawValueData<T> getRawValue(int pos) {
+    throw new UnsupportedOperationException("Not supported yet.");
+  }
+
+  @Override
+  public byte[] getBinary(int pos) {
+    return isNullAt(pos) ? null : getBinaryInternal(pos);
+  }
+
+  private byte[] getBinaryInternal(int pos) {
+    Object bytes = struct.get(pos, Object.class);
+
+    // should only be either ByteBuffer or byte[]
+    if (bytes instanceof ByteBuffer) {
+      return ByteBuffers.toByteArray((ByteBuffer) bytes);
+    } else if (bytes instanceof byte[]) {
+      return (byte[]) bytes;
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for binary field. Type name: " + bytes.getClass().getName());
+    }
+  }
+
+  @Override
+  public ArrayData getArray(int pos) {
+    return isNullAt(pos)
+        ? null
+        : (ArrayData)
+            covertFromStructToRowData(
+                type.fields().get(pos).type().asListType(), struct.get(pos, List.class));
+  }
+
+  @Override
+  public MapData getMap(int pos) {
+    return isNullAt(pos)
+        ? null
+        : (MapData)
+            covertFromStructToRowData(
+                type.fields().get(pos).type().asMapType(), struct.get(pos, Map.class));
+  }
+
+  @Override
+  public RowData getRow(int pos, int numFields) {
+    return isNullAt(pos) ? null : getStructRowData(pos, numFields);
+  }
+
+  private StructRowData getStructRowData(int pos, int numFields) {
+    return new StructRowData(
+        type.fields().get(pos).type().asStructType(), struct.get(pos, StructLike.class));
+  }
+
+  private Object covertFromStructToRowData(Type elementType, Object value) {
+    switch (elementType.typeId()) {
+      case BOOLEAN:
+      case INTEGER:
+      case DATE:
+      case TIME:
+      case LONG:
+      case FLOAT:
+      case DOUBLE:
+      case DECIMAL:
+        return value;
+      case TIMESTAMP:
+        return TimestampData.fromEpochMillis(
+            (Long) value / 1000, (int) ((Long) value % 1000) * 1000);
+      case STRING:
+        return StringData.fromString(value.toString());
+      case FIXED:
+      case BINARY:
+        return ByteBuffers.toByteArray((ByteBuffer) value);
+      case STRUCT:
+        return new StructRowData(elementType.asStructType(), (StructLike) value);
+      case LIST:
+        List<?> list = (List<?>) value;
+        Object[] array = new Object[list.size()];

Review Comment:
   It seems that both primitive and boxed types are acceptable. so we are probably ok here.
   
   ```
       @Override
       public int getInt(int pos) {
           return isPrimitiveArray ? ((int[]) array)[pos] : (int) getObject(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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] stevenzwu commented on a diff in pull request #6222: Flink: Support inspecting table

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on code in PR #6222:
URL: https://github.com/apache/iceberg/pull/6222#discussion_r1066146119


##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java:
##########
@@ -0,0 +1,286 @@
+/*
+ * 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.Duration;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.OffsetDateTime;
+import java.util.List;
+import java.util.Map;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericMapData;
+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.flink.types.RowKind;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.ByteBuffers;
+
+@Internal
+public class StructRowData implements RowData {
+  private final Types.StructType type;
+  private RowKind kind;
+  private StructLike struct;
+
+  public StructRowData(Types.StructType type) {
+    this(type, RowKind.INSERT);
+  }
+
+  public StructRowData(Types.StructType type, RowKind kind) {
+    this(type, null, kind);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct) {
+    this(type, struct, RowKind.INSERT);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct, RowKind kind) {
+    this.type = type;
+    this.struct = struct;
+    this.kind = kind;
+  }
+
+  public StructRowData setStruct(StructLike newStruct) {
+    this.struct = newStruct;
+    return this;
+  }
+
+  @Override
+  public int getArity() {
+    return struct.size();
+  }
+
+  @Override
+  public RowKind getRowKind() {
+    return kind;
+  }
+
+  @Override
+  public void setRowKind(RowKind newKind) {
+    Preconditions.checkNotNull(newKind, "kind can not be null");
+    this.kind = newKind;
+  }
+
+  @Override
+  public boolean isNullAt(int pos) {
+    return struct.get(pos, Object.class) == null;
+  }
+
+  @Override
+  public boolean getBoolean(int pos) {
+    return struct.get(pos, Boolean.class);
+  }
+
+  @Override
+  public byte getByte(int pos) {
+    return (byte) (int) struct.get(pos, Integer.class);
+  }
+
+  @Override
+  public short getShort(int pos) {
+    return (short) (int) struct.get(pos, Integer.class);
+  }
+
+  @Override
+  public int getInt(int pos) {
+    Object integer = struct.get(pos, Object.class);
+
+    if (integer instanceof Integer) {
+      return (int) integer;
+    } else if (integer instanceof LocalDate) {
+      return (int) ((LocalDate) integer).toEpochDay();
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for int field. Type name: " + integer.getClass().getName());
+    }
+  }
+
+  @Override
+  public long getLong(int pos) {
+    Object longVal = struct.get(pos, Object.class);
+
+    if (longVal instanceof Long) {
+      return (long) longVal;
+    } else if (longVal instanceof OffsetDateTime) {
+      return Duration.between(Instant.EPOCH, (OffsetDateTime) longVal).toNanos() / 1000;
+    } else if (longVal instanceof LocalDate) {
+      return ((LocalDate) longVal).toEpochDay();
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for long field. Type name: " + longVal.getClass().getName());
+    }
+  }
+
+  @Override
+  public float getFloat(int pos) {
+    return struct.get(pos, Float.class);
+  }
+
+  @Override
+  public double getDouble(int pos) {
+    return struct.get(pos, Double.class);
+  }
+
+  @Override
+  public StringData getString(int pos) {
+    return isNullAt(pos) ? null : getStringDataInternal(pos);
+  }
+
+  private StringData getStringDataInternal(int pos) {
+    CharSequence seq = struct.get(pos, CharSequence.class);
+    return StringData.fromString(seq.toString());
+  }
+
+  @Override
+  public DecimalData getDecimal(int pos, int precision, int scale) {
+    return isNullAt(pos)
+        ? null
+        : DecimalData.fromBigDecimal(getDecimalInternal(pos), precision, scale);
+  }
+
+  private BigDecimal getDecimalInternal(int pos) {
+    return struct.get(pos, BigDecimal.class);
+  }
+
+  @Override
+  public TimestampData getTimestamp(int pos, int precision) {
+    long timeLong = getLong(pos);
+    return TimestampData.fromEpochMillis(timeLong / 1000, (int) (timeLong % 1000) * 1000);
+  }
+
+  @Override
+  public <T> RawValueData<T> getRawValue(int pos) {
+    throw new UnsupportedOperationException("Not supported yet.");
+  }
+
+  @Override
+  public byte[] getBinary(int pos) {
+    return isNullAt(pos) ? null : getBinaryInternal(pos);
+  }
+
+  private byte[] getBinaryInternal(int pos) {
+    Object bytes = struct.get(pos, Object.class);
+
+    // should only be either ByteBuffer or byte[]
+    if (bytes instanceof ByteBuffer) {
+      return ByteBuffers.toByteArray((ByteBuffer) bytes);
+    } else if (bytes instanceof byte[]) {
+      return (byte[]) bytes;
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for binary field. Type name: " + bytes.getClass().getName());
+    }
+  }
+
+  @Override
+  public ArrayData getArray(int pos) {
+    return isNullAt(pos)
+        ? null
+        : (ArrayData)
+            covertFromStructToRowData(
+                type.fields().get(pos).type().asListType(), struct.get(pos, List.class));
+  }
+
+  @Override
+  public MapData getMap(int pos) {
+    return isNullAt(pos)
+        ? null
+        : (MapData)
+            covertFromStructToRowData(
+                type.fields().get(pos).type().asMapType(), struct.get(pos, Map.class));
+  }
+
+  @Override
+  public RowData getRow(int pos, int numFields) {
+    return isNullAt(pos) ? null : getStructRowData(pos, numFields);
+  }
+
+  private StructRowData getStructRowData(int pos, int numFields) {
+    return new StructRowData(
+        type.fields().get(pos).type().asStructType(), struct.get(pos, StructLike.class));
+  }
+
+  private Object covertFromStructToRowData(Type elementType, Object value) {
+    switch (elementType.typeId()) {
+      case BOOLEAN:
+      case INTEGER:
+      case DATE:
+      case TIME:
+      case LONG:
+      case FLOAT:
+      case DOUBLE:
+      case DECIMAL:
+        return value;
+      case TIMESTAMP:
+        return TimestampData.fromEpochMillis(
+            (Long) value / 1000, (int) ((Long) value % 1000) * 1000);
+      case STRING:
+        return StringData.fromString(value.toString());
+      case FIXED:
+      case BINARY:
+        return ByteBuffers.toByteArray((ByteBuffer) value);
+      case STRUCT:
+        return new StructRowData(elementType.asStructType(), (StructLike) value);
+      case LIST:
+        List<?> list = (List<?>) value;
+        Object[] array = new Object[list.size()];

Review Comment:
   It seems that both primitive and boxed types are acceptable. so we are probably ok here in terms of correctness, although primitive array is more space efficient.
   
   ```
       @Override
       public int getInt(int pos) {
           return isPrimitiveArray ? ((int[]) array)[pos] : (int) getObject(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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] hililiwei commented on pull request #6222: Flink: Support inspecting table

Posted by GitBox <gi...@apache.org>.
hililiwei commented on PR #6222:
URL: https://github.com/apache/iceberg/pull/6222#issuecomment-1338913731

   > @hililiwei we should add comprehensive unit test for `StructRowData`.
   > 
   > I have some internal DataGenerators for unit test code with very comprehensive coverage all field types (including complex nested types). Maybe I will submit a separate PR for that, which will cover Flink `RowData` and Iceberg `GenericRecord`. You can expand it with support for Iceberg `StructLike`. With that, we can write unit test/assertions compare the actual and expected.
   
   That would be great. I'm also thinking of adding UT for it. So i'll do it based on your code.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] stevenzwu commented on a diff in pull request #6222: Flink: Support inspecting table

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on code in PR #6222:
URL: https://github.com/apache/iceberg/pull/6222#discussion_r1042762795


##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java:
##########
@@ -140,15 +142,25 @@ public Catalog catalog() {
     return icebergCatalog;
   }
 
-  private Namespace toNamespace(String database) {
+  /** Append a new level to the base namespace */
+  private static Namespace appendLevel(Namespace baseNamespace, String newLevel) {
     String[] namespace = new String[baseNamespace.levels().length + 1];
     System.arraycopy(baseNamespace.levels(), 0, namespace, 0, baseNamespace.levels().length);
-    namespace[baseNamespace.levels().length] = database;
+    namespace[baseNamespace.levels().length] = newLevel;
     return Namespace.of(namespace);
   }
 
   TableIdentifier toIdentifier(ObjectPath path) {
-    return TableIdentifier.of(toNamespace(path.getDatabaseName()), path.getObjectName());
+    String objectName = path.getObjectName();
+    List<String> tableName = Splitter.on('$').splitToList(objectName);
+    if (tableName.size() == 2 && MetadataTableType.from(tableName.get(1)) != null) {

Review Comment:
   after split, the list should have size of either 0 (regular table) or 2 (metadata table). `$` shouldn't be a valid char in other names.
   
   if the list size is 1, 3, or any other numbers, it should be an illegal case.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] stevenzwu commented on a diff in pull request #6222: Flink: Support inspecting table

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on code in PR #6222:
URL: https://github.com/apache/iceberg/pull/6222#discussion_r1042762795


##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java:
##########
@@ -140,15 +142,25 @@ public Catalog catalog() {
     return icebergCatalog;
   }
 
-  private Namespace toNamespace(String database) {
+  /** Append a new level to the base namespace */
+  private static Namespace appendLevel(Namespace baseNamespace, String newLevel) {
     String[] namespace = new String[baseNamespace.levels().length + 1];
     System.arraycopy(baseNamespace.levels(), 0, namespace, 0, baseNamespace.levels().length);
-    namespace[baseNamespace.levels().length] = database;
+    namespace[baseNamespace.levels().length] = newLevel;
     return Namespace.of(namespace);
   }
 
   TableIdentifier toIdentifier(ObjectPath path) {
-    return TableIdentifier.of(toNamespace(path.getDatabaseName()), path.getObjectName());
+    String objectName = path.getObjectName();
+    List<String> tableName = Splitter.on('$').splitToList(objectName);
+    if (tableName.size() == 2 && MetadataTableType.from(tableName.get(1)) != null) {

Review Comment:
   after split, the list should have size of either 1 (regular table) or 2 (metadata table). `$` shouldn't be a valid char in other names.
   
   if the list size is 3 or any other numbers, it should be an illegal case.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] szehon-ho commented on a diff in pull request #6222: Flink: Support inspecting table

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on code in PR #6222:
URL: https://github.com/apache/iceberg/pull/6222#discussion_r1042775511


##########
core/src/main/java/org/apache/iceberg/BaseFilesTable.java:
##########
@@ -177,14 +177,17 @@ public CloseableIterable<StructLike> rows() {
         Set<Integer> readableMetricsIds = TypeUtil.getProjectedIds(readableMetricsField.type());
         Schema fileProjection = TypeUtil.selectNot(projection, readableMetricsIds);
 
+        int position = projection.columns().indexOf(readableMetricsField);

Review Comment:
   I guess this is the else case (where the field exists in user-projection).



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] hililiwei commented on a diff in pull request #6222: Flink: Support inspecting table

Posted by GitBox <gi...@apache.org>.
hililiwei commented on code in PR #6222:
URL: https://github.com/apache/iceberg/pull/6222#discussion_r1042013519


##########
core/src/main/java/org/apache/iceberg/BaseFilesTable.java:
##########
@@ -223,34 +225,28 @@ ManifestFile manifest() {
   static class ContentFileStructWithMetrics implements StructLike {
     private final StructLike fileAsStruct;
     private final MetricsUtil.ReadableMetricsStruct readableMetrics;
-    private final int expectedSize;
+    private final int position;

Review Comment:
   In flink, fields are obtained strictly in the order of  the projection. We can't guarantee that `ReadableMetrics` is always at the end, so I changed it. 
   cc @szehon-ho 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] stevenzwu commented on a diff in pull request #6222: Flink: Support inspecting table

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on code in PR #6222:
URL: https://github.com/apache/iceberg/pull/6222#discussion_r1036692307


##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java:
##########
@@ -148,6 +150,17 @@ private Namespace toNamespace(String database) {
   }
 
   TableIdentifier toIdentifier(ObjectPath path) {
+    String objectName = path.getObjectName();
+    List<String> tableName = Splitter.on('$').splitToList(objectName);
+    if (tableName.size() > 1 && MetadataTableType.from(tableName.get(1)) != null) {

Review Comment:
   we can apply more strict check here. if `objectName` contains `$`, it should only be one occurrence. In another word, `tableName.size()` should be 2.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] hililiwei commented on a diff in pull request #6222: Flink: Support inspecting table

Posted by GitBox <gi...@apache.org>.
hililiwei commented on code in PR #6222:
URL: https://github.com/apache/iceberg/pull/6222#discussion_r1053954779


##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java:
##########
@@ -0,0 +1,300 @@
+/*
+ * 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.Duration;
+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.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericMapData;
+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.flink.types.RowKind;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.ByteBuffers;
+
+@Internal
+public class StructRowData implements RowData {
+  private final Types.StructType type;
+  private RowKind kind;
+  private StructLike struct;
+
+  public StructRowData(Types.StructType type) {
+    this(type, RowKind.INSERT);
+  }
+
+  public StructRowData(Types.StructType type, RowKind kind) {
+    this(type, null, kind);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct) {
+    this(type, struct, RowKind.INSERT);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct, RowKind kind) {
+    this.type = type;
+    this.struct = struct;
+    this.kind = kind;
+  }
+
+  public StructRowData setStruct(StructLike newStruct) {
+    this.struct = newStruct;
+    return this;
+  }
+
+  @Override
+  public int getArity() {
+    return struct.size();
+  }
+
+  @Override
+  public RowKind getRowKind() {
+    return kind;
+  }
+
+  @Override
+  public void setRowKind(RowKind newKind) {
+    Preconditions.checkNotNull(newKind, "kind can not be null");
+    this.kind = newKind;
+  }
+
+  @Override
+  public boolean isNullAt(int pos) {
+    return struct.get(pos, Object.class) == null;
+  }
+
+  @Override
+  public boolean getBoolean(int pos) {
+    return struct.get(pos, Boolean.class);
+  }
+
+  @Override
+  public byte getByte(int pos) {
+    return (byte) (int) struct.get(pos, Integer.class);
+  }
+
+  @Override
+  public short getShort(int pos) {
+    return (short) (int) struct.get(pos, Integer.class);
+  }
+
+  @Override
+  public int getInt(int pos) {
+    Object integer = struct.get(pos, Object.class);
+
+    if (integer instanceof Integer) {
+      return (int) integer;
+    } else if (integer instanceof LocalDate) {
+      return (int) ((LocalDate) integer).toEpochDay();
+    } else if (integer instanceof LocalTime) {
+      return (int) (((LocalTime) integer).toNanoOfDay() / 1000_000);
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for int field. Type name: " + integer.getClass().getName());
+    }
+  }
+
+  @Override
+  public long getLong(int pos) {
+    Object longVal = struct.get(pos, Object.class);
+
+    if (longVal instanceof Long) {
+      return (long) longVal;
+    } else if (longVal instanceof OffsetDateTime) {
+      return Duration.between(Instant.EPOCH, (OffsetDateTime) longVal).toNanos() / 1000;
+    } else if (longVal instanceof LocalDate) {
+      return ((LocalDate) longVal).toEpochDay();
+    } else if (longVal instanceof LocalTime) {
+      return ((LocalTime) longVal).toNanoOfDay();
+    } else if (longVal instanceof LocalDateTime) {
+      return Duration.between(Instant.EPOCH, ((LocalDateTime) longVal).atOffset(ZoneOffset.UTC))
+              .toNanos()
+          / 1000;
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for long field. Type name: " + longVal.getClass().getName());
+    }
+  }
+
+  @Override
+  public float getFloat(int pos) {
+    return struct.get(pos, Float.class);
+  }
+
+  @Override
+  public double getDouble(int pos) {
+    return struct.get(pos, Double.class);
+  }
+
+  @Override
+  public StringData getString(int pos) {
+    return isNullAt(pos) ? null : getStringDataInternal(pos);
+  }
+
+  private StringData getStringDataInternal(int pos) {
+    CharSequence seq = struct.get(pos, CharSequence.class);
+    return StringData.fromString(seq.toString());
+  }
+
+  @Override
+  public DecimalData getDecimal(int pos, int precision, int scale) {
+    return isNullAt(pos)
+        ? null
+        : DecimalData.fromBigDecimal(getDecimalInternal(pos), precision, scale);
+  }
+
+  private BigDecimal getDecimalInternal(int pos) {
+    return struct.get(pos, BigDecimal.class);
+  }
+
+  @Override
+  public TimestampData getTimestamp(int pos, int precision) {
+    long timeLong = getLong(pos);
+    return TimestampData.fromEpochMillis(timeLong / 1000, (int) (timeLong % 1000) * 1000);
+  }
+
+  @Override
+  public <T> RawValueData<T> getRawValue(int pos) {
+    throw new UnsupportedOperationException("Not supported yet.");
+  }
+
+  @Override
+  public byte[] getBinary(int pos) {
+    return isNullAt(pos) ? null : getBinaryInternal(pos);
+  }
+
+  private byte[] getBinaryInternal(int pos) {
+    Object bytes = struct.get(pos, Object.class);
+
+    // should only be either ByteBuffer or byte[]
+    if (bytes instanceof ByteBuffer) {
+      return ByteBuffers.toByteArray((ByteBuffer) bytes);
+    } else if (bytes instanceof byte[]) {
+      return (byte[]) bytes;
+    } else if (bytes instanceof UUID) {
+      UUID uuid = (UUID) bytes;
+      ByteBuffer bb = ByteBuffer.allocate(16);
+      bb.putLong(uuid.getMostSignificantBits());
+      bb.putLong(uuid.getLeastSignificantBits());
+      return bb.array();
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for binary field. Type name: " + bytes.getClass().getName());
+    }
+  }
+
+  @Override
+  public ArrayData getArray(int pos) {
+    return isNullAt(pos)
+        ? null
+        : (ArrayData)
+            coverValue(type.fields().get(pos).type().asListType(), struct.get(pos, List.class));
+  }
+
+  @Override
+  public MapData getMap(int pos) {
+    return isNullAt(pos)
+        ? null
+        : (MapData)
+            coverValue(type.fields().get(pos).type().asMapType(), struct.get(pos, Map.class));
+  }
+
+  @Override
+  public RowData getRow(int pos, int numFields) {
+    return isNullAt(pos) ? null : getStructRowData(pos, numFields);
+  }
+
+  private StructRowData getStructRowData(int pos, int numFields) {
+    return new StructRowData(
+        type.fields().get(pos).type().asStructType(), struct.get(pos, StructLike.class));
+  }
+
+  private <T> Object coverValue(Type elementType, Object value) {

Review Comment:
   thx, 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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] hililiwei commented on a diff in pull request #6222: Flink: Support inspecting table

Posted by GitBox <gi...@apache.org>.
hililiwei commented on code in PR #6222:
URL: https://github.com/apache/iceberg/pull/6222#discussion_r1039702122


##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java:
##########
@@ -0,0 +1,340 @@
+/*
+ * 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.Duration;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.OffsetDateTime;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.function.BiConsumer;
+import java.util.function.Function;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericMapData;
+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.flink.types.RowKind;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+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.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.ByteBuffers;
+
+public class StructRowData implements RowData {
+  private final Types.StructType type;
+  private RowKind kind;
+  private StructLike struct;
+
+  public StructRowData(Types.StructType type) {
+    this(type, RowKind.INSERT);
+  }
+
+  public StructRowData(Types.StructType type, RowKind kind) {
+    this(type, null, kind);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct) {
+    this(type, struct, RowKind.INSERT);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct, RowKind kind) {
+    this.type = type;
+    this.struct = struct;
+    this.kind = kind;
+  }
+
+  public StructRowData setStruct(StructLike newStruct) {
+    this.struct = newStruct;
+    return this;
+  }
+
+  @Override
+  public int getArity() {
+    return struct.size();
+  }
+
+  @Override
+  public RowKind getRowKind() {
+    return kind;
+  }
+
+  @Override
+  public void setRowKind(RowKind newKind) {
+    Preconditions.checkNotNull(newKind, "kind can not be null");
+    this.kind = newKind;
+  }
+
+  @Override
+  public boolean isNullAt(int pos) {
+    return struct.get(pos, Object.class) == null;
+  }
+
+  @Override
+  public boolean getBoolean(int pos) {
+    return struct.get(pos, Boolean.class);
+  }
+
+  @Override
+  public byte getByte(int pos) {
+    return (byte) (int) struct.get(pos, Integer.class);
+  }
+
+  @Override
+  public short getShort(int pos) {
+    return (short) (int) struct.get(pos, Integer.class);
+  }
+
+  @Override
+  public int getInt(int pos) {
+    Object integer = struct.get(pos, Object.class);
+
+    if (integer instanceof Integer) {
+      return (int) integer;
+    } else if (integer instanceof LocalDate) {
+      return (int) ((LocalDate) integer).toEpochDay();
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for int field. Type name: " + integer.getClass().getName());
+    }
+  }
+
+  @Override
+  public long getLong(int pos) {
+    Object longVal = struct.get(pos, Object.class);
+
+    if (longVal instanceof Long) {
+      return (long) longVal;
+    } else if (longVal instanceof OffsetDateTime) {
+      return Duration.between(Instant.EPOCH, (OffsetDateTime) longVal).toNanos() / 1000;
+    } else if (longVal instanceof LocalDate) {
+      return ((LocalDate) longVal).toEpochDay();
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for long field. Type name: " + longVal.getClass().getName());
+    }
+  }
+
+  @Override
+  public float getFloat(int pos) {
+    return struct.get(pos, Float.class);
+  }
+
+  @Override
+  public double getDouble(int pos) {
+    return struct.get(pos, Double.class);
+  }
+
+  @Override
+  public StringData getString(int pos) {
+    return isNullAt(pos) ? null : getStringDataInternal(pos);
+  }
+
+  private StringData getStringDataInternal(int pos) {
+    CharSequence seq = struct.get(pos, CharSequence.class);
+    return StringData.fromString(seq.toString());
+  }
+
+  @Override
+  public DecimalData getDecimal(int pos, int precision, int scale) {
+    return isNullAt(pos)
+        ? null
+        : DecimalData.fromBigDecimal(getDecimalInternal(pos), precision, scale);
+  }
+
+  private BigDecimal getDecimalInternal(int pos) {
+    return struct.get(pos, BigDecimal.class);
+  }
+
+  @Override
+  public TimestampData getTimestamp(int pos, int precision) {
+    long timeLong = getLong(pos);

Review Comment:
   The SnapshotsTable multiplies the timeMicro by 1000 before returning the data.
   
   https://github.com/apache/iceberg/blob/2ecce262355c3362ba126d9d30e6df0ce42d14c2/core/src/main/java/org/apache/iceberg/SnapshotsTable.java#L104-L113



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] hililiwei commented on pull request #6222: Flink: Support inspecting table

Posted by GitBox <gi...@apache.org>.
hililiwei commented on PR #6222:
URL: https://github.com/apache/iceberg/pull/6222#issuecomment-1330562655

   also cc @pvary, thx.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] stevenzwu commented on a diff in pull request #6222: Flink: Support inspecting table

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on code in PR #6222:
URL: https://github.com/apache/iceberg/pull/6222#discussion_r1040359570


##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java:
##########
@@ -0,0 +1,340 @@
+/*
+ * 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.Duration;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.OffsetDateTime;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.function.BiConsumer;
+import java.util.function.Function;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericMapData;
+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.flink.types.RowKind;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+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.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.ByteBuffers;
+
+public class StructRowData implements RowData {
+  private final Types.StructType type;
+  private RowKind kind;
+  private StructLike struct;
+
+  public StructRowData(Types.StructType type) {
+    this(type, RowKind.INSERT);
+  }
+
+  public StructRowData(Types.StructType type, RowKind kind) {
+    this(type, null, kind);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct) {
+    this(type, struct, RowKind.INSERT);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct, RowKind kind) {
+    this.type = type;
+    this.struct = struct;
+    this.kind = kind;
+  }
+
+  public StructRowData setStruct(StructLike newStruct) {
+    this.struct = newStruct;
+    return this;
+  }
+
+  @Override
+  public int getArity() {
+    return struct.size();
+  }
+
+  @Override
+  public RowKind getRowKind() {
+    return kind;
+  }
+
+  @Override
+  public void setRowKind(RowKind newKind) {
+    Preconditions.checkNotNull(newKind, "kind can not be null");
+    this.kind = newKind;
+  }
+
+  @Override
+  public boolean isNullAt(int pos) {
+    return struct.get(pos, Object.class) == null;
+  }
+
+  @Override
+  public boolean getBoolean(int pos) {
+    return struct.get(pos, Boolean.class);
+  }
+
+  @Override
+  public byte getByte(int pos) {
+    return (byte) (int) struct.get(pos, Integer.class);
+  }
+
+  @Override
+  public short getShort(int pos) {
+    return (short) (int) struct.get(pos, Integer.class);
+  }
+
+  @Override
+  public int getInt(int pos) {
+    Object integer = struct.get(pos, Object.class);
+
+    if (integer instanceof Integer) {
+      return (int) integer;
+    } else if (integer instanceof LocalDate) {
+      return (int) ((LocalDate) integer).toEpochDay();
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for int field. Type name: " + integer.getClass().getName());
+    }
+  }
+
+  @Override
+  public long getLong(int pos) {
+    Object longVal = struct.get(pos, Object.class);
+
+    if (longVal instanceof Long) {
+      return (long) longVal;
+    } else if (longVal instanceof OffsetDateTime) {
+      return Duration.between(Instant.EPOCH, (OffsetDateTime) longVal).toNanos() / 1000;
+    } else if (longVal instanceof LocalDate) {
+      return ((LocalDate) longVal).toEpochDay();
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for long field. Type name: " + longVal.getClass().getName());
+    }
+  }
+
+  @Override
+  public float getFloat(int pos) {
+    return struct.get(pos, Float.class);
+  }
+
+  @Override
+  public double getDouble(int pos) {
+    return struct.get(pos, Double.class);
+  }
+
+  @Override
+  public StringData getString(int pos) {
+    return isNullAt(pos) ? null : getStringDataInternal(pos);
+  }
+
+  private StringData getStringDataInternal(int pos) {
+    CharSequence seq = struct.get(pos, CharSequence.class);
+    return StringData.fromString(seq.toString());
+  }
+
+  @Override
+  public DecimalData getDecimal(int pos, int precision, int scale) {
+    return isNullAt(pos)
+        ? null
+        : DecimalData.fromBigDecimal(getDecimalInternal(pos), precision, scale);
+  }
+
+  private BigDecimal getDecimalInternal(int pos) {
+    return struct.get(pos, BigDecimal.class);
+  }
+
+  @Override
+  public TimestampData getTimestamp(int pos, int precision) {
+    long timeLong = getLong(pos);

Review Comment:
   it is `millis * 1000`. so it should be `micro` seconds.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] hililiwei commented on a diff in pull request #6222: Flink: Support inspecting table

Posted by GitBox <gi...@apache.org>.
hililiwei commented on code in PR #6222:
URL: https://github.com/apache/iceberg/pull/6222#discussion_r1040547304


##########
flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java:
##########
@@ -295,6 +299,161 @@ private static void assertEquals(
     }
   }
 
+  public static void assertEqualsSafe(
+      Schema schema, List<GenericData.Record> recs, List<Row> rows) {

Review Comment:
   The metadata files are stored in avro format, and when we read them directly, we get the Avro generic record.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] pvary commented on pull request #6222: Flink: Support inspecting table

Posted by GitBox <gi...@apache.org>.
pvary commented on PR #6222:
URL: https://github.com/apache/iceberg/pull/6222#issuecomment-1336096520

   Sadly, I do not have time to properly review the PR. I just wanted to mention that @szlta found issues with partition evolution for v1 and v2 tables as well when we tested the Hive metadata tables.
   IIRC the issue was happening when we created a partition with 2 columns (a, b), then altered the partitioning scheme to one where 1 colum was changed (a, c).
   See the relevant PRs here: https://github.com/apache/iceberg/pulls?q=is%3Apr+szlta+partition
   
   If I understand correctly, this is solved on the API level 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] pvary closed pull request #6222: Flink: Support inspecting table

Posted by GitBox <gi...@apache.org>.
pvary closed pull request #6222: Flink: Support inspecting table
URL: https://github.com/apache/iceberg/pull/6222


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] hililiwei commented on pull request #6222: Flink: Support inspecting table

Posted by GitBox <gi...@apache.org>.
hililiwei commented on PR #6222:
URL: https://github.com/apache/iceberg/pull/6222#issuecomment-1321565947

   @stevenzwu @openinx @rdblue  @Fokko    could you please take a look at it when you get a chance? thx.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] stevenzwu commented on a diff in pull request #6222: Flink: Support inspecting table

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on code in PR #6222:
URL: https://github.com/apache/iceberg/pull/6222#discussion_r1042716897


##########
core/src/main/java/org/apache/iceberg/BaseFilesTable.java:
##########
@@ -223,34 +225,28 @@ ManifestFile manifest() {
   static class ContentFileStructWithMetrics implements StructLike {
     private final StructLike fileAsStruct;
     private final MetricsUtil.ReadableMetricsStruct readableMetrics;
-    private final int expectedSize;
+    private final int position;
 
     ContentFileStructWithMetrics(
-        int expectedSize,
-        StructLike fileAsStruct,
-        MetricsUtil.ReadableMetricsStruct readableMetrics) {
+        int position, StructLike fileAsStruct, MetricsUtil.ReadableMetricsStruct readableMetrics) {
       this.fileAsStruct = fileAsStruct;
       this.readableMetrics = readableMetrics;
-      this.expectedSize = expectedSize;
+      this.position = position;
     }
 
     @Override
     public int size() {
-      return expectedSize;
+      return position;
     }
 
     @Override
     public <T> T get(int pos, Class<T> javaClass) {
-      int lastExpectedIndex = expectedSize - 1;
-      if (pos < lastExpectedIndex) {
+      if (pos < position) {
         return fileAsStruct.get(pos, javaClass);
-      } else if (pos == lastExpectedIndex) {
+      } else if (pos == position) {
         return javaClass.cast(readableMetrics);
       } else {
-        throw new IllegalArgumentException(

Review Comment:
   I don't quite understand the logic here. shouldn't we still need the `expectedSize` and throw an exception if the `pos` is over the size limit?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] stevenzwu commented on a diff in pull request #6222: Flink: Support inspecting table

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on code in PR #6222:
URL: https://github.com/apache/iceberg/pull/6222#discussion_r1042759545


##########
core/src/main/java/org/apache/iceberg/BaseFilesTable.java:
##########
@@ -223,34 +225,28 @@ ManifestFile manifest() {
   static class ContentFileStructWithMetrics implements StructLike {
     private final StructLike fileAsStruct;
     private final MetricsUtil.ReadableMetricsStruct readableMetrics;
-    private final int expectedSize;
+    private final int position;

Review Comment:
   It is probably better to call this `metricsPosition`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] hililiwei commented on a diff in pull request #6222: Flink: Support inspecting table

Posted by GitBox <gi...@apache.org>.
hililiwei commented on code in PR #6222:
URL: https://github.com/apache/iceberg/pull/6222#discussion_r1039674684


##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java:
##########
@@ -0,0 +1,340 @@
+/*
+ * 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.Duration;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.OffsetDateTime;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.function.BiConsumer;
+import java.util.function.Function;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericMapData;
+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.flink.types.RowKind;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+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.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.ByteBuffers;
+
+public class StructRowData implements RowData {
+  private final Types.StructType type;
+  private RowKind kind;
+  private StructLike struct;
+
+  public StructRowData(Types.StructType type) {
+    this(type, RowKind.INSERT);
+  }
+
+  public StructRowData(Types.StructType type, RowKind kind) {
+    this(type, null, kind);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct) {
+    this(type, struct, RowKind.INSERT);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct, RowKind kind) {
+    this.type = type;
+    this.struct = struct;
+    this.kind = kind;
+  }
+
+  public StructRowData setStruct(StructLike newStruct) {
+    this.struct = newStruct;
+    return this;
+  }
+
+  @Override
+  public int getArity() {
+    return struct.size();
+  }
+
+  @Override
+  public RowKind getRowKind() {
+    return kind;
+  }
+
+  @Override
+  public void setRowKind(RowKind newKind) {
+    Preconditions.checkNotNull(newKind, "kind can not be null");
+    this.kind = newKind;
+  }
+
+  @Override
+  public boolean isNullAt(int pos) {
+    return struct.get(pos, Object.class) == null;
+  }
+
+  @Override
+  public boolean getBoolean(int pos) {
+    return struct.get(pos, Boolean.class);
+  }
+
+  @Override
+  public byte getByte(int pos) {
+    return (byte) (int) struct.get(pos, Integer.class);
+  }
+
+  @Override
+  public short getShort(int pos) {
+    return (short) (int) struct.get(pos, Integer.class);
+  }
+
+  @Override
+  public int getInt(int pos) {
+    Object integer = struct.get(pos, Object.class);
+
+    if (integer instanceof Integer) {
+      return (int) integer;
+    } else if (integer instanceof LocalDate) {
+      return (int) ((LocalDate) integer).toEpochDay();
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for int field. Type name: " + integer.getClass().getName());
+    }
+  }
+
+  @Override
+  public long getLong(int pos) {
+    Object longVal = struct.get(pos, Object.class);
+
+    if (longVal instanceof Long) {
+      return (long) longVal;
+    } else if (longVal instanceof OffsetDateTime) {
+      return Duration.between(Instant.EPOCH, (OffsetDateTime) longVal).toNanos() / 1000;
+    } else if (longVal instanceof LocalDate) {
+      return ((LocalDate) longVal).toEpochDay();
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for long field. Type name: " + longVal.getClass().getName());
+    }
+  }
+
+  @Override
+  public float getFloat(int pos) {
+    return struct.get(pos, Float.class);
+  }
+
+  @Override
+  public double getDouble(int pos) {
+    return struct.get(pos, Double.class);
+  }
+
+  @Override
+  public StringData getString(int pos) {
+    return isNullAt(pos) ? null : getStringDataInternal(pos);
+  }
+
+  private StringData getStringDataInternal(int pos) {
+    CharSequence seq = struct.get(pos, CharSequence.class);
+    return StringData.fromString(seq.toString());
+  }
+
+  @Override
+  public DecimalData getDecimal(int pos, int precision, int scale) {
+    return isNullAt(pos)
+        ? null
+        : DecimalData.fromBigDecimal(getDecimalInternal(pos), precision, scale);
+  }
+
+  private BigDecimal getDecimalInternal(int pos) {
+    return struct.get(pos, BigDecimal.class);
+  }
+
+  @Override
+  public TimestampData getTimestamp(int pos, int precision) {
+    long timeLong = getLong(pos);
+    if (precision == 6) {
+      int nanosOfMillisecond = (int) (timeLong % 1000);
+      return TimestampData.fromEpochMillis(timeLong / 1000, nanosOfMillisecond);
+    } else {
+      return TimestampData.fromEpochMillis(timeLong);
+    }
+  }
+
+  @Override
+  public <T> RawValueData<T> getRawValue(int pos) {
+    return RawValueData.fromBytes(getBinary(pos));
+  }
+
+  @Override
+  public byte[] getBinary(int pos) {
+    return isNullAt(pos) ? null : getBinaryInternal(pos);
+  }
+
+  private byte[] getBinaryInternal(int pos) {
+    Object bytes = struct.get(pos, Object.class);
+
+    // should only be either ByteBuffer or byte[]
+    if (bytes instanceof ByteBuffer) {
+      return ByteBuffers.toByteArray((ByteBuffer) bytes);
+    } else if (bytes instanceof byte[]) {
+      return (byte[]) bytes;
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for binary field. Type name: " + bytes.getClass().getName());
+    }
+  }
+
+  @Override
+  public ArrayData getArray(int pos) {
+    return isNullAt(pos) ? null : getArrayInternal(pos);
+  }
+
+  private ArrayData getArrayInternal(int pos) {
+    return collectionToArrayData(
+        type.fields().get(pos).type().asListType().elementType(),
+        struct.get(pos, Collection.class));
+  }
+
+  @Override
+  public MapData getMap(int pos) {
+    return isNullAt(pos) ? null : getMapInternal(pos);
+  }
+
+  private MapData getMapInternal(int pos) {
+    return mapToMapData(type.fields().get(pos).type().asMapType(), struct.get(pos, Map.class));
+  }
+
+  @Override
+  public RowData getRow(int pos, int numFields) {
+    return isNullAt(pos) ? null : getStructRowData(pos, numFields);
+  }
+
+  private StructRowData getStructRowData(int ordinal, int numFields) {
+    return new StructRowData(
+        type.fields().get(ordinal).type().asStructType(), struct.get(ordinal, StructLike.class));
+  }
+
+  private MapData mapToMapData(Types.MapType mapType, Map<?, ?> map) {
+    // make a defensive copy to ensure entries do not change
+    List<Map.Entry<?, ?>> entries = ImmutableList.copyOf(map.entrySet());
+
+    ArrayData keyArray =

Review Comment:
   We need to convert the values in map according to the type of Flink, and put type and value into mapdata. The difference can be seen below:
   ![image](https://user-images.githubusercontent.com/59213263/205661819-874742df-a9b8-48ae-ad57-2d27e019632a.png)
   only `new GenericMapData(map)`
   
   ![image](https://user-images.githubusercontent.com/59213263/205661843-846f9be9-6b3a-49cc-8187-fd2c7b442e45.png)
    converting map to two lists
   
   If we just use `new GenericMapData(map)`, when try to run `SELECT summary['flink.job-id'] FROM %s$snapshots`, we will get `ClassCastException: java.lang.String cannot be cast to org.apache.flink.table.data.StringData`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] hililiwei commented on a diff in pull request #6222: Flink: Support inspecting table

Posted by GitBox <gi...@apache.org>.
hililiwei commented on code in PR #6222:
URL: https://github.com/apache/iceberg/pull/6222#discussion_r1039570407


##########
flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMetaDataTable.java:
##########
@@ -0,0 +1,713 @@
+/*
+ * 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.source;
+
+import java.io.IOException;
+import java.time.Instant;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.avro.generic.GenericData;
+import org.apache.commons.collections.ListUtils;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.CoreOptions;
+import org.apache.flink.table.api.TableEnvironment;
+import org.apache.flink.types.Row;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileContent;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Files;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.MetadataTableUtils;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.data.FileHelpers;
+import org.apache.iceberg.data.GenericRecord;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.flink.FlinkCatalogTestBase;
+import org.apache.iceberg.flink.FlinkConfigOptions;
+import org.apache.iceberg.flink.TestHelpers;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.util.SnapshotUtil;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+public class TestFlinkMetaDataTable extends FlinkCatalogTestBase {
+
+  private static final String TABLE_NAME = "test_table";
+  private final FileFormat format = FileFormat.AVRO;
+  private static TemporaryFolder temp = new TemporaryFolder();
+
+  public TestFlinkMetaDataTable(String catalogName, Namespace baseNamespace) {
+    super(catalogName, baseNamespace);
+  }
+
+  @Override
+  protected TableEnvironment getTableEnv() {
+    Configuration configuration = super.getTableEnv().getConfig().getConfiguration();
+    configuration.set(CoreOptions.DEFAULT_PARALLELISM, 1);
+    configuration.set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE, true);
+    return super.getTableEnv();
+  }
+
+  @Before
+  public void before() {
+    super.before();
+    sql("CREATE DATABASE %s", flinkDatabase);
+    sql("USE CATALOG %s", catalogName);
+    sql("USE %s", DATABASE);
+  }
+
+  @Override
+  @After
+  public void clean() {
+    sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, TABLE_NAME);
+    sql("DROP DATABASE IF EXISTS %s", flinkDatabase);
+    super.clean();
+  }
+
+  @Test
+  public void testSnapshots() {
+    sql(
+        "CREATE TABLE %s (id INT, data VARCHAR,d DOUBLE) WITH ('format-version'='2', 'write.format.default'='%s')",
+        TABLE_NAME, format.name());
+    sql(
+        "INSERT INTO %s VALUES (1,'iceberg',10),(2,'b',20),(3,CAST(NULL AS VARCHAR),30)",
+        TABLE_NAME);
+    String sql = String.format("SELECT * FROM %s$snapshots ", TABLE_NAME);
+    List<Row> result = sql(sql);
+
+    Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME));
+    Iterator<Snapshot> snapshots = table.snapshots().iterator();
+    for (Row row : result) {
+      Snapshot next = snapshots.next();
+      Assert.assertEquals(
+          "Should have expected timestamp",
+          ((Instant) row.getField(0)).toEpochMilli(),
+          next.timestampMillis());
+      Assert.assertEquals("Should have expected snapshot id", row.getField(1), next.snapshotId());
+      Assert.assertEquals("Should have expected parent id", row.getField(2), next.parentId());
+      Assert.assertEquals("Should have expected operation", row.getField(3), next.operation());
+      Assert.assertEquals(
+          "Should have expected manifest list location",
+          row.getField(4),
+          next.manifestListLocation());
+      Assert.assertEquals("Should have expected summary", row.getField(5), next.summary());
+    }
+  }
+
+  @Test
+  public void testHistory() {
+    sql(
+        "CREATE TABLE %s (id INT, data VARCHAR,d DOUBLE) WITH ('format-version'='2', 'write.format.default'='%s')",
+        TABLE_NAME, format.name());
+    sql(
+        "INSERT INTO %s VALUES (1,'iceberg',10),(2,'b',20),(3,CAST(NULL AS VARCHAR),30)",
+        TABLE_NAME);
+    String sql = String.format("SELECT * FROM %s$history ", TABLE_NAME);
+    List<Row> result = sql(sql);
+
+    Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME));
+    Iterator<Snapshot> snapshots = table.snapshots().iterator();
+    for (Row row : result) {
+      Snapshot next = snapshots.next();
+      Assert.assertEquals(
+          "Should have expected made_current_at",
+          ((Instant) row.getField(0)).toEpochMilli(),
+          next.timestampMillis());
+      Assert.assertEquals("Should have expected snapshot id", row.getField(1), next.snapshotId());
+      Assert.assertEquals("Should have expected parent id", row.getField(2), next.parentId());
+      Assert.assertEquals(
+          "Should have expected is current ancestor",
+          row.getField(3),
+          SnapshotUtil.isAncestorOf(
+              table, next.snapshotId(), table.currentSnapshot().snapshotId()));
+    }
+  }
+
+  @Test
+  public void testManifests() {
+    sql(
+        "CREATE TABLE %s (id INT, data VARCHAR,d DOUBLE) WITH ('format-version'='2', 'write.format.default'='%s')",
+        TABLE_NAME, format.name());
+    sql(
+        "INSERT INTO %s VALUES (1,'iceberg',10),(2,'b',20),(3,CAST(NULL AS VARCHAR),30)",
+        TABLE_NAME);
+    String sql = String.format("SELECT * FROM %s$manifests ", TABLE_NAME);
+    List<Row> result = sql(sql);
+
+    Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME));
+    List<ManifestFile> expectedDataManifests = dataManifests(table);
+
+    for (int i = 0; i < result.size(); i++) {
+      Row row = result.get(i);
+      ManifestFile manifestFile = expectedDataManifests.get(i);
+      Assert.assertEquals(
+          "Should have expected content", row.getField(0), manifestFile.content().id());
+      Assert.assertEquals("Should have expected path", row.getField(1), manifestFile.path());
+      Assert.assertEquals("Should have expected length", row.getField(2), manifestFile.length());
+      Assert.assertEquals(
+          "Should have expected partition_spec_id",
+          row.getField(3),
+          manifestFile.partitionSpecId());
+      Assert.assertEquals(
+          "Should have expected added_snapshot_id", row.getField(4), manifestFile.snapshotId());
+      Assert.assertEquals(
+          "Should have expected added_data_files_count",
+          row.getField(5),
+          manifestFile.addedFilesCount());
+      Assert.assertEquals(
+          "Should have expected existing_data_files_count",
+          row.getField(6),
+          manifestFile.existingFilesCount());
+      Assert.assertEquals(
+          "Should have expected deleted_data_files_count",
+          row.getField(7),
+          manifestFile.deletedFilesCount());
+    }
+  }
+
+  @Test
+  public void testAllManifests() {

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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] hililiwei commented on a diff in pull request #6222: Flink: Support inspecting table

Posted by GitBox <gi...@apache.org>.
hililiwei commented on code in PR #6222:
URL: https://github.com/apache/iceberg/pull/6222#discussion_r1039570783


##########
flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMetaDataTable.java:
##########
@@ -0,0 +1,713 @@
+/*
+ * 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.source;
+
+import java.io.IOException;
+import java.time.Instant;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.avro.generic.GenericData;
+import org.apache.commons.collections.ListUtils;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.CoreOptions;
+import org.apache.flink.table.api.TableEnvironment;
+import org.apache.flink.types.Row;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileContent;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Files;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.MetadataTableUtils;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.data.FileHelpers;
+import org.apache.iceberg.data.GenericRecord;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.flink.FlinkCatalogTestBase;
+import org.apache.iceberg.flink.FlinkConfigOptions;
+import org.apache.iceberg.flink.TestHelpers;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.util.SnapshotUtil;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+public class TestFlinkMetaDataTable extends FlinkCatalogTestBase {
+
+  private static final String TABLE_NAME = "test_table";
+  private final FileFormat format = FileFormat.AVRO;
+  private static TemporaryFolder temp = new TemporaryFolder();
+
+  public TestFlinkMetaDataTable(String catalogName, Namespace baseNamespace) {
+    super(catalogName, baseNamespace);
+  }
+
+  @Override
+  protected TableEnvironment getTableEnv() {
+    Configuration configuration = super.getTableEnv().getConfig().getConfiguration();
+    configuration.set(CoreOptions.DEFAULT_PARALLELISM, 1);
+    configuration.set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE, true);
+    return super.getTableEnv();
+  }
+
+  @Before
+  public void before() {
+    super.before();
+    sql("CREATE DATABASE %s", flinkDatabase);
+    sql("USE CATALOG %s", catalogName);
+    sql("USE %s", DATABASE);
+  }
+
+  @Override
+  @After
+  public void clean() {
+    sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, TABLE_NAME);
+    sql("DROP DATABASE IF EXISTS %s", flinkDatabase);
+    super.clean();
+  }
+
+  @Test
+  public void testSnapshots() {
+    sql(
+        "CREATE TABLE %s (id INT, data VARCHAR,d DOUBLE) WITH ('format-version'='2', 'write.format.default'='%s')",
+        TABLE_NAME, format.name());
+    sql(
+        "INSERT INTO %s VALUES (1,'iceberg',10),(2,'b',20),(3,CAST(NULL AS VARCHAR),30)",
+        TABLE_NAME);
+    String sql = String.format("SELECT * FROM %s$snapshots ", TABLE_NAME);
+    List<Row> result = sql(sql);
+
+    Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME));
+    Iterator<Snapshot> snapshots = table.snapshots().iterator();
+    for (Row row : result) {
+      Snapshot next = snapshots.next();
+      Assert.assertEquals(
+          "Should have expected timestamp",
+          ((Instant) row.getField(0)).toEpochMilli(),
+          next.timestampMillis());
+      Assert.assertEquals("Should have expected snapshot id", row.getField(1), next.snapshotId());
+      Assert.assertEquals("Should have expected parent id", row.getField(2), next.parentId());
+      Assert.assertEquals("Should have expected operation", row.getField(3), next.operation());
+      Assert.assertEquals(
+          "Should have expected manifest list location",
+          row.getField(4),
+          next.manifestListLocation());
+      Assert.assertEquals("Should have expected summary", row.getField(5), next.summary());
+    }
+  }
+
+  @Test
+  public void testHistory() {
+    sql(
+        "CREATE TABLE %s (id INT, data VARCHAR,d DOUBLE) WITH ('format-version'='2', 'write.format.default'='%s')",
+        TABLE_NAME, format.name());
+    sql(
+        "INSERT INTO %s VALUES (1,'iceberg',10),(2,'b',20),(3,CAST(NULL AS VARCHAR),30)",
+        TABLE_NAME);
+    String sql = String.format("SELECT * FROM %s$history ", TABLE_NAME);
+    List<Row> result = sql(sql);
+
+    Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME));
+    Iterator<Snapshot> snapshots = table.snapshots().iterator();
+    for (Row row : result) {
+      Snapshot next = snapshots.next();
+      Assert.assertEquals(
+          "Should have expected made_current_at",
+          ((Instant) row.getField(0)).toEpochMilli(),
+          next.timestampMillis());
+      Assert.assertEquals("Should have expected snapshot id", row.getField(1), next.snapshotId());
+      Assert.assertEquals("Should have expected parent id", row.getField(2), next.parentId());
+      Assert.assertEquals(

Review Comment:
   Multiple snapshots have been 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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] hililiwei commented on a diff in pull request #6222: Flink: Support inspecting table

Posted by GitBox <gi...@apache.org>.
hililiwei commented on code in PR #6222:
URL: https://github.com/apache/iceberg/pull/6222#discussion_r1037191643


##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java:
##########
@@ -0,0 +1,340 @@
+/*
+ * 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.Duration;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.OffsetDateTime;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.function.BiConsumer;
+import java.util.function.Function;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericMapData;
+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.flink.types.RowKind;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+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.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.ByteBuffers;
+
+public class StructRowData implements RowData {
+  private final Types.StructType type;
+  private RowKind kind;
+  private StructLike struct;
+
+  public StructRowData(Types.StructType type) {
+    this(type, RowKind.INSERT);
+  }
+
+  public StructRowData(Types.StructType type, RowKind kind) {
+    this(type, null, kind);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct) {
+    this(type, struct, RowKind.INSERT);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct, RowKind kind) {
+    this.type = type;
+    this.struct = struct;
+    this.kind = kind;
+  }
+
+  public StructRowData setStruct(StructLike newStruct) {
+    this.struct = newStruct;
+    return this;
+  }
+
+  @Override
+  public int getArity() {
+    return struct.size();
+  }
+
+  @Override
+  public RowKind getRowKind() {
+    return kind;
+  }
+
+  @Override
+  public void setRowKind(RowKind newKind) {
+    Preconditions.checkNotNull(newKind, "kind can not be null");
+    this.kind = newKind;
+  }
+
+  @Override
+  public boolean isNullAt(int pos) {
+    return struct.get(pos, Object.class) == null;
+  }
+
+  @Override
+  public boolean getBoolean(int pos) {
+    return struct.get(pos, Boolean.class);
+  }
+
+  @Override
+  public byte getByte(int pos) {
+    return (byte) (int) struct.get(pos, Integer.class);

Review Comment:
   All types here are derived from 
   https://github.com/apache/iceberg/blob/49a0ea956e3a4b9979754c887d803d4ab51131ae/api/src/main/java/org/apache/iceberg/types/Types.java#L40-L54
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] hililiwei commented on a diff in pull request #6222: Flink: Support inspecting table

Posted by GitBox <gi...@apache.org>.
hililiwei commented on code in PR #6222:
URL: https://github.com/apache/iceberg/pull/6222#discussion_r1037936682


##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java:
##########
@@ -148,6 +150,17 @@ private Namespace toNamespace(String database) {
   }
 
   TableIdentifier toIdentifier(ObjectPath path) {
+    String objectName = path.getObjectName();
+    List<String> tableName = Splitter.on('$').splitToList(objectName);
+    if (tableName.size() > 1 && MetadataTableType.from(tableName.get(1)) != null) {
+      return TableIdentifier.parse(
+          toNamespace(path.getDatabaseName()).toString()

Review Comment:
   Done. Please see if it meets your expectations.
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] stevenzwu commented on a diff in pull request #6222: Flink: Support inspecting table

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on code in PR #6222:
URL: https://github.com/apache/iceberg/pull/6222#discussion_r1036696028


##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java:
##########
@@ -148,6 +150,17 @@ private Namespace toNamespace(String database) {
   }
 
   TableIdentifier toIdentifier(ObjectPath path) {
+    String objectName = path.getObjectName();
+    List<String> tableName = Splitter.on('$').splitToList(objectName);
+    if (tableName.size() > 1 && MetadataTableType.from(tableName.get(1)) != null) {
+      return TableIdentifier.parse(
+          toNamespace(path.getDatabaseName()).toString()

Review Comment:
   I would make `toNamespace` a static method taking two params (instead of one currently)
   ```
   /** Append a new level to the base namespace */
   private static Namespace appendLevel(Namespace baseNamespace, String newLevel)
   ```
   
   Then we can append a new level `tableName.get(0)` to the baseNamespace. treat `tableName.get(1)` as name when constructing the TableIdentifier.
   ```
   public static TableIdentifier of(Namespace namespace, String name)
   ```
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] stevenzwu commented on a diff in pull request #6222: Flink: Support inspecting table

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on code in PR #6222:
URL: https://github.com/apache/iceberg/pull/6222#discussion_r1040393766


##########
flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java:
##########
@@ -295,6 +299,161 @@ private static void assertEquals(
     }
   }
 
+  public static void assertEqualsSafe(
+      Schema schema, List<GenericData.Record> recs, List<Row> rows) {

Review Comment:
   why do we need the helper methods with Avro generic record?



##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java:
##########
@@ -0,0 +1,340 @@
+/*
+ * 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.Duration;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.OffsetDateTime;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.function.BiConsumer;
+import java.util.function.Function;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericMapData;
+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.flink.types.RowKind;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+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.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.ByteBuffers;
+
+public class StructRowData implements RowData {
+  private final Types.StructType type;
+  private RowKind kind;
+  private StructLike struct;
+
+  public StructRowData(Types.StructType type) {
+    this(type, RowKind.INSERT);
+  }
+
+  public StructRowData(Types.StructType type, RowKind kind) {
+    this(type, null, kind);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct) {
+    this(type, struct, RowKind.INSERT);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct, RowKind kind) {
+    this.type = type;
+    this.struct = struct;
+    this.kind = kind;
+  }
+
+  public StructRowData setStruct(StructLike newStruct) {
+    this.struct = newStruct;
+    return this;
+  }
+
+  @Override
+  public int getArity() {
+    return struct.size();
+  }
+
+  @Override
+  public RowKind getRowKind() {
+    return kind;
+  }
+
+  @Override
+  public void setRowKind(RowKind newKind) {
+    Preconditions.checkNotNull(newKind, "kind can not be null");
+    this.kind = newKind;
+  }
+
+  @Override
+  public boolean isNullAt(int pos) {
+    return struct.get(pos, Object.class) == null;
+  }
+
+  @Override
+  public boolean getBoolean(int pos) {
+    return struct.get(pos, Boolean.class);
+  }
+
+  @Override
+  public byte getByte(int pos) {
+    return (byte) (int) struct.get(pos, Integer.class);
+  }
+
+  @Override
+  public short getShort(int pos) {
+    return (short) (int) struct.get(pos, Integer.class);
+  }
+
+  @Override
+  public int getInt(int pos) {
+    Object integer = struct.get(pos, Object.class);
+
+    if (integer instanceof Integer) {
+      return (int) integer;
+    } else if (integer instanceof LocalDate) {
+      return (int) ((LocalDate) integer).toEpochDay();
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for int field. Type name: " + integer.getClass().getName());
+    }
+  }
+
+  @Override
+  public long getLong(int pos) {
+    Object longVal = struct.get(pos, Object.class);
+
+    if (longVal instanceof Long) {
+      return (long) longVal;
+    } else if (longVal instanceof OffsetDateTime) {
+      return Duration.between(Instant.EPOCH, (OffsetDateTime) longVal).toNanos() / 1000;
+    } else if (longVal instanceof LocalDate) {
+      return ((LocalDate) longVal).toEpochDay();
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for long field. Type name: " + longVal.getClass().getName());
+    }
+  }
+
+  @Override
+  public float getFloat(int pos) {
+    return struct.get(pos, Float.class);
+  }
+
+  @Override
+  public double getDouble(int pos) {
+    return struct.get(pos, Double.class);
+  }
+
+  @Override
+  public StringData getString(int pos) {
+    return isNullAt(pos) ? null : getStringDataInternal(pos);
+  }
+
+  private StringData getStringDataInternal(int pos) {
+    CharSequence seq = struct.get(pos, CharSequence.class);
+    return StringData.fromString(seq.toString());
+  }
+
+  @Override
+  public DecimalData getDecimal(int pos, int precision, int scale) {
+    return isNullAt(pos)
+        ? null
+        : DecimalData.fromBigDecimal(getDecimalInternal(pos), precision, scale);
+  }
+
+  private BigDecimal getDecimalInternal(int pos) {
+    return struct.get(pos, BigDecimal.class);
+  }
+
+  @Override
+  public TimestampData getTimestamp(int pos, int precision) {
+    long timeLong = getLong(pos);
+    if (precision == 6) {
+      int nanosOfMillisecond = (int) (timeLong % 1000);
+      return TimestampData.fromEpochMillis(timeLong / 1000, nanosOfMillisecond);
+    } else {
+      return TimestampData.fromEpochMillis(timeLong);
+    }
+  }
+
+  @Override
+  public <T> RawValueData<T> getRawValue(int pos) {
+    return RawValueData.fromBytes(getBinary(pos));
+  }
+
+  @Override
+  public byte[] getBinary(int pos) {
+    return isNullAt(pos) ? null : getBinaryInternal(pos);
+  }
+
+  private byte[] getBinaryInternal(int pos) {
+    Object bytes = struct.get(pos, Object.class);
+
+    // should only be either ByteBuffer or byte[]
+    if (bytes instanceof ByteBuffer) {
+      return ByteBuffers.toByteArray((ByteBuffer) bytes);
+    } else if (bytes instanceof byte[]) {
+      return (byte[]) bytes;
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for binary field. Type name: " + bytes.getClass().getName());
+    }
+  }
+
+  @Override
+  public ArrayData getArray(int pos) {
+    return isNullAt(pos) ? null : getArrayInternal(pos);
+  }
+
+  private ArrayData getArrayInternal(int pos) {
+    return collectionToArrayData(
+        type.fields().get(pos).type().asListType().elementType(),
+        struct.get(pos, Collection.class));
+  }
+
+  @Override
+  public MapData getMap(int pos) {
+    return isNullAt(pos) ? null : getMapInternal(pos);
+  }
+
+  private MapData getMapInternal(int pos) {
+    return mapToMapData(type.fields().get(pos).type().asMapType(), struct.get(pos, Map.class));
+  }
+
+  @Override
+  public RowData getRow(int pos, int numFields) {
+    return isNullAt(pos) ? null : getStructRowData(pos, numFields);
+  }
+
+  private StructRowData getStructRowData(int ordinal, int numFields) {
+    return new StructRowData(
+        type.fields().get(ordinal).type().asStructType(), struct.get(ordinal, StructLike.class));
+  }
+
+  private MapData mapToMapData(Types.MapType mapType, Map<?, ?> map) {
+    // make a defensive copy to ensure entries do not change
+    List<Map.Entry<?, ?>> entries = ImmutableList.copyOf(map.entrySet());
+
+    ArrayData keyArray =

Review Comment:
   ok. I got it that we need run the conversion recursively (e.g. for StringData key or RowData value). I like to propose a diff structure for this class.
   
   We can repurpose `collectionToArrayData` to a `convertFromStructToRowData(Type type, Object value)` method. If `type` is most primitive, it should be simple pass-thru. for other types (like timestamp, struct, map, array) there are some conversions to Flink types (like TimestampData, RowData, MapData, ArrayData). 
   
   Then for the map, we don't have to convert to key and value array. I believe the reason you did it is to reuse the `collectionToArrayData`. 



##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java:
##########
@@ -0,0 +1,340 @@
+/*
+ * 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.Duration;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.OffsetDateTime;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.function.BiConsumer;
+import java.util.function.Function;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericMapData;
+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.flink.types.RowKind;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+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.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.ByteBuffers;
+
+public class StructRowData implements RowData {
+  private final Types.StructType type;
+  private RowKind kind;
+  private StructLike struct;
+
+  public StructRowData(Types.StructType type) {
+    this(type, RowKind.INSERT);
+  }
+
+  public StructRowData(Types.StructType type, RowKind kind) {
+    this(type, null, kind);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct) {
+    this(type, struct, RowKind.INSERT);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct, RowKind kind) {
+    this.type = type;
+    this.struct = struct;
+    this.kind = kind;
+  }
+
+  public StructRowData setStruct(StructLike newStruct) {
+    this.struct = newStruct;
+    return this;
+  }
+
+  @Override
+  public int getArity() {
+    return struct.size();
+  }
+
+  @Override
+  public RowKind getRowKind() {
+    return kind;
+  }
+
+  @Override
+  public void setRowKind(RowKind newKind) {
+    Preconditions.checkNotNull(newKind, "kind can not be null");
+    this.kind = newKind;
+  }
+
+  @Override
+  public boolean isNullAt(int pos) {
+    return struct.get(pos, Object.class) == null;
+  }
+
+  @Override
+  public boolean getBoolean(int pos) {
+    return struct.get(pos, Boolean.class);
+  }
+
+  @Override
+  public byte getByte(int pos) {
+    return (byte) (int) struct.get(pos, Integer.class);
+  }
+
+  @Override
+  public short getShort(int pos) {
+    return (short) (int) struct.get(pos, Integer.class);
+  }
+
+  @Override
+  public int getInt(int pos) {
+    Object integer = struct.get(pos, Object.class);
+
+    if (integer instanceof Integer) {
+      return (int) integer;
+    } else if (integer instanceof LocalDate) {
+      return (int) ((LocalDate) integer).toEpochDay();
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for int field. Type name: " + integer.getClass().getName());
+    }
+  }
+
+  @Override
+  public long getLong(int pos) {
+    Object longVal = struct.get(pos, Object.class);
+
+    if (longVal instanceof Long) {
+      return (long) longVal;
+    } else if (longVal instanceof OffsetDateTime) {
+      return Duration.between(Instant.EPOCH, (OffsetDateTime) longVal).toNanos() / 1000;
+    } else if (longVal instanceof LocalDate) {
+      return ((LocalDate) longVal).toEpochDay();
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for long field. Type name: " + longVal.getClass().getName());
+    }
+  }
+
+  @Override
+  public float getFloat(int pos) {
+    return struct.get(pos, Float.class);
+  }
+
+  @Override
+  public double getDouble(int pos) {
+    return struct.get(pos, Double.class);
+  }
+
+  @Override
+  public StringData getString(int pos) {
+    return isNullAt(pos) ? null : getStringDataInternal(pos);
+  }
+
+  private StringData getStringDataInternal(int pos) {
+    CharSequence seq = struct.get(pos, CharSequence.class);
+    return StringData.fromString(seq.toString());
+  }
+
+  @Override
+  public DecimalData getDecimal(int pos, int precision, int scale) {
+    return isNullAt(pos)
+        ? null
+        : DecimalData.fromBigDecimal(getDecimalInternal(pos), precision, scale);
+  }
+
+  private BigDecimal getDecimalInternal(int pos) {
+    return struct.get(pos, BigDecimal.class);
+  }
+
+  @Override
+  public TimestampData getTimestamp(int pos, int precision) {
+    long timeLong = getLong(pos);

Review Comment:
   I think we can simply call this `Timestamp` constructor
   ```
       private TimestampData(long millisecond, int nanoOfMillisecond) {
           Preconditions.checkArgument(nanoOfMillisecond >= 0 && nanoOfMillisecond <= 999_999);
           this.millisecond = millisecond;
           this.nanoOfMillisecond = nanoOfMillisecond;
       }
   ```
   
   We can ignore the `precision` arg like `ArrowTimestampColumnVector` from Flink.
   ```
       @Override
       public TimestampData getTimestamp(int i, int precision) {
           if (valueVector instanceof TimeStampSecVector) {
               return TimestampData.fromEpochMillis(((TimeStampSecVector) valueVector).get(i) * 1000);
           } else if (valueVector instanceof TimeStampMilliVector) {
               return TimestampData.fromEpochMillis(((TimeStampMilliVector) valueVector).get(i));
           } else if (valueVector instanceof TimeStampMicroVector) {
               long micros = ((TimeStampMicroVector) valueVector).get(i);
               return TimestampData.fromEpochMillis(micros / 1000, (int) (micros % 1000) * 1000);
           } else {
               long nanos = ((TimeStampNanoVector) valueVector).get(i);
               return TimestampData.fromEpochMillis(nanos / 1_000_000, (int) (nanos % 1_000_000));
           }
       }
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] hililiwei commented on a diff in pull request #6222: Flink: Support inspecting table

Posted by GitBox <gi...@apache.org>.
hililiwei commented on code in PR #6222:
URL: https://github.com/apache/iceberg/pull/6222#discussion_r1053917332


##########
flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java:
##########
@@ -295,6 +299,161 @@ private static void assertEquals(
     }
   }
 
+  public static void assertEqualsSafe(
+      Schema schema, List<GenericData.Record> recs, List<Row> rows) {
+    Streams.forEachPair(
+        recs.stream(), rows.stream(), (rec, row) -> assertEqualsSafe(schema, rec, row));
+  }
+
+  public static void assertEqualsSafe(Schema schema, GenericData.Record rec, Row row) {
+    List<Types.NestedField> fields = schema.asStruct().fields();
+    RowType rowType = FlinkSchemaUtil.convert(schema);
+    for (int i = 0; i < fields.size(); i += 1) {
+      Type fieldType = fields.get(i).type();
+      Object expectedValue = rec.get(i);
+      Object actualValue = row.getField(i);
+      LogicalType logicalType = rowType.getTypeAt(i);
+      assertAvroEquals(fieldType, logicalType, expectedValue, actualValue);
+    }
+  }
+
+  private static void assertEqualsSafe(Types.StructType struct, GenericData.Record rec, Row row) {
+    List<Types.NestedField> fields = struct.fields();
+    for (int i = 0; i < fields.size(); i += 1) {
+      Type fieldType = fields.get(i).type();
+      Object expectedValue = rec.get(i);
+      Object actualValue = row.getField(i);
+      assertAvroEquals(fieldType, null, expectedValue, actualValue);
+    }
+  }
+
+  private static void assertAvroEquals(
+      Type type, LogicalType logicalType, Object expected, Object actual) {
+
+    if (expected == null && actual == null) {
+      return;
+    }
+
+    Assert.assertTrue(
+        "expected and actual should be both null or not null", expected != null && actual != null);
+
+    switch (type.typeId()) {
+      case BOOLEAN:
+        Assert.assertEquals("boolean value should be equal", expected, actual);
+        break;
+      case INTEGER:
+        Assert.assertEquals("int value should be equal", expected, actual);
+        break;
+      case LONG:
+        Assert.assertEquals("long value should be equal", expected, actual);
+        break;
+      case FLOAT:
+        Assert.assertEquals("float value should be equal", expected, actual);
+        break;
+      case DOUBLE:
+        Assert.assertEquals("double value should be equal", expected, actual);
+        break;
+      case STRING:
+        Assertions.assertThat(expected)
+            .as("Should expect a CharSequence")
+            .isInstanceOf(CharSequence.class);
+        Assert.assertEquals("string should be equal", String.valueOf(expected), actual.toString());
+        break;
+      case DATE:
+        Assertions.assertThat(expected).as("Should expect a Date").isInstanceOf(LocalDate.class);

Review Comment:
   I'm not sure about that. If it is needed, it would be good to add it.
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] hililiwei commented on pull request #6222: Flink: Support inspecting table

Posted by GitBox <gi...@apache.org>.
hililiwei commented on PR #6222:
URL: https://github.com/apache/iceberg/pull/6222#issuecomment-1360903987

   > like to get more feedback for adding a new public method in the api module. cc @rdblue @aokolnychyi @RussellSpitzer @szehon-ho
   > 
   > Here is the context why we need this. The converter in `FlinkSchemaUtil` creates Iceberg `Schema` without doc attributes. Hence @hililiwei implemented the util class and method to carry over the doc attribute from `docSourceSchema` to `schema` (converted from Flink `TableSchema`).
   > 
   > ```
   >   public static Schema convert(TableSchema schema) {
   >   }
   > ```
   > 
   > I suggested moving it into `TypeUtil` class in api module as it is very similar to the `reassignIds` method in the `TypeUtil` class.
   
   cc @rdblue @aokolnychyi @RussellSpitzer @szehon-ho


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] hililiwei commented on pull request #6222: Flink: Support inspecting table

Posted by GitBox <gi...@apache.org>.
hililiwei commented on PR #6222:
URL: https://github.com/apache/iceberg/pull/6222#issuecomment-1359359424

   Added test case for StructRowData. @stevenzwu please take a look, thx.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] stevenzwu commented on a diff in pull request #6222: Flink: Support inspecting table

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on code in PR #6222:
URL: https://github.com/apache/iceberg/pull/6222#discussion_r1053535295


##########
core/src/main/java/org/apache/iceberg/BaseFilesTable.java:
##########
@@ -223,34 +225,28 @@ ManifestFile manifest() {
   static class ContentFileStructWithMetrics implements StructLike {
     private final StructLike fileAsStruct;
     private final MetricsUtil.ReadableMetricsStruct readableMetrics;
-    private final int expectedSize;
+    private final int position;
 
     ContentFileStructWithMetrics(
-        int expectedSize,
-        StructLike fileAsStruct,
-        MetricsUtil.ReadableMetricsStruct readableMetrics) {
+        int position, StructLike fileAsStruct, MetricsUtil.ReadableMetricsStruct readableMetrics) {
       this.fileAsStruct = fileAsStruct;
       this.readableMetrics = readableMetrics;
-      this.expectedSize = expectedSize;
+      this.position = position;
     }
 
     @Override
     public int size() {
-      return expectedSize;
+      return position;
     }
 
     @Override
     public <T> T get(int pos, Class<T> javaClass) {
-      int lastExpectedIndex = expectedSize - 1;
-      if (pos < lastExpectedIndex) {
+      if (pos < position) {
         return fileAsStruct.get(pos, javaClass);
-      } else if (pos == lastExpectedIndex) {
+      } else if (pos == position) {
         return javaClass.cast(readableMetrics);
       } else {
-        throw new IllegalArgumentException(

Review Comment:
   got it now. let's add some comments to explain the subtraction by 1.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] hililiwei commented on a diff in pull request #6222: Flink: Support inspecting table

Posted by GitBox <gi...@apache.org>.
hililiwei commented on code in PR #6222:
URL: https://github.com/apache/iceberg/pull/6222#discussion_r1051383730


##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java:
##########
@@ -0,0 +1,286 @@
+/*
+ * 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.Duration;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.OffsetDateTime;
+import java.util.List;
+import java.util.Map;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericMapData;
+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.flink.types.RowKind;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.ByteBuffers;
+
+@Internal
+public class StructRowData implements RowData {
+  private final Types.StructType type;
+  private RowKind kind;
+  private StructLike struct;
+
+  public StructRowData(Types.StructType type) {
+    this(type, RowKind.INSERT);
+  }
+
+  public StructRowData(Types.StructType type, RowKind kind) {
+    this(type, null, kind);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct) {
+    this(type, struct, RowKind.INSERT);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct, RowKind kind) {
+    this.type = type;
+    this.struct = struct;
+    this.kind = kind;
+  }
+
+  public StructRowData setStruct(StructLike newStruct) {
+    this.struct = newStruct;
+    return this;
+  }
+
+  @Override
+  public int getArity() {
+    return struct.size();
+  }
+
+  @Override
+  public RowKind getRowKind() {
+    return kind;
+  }
+
+  @Override
+  public void setRowKind(RowKind newKind) {
+    Preconditions.checkNotNull(newKind, "kind can not be null");
+    this.kind = newKind;
+  }
+
+  @Override
+  public boolean isNullAt(int pos) {
+    return struct.get(pos, Object.class) == null;
+  }
+
+  @Override
+  public boolean getBoolean(int pos) {
+    return struct.get(pos, Boolean.class);
+  }
+
+  @Override
+  public byte getByte(int pos) {
+    return (byte) (int) struct.get(pos, Integer.class);
+  }
+
+  @Override
+  public short getShort(int pos) {
+    return (short) (int) struct.get(pos, Integer.class);
+  }
+
+  @Override
+  public int getInt(int pos) {
+    Object integer = struct.get(pos, Object.class);
+
+    if (integer instanceof Integer) {
+      return (int) integer;
+    } else if (integer instanceof LocalDate) {
+      return (int) ((LocalDate) integer).toEpochDay();
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for int field. Type name: " + integer.getClass().getName());
+    }
+  }
+
+  @Override
+  public long getLong(int pos) {
+    Object longVal = struct.get(pos, Object.class);
+
+    if (longVal instanceof Long) {
+      return (long) longVal;
+    } else if (longVal instanceof OffsetDateTime) {
+      return Duration.between(Instant.EPOCH, (OffsetDateTime) longVal).toNanos() / 1000;
+    } else if (longVal instanceof LocalDate) {
+      return ((LocalDate) longVal).toEpochDay();
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for long field. Type name: " + longVal.getClass().getName());
+    }
+  }
+
+  @Override
+  public float getFloat(int pos) {
+    return struct.get(pos, Float.class);
+  }
+
+  @Override
+  public double getDouble(int pos) {
+    return struct.get(pos, Double.class);
+  }
+
+  @Override
+  public StringData getString(int pos) {
+    return isNullAt(pos) ? null : getStringDataInternal(pos);
+  }
+
+  private StringData getStringDataInternal(int pos) {
+    CharSequence seq = struct.get(pos, CharSequence.class);
+    return StringData.fromString(seq.toString());
+  }
+
+  @Override
+  public DecimalData getDecimal(int pos, int precision, int scale) {
+    return isNullAt(pos)
+        ? null
+        : DecimalData.fromBigDecimal(getDecimalInternal(pos), precision, scale);
+  }
+
+  private BigDecimal getDecimalInternal(int pos) {
+    return struct.get(pos, BigDecimal.class);
+  }
+
+  @Override
+  public TimestampData getTimestamp(int pos, int precision) {
+    long timeLong = getLong(pos);
+    return TimestampData.fromEpochMillis(timeLong / 1000, (int) (timeLong % 1000) * 1000);
+  }
+
+  @Override
+  public <T> RawValueData<T> getRawValue(int pos) {
+    throw new UnsupportedOperationException("Not supported yet.");
+  }
+
+  @Override
+  public byte[] getBinary(int pos) {
+    return isNullAt(pos) ? null : getBinaryInternal(pos);
+  }
+
+  private byte[] getBinaryInternal(int pos) {
+    Object bytes = struct.get(pos, Object.class);
+
+    // should only be either ByteBuffer or byte[]
+    if (bytes instanceof ByteBuffer) {
+      return ByteBuffers.toByteArray((ByteBuffer) bytes);
+    } else if (bytes instanceof byte[]) {
+      return (byte[]) bytes;
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for binary field. Type name: " + bytes.getClass().getName());
+    }
+  }
+
+  @Override
+  public ArrayData getArray(int pos) {
+    return isNullAt(pos)
+        ? null
+        : (ArrayData)
+            covertFromStructToRowData(
+                type.fields().get(pos).type().asListType(), struct.get(pos, List.class));
+  }
+
+  @Override
+  public MapData getMap(int pos) {
+    return isNullAt(pos)
+        ? null
+        : (MapData)
+            covertFromStructToRowData(
+                type.fields().get(pos).type().asMapType(), struct.get(pos, Map.class));
+  }
+
+  @Override
+  public RowData getRow(int pos, int numFields) {
+    return isNullAt(pos) ? null : getStructRowData(pos, numFields);
+  }
+
+  private StructRowData getStructRowData(int pos, int numFields) {
+    return new StructRowData(
+        type.fields().get(pos).type().asStructType(), struct.get(pos, StructLike.class));
+  }
+
+  private Object covertFromStructToRowData(Type elementType, Object value) {
+    switch (elementType.typeId()) {
+      case BOOLEAN:
+      case INTEGER:
+      case DATE:
+      case TIME:
+      case LONG:
+      case FLOAT:
+      case DOUBLE:
+      case DECIMAL:
+        return value;
+      case TIMESTAMP:
+        return TimestampData.fromEpochMillis(
+            (Long) value / 1000, (int) ((Long) value % 1000) * 1000);
+      case STRING:
+        return StringData.fromString(value.toString());
+      case FIXED:
+      case BINARY:
+        return ByteBuffers.toByteArray((ByteBuffer) value);
+      case STRUCT:
+        return new StructRowData(elementType.asStructType(), (StructLike) value);
+      case LIST:
+        List<?> list = (List<?>) value;
+        Object[] array = new Object[list.size()];

Review Comment:
   I tried it, please review it. But I think, even if wo use `T` here, eventually java will erase it. Is it necessary? 
   Thx.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] hililiwei commented on pull request #6222: Flink: Support inspecting table

Posted by GitBox <gi...@apache.org>.
hililiwei commented on PR #6222:
URL: https://github.com/apache/iceberg/pull/6222#issuecomment-1383338686

   thanks @stevenzwu @pvary @szehon-ho 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] hililiwei commented on a diff in pull request #6222: Flink: Support inspecting table

Posted by GitBox <gi...@apache.org>.
hililiwei commented on code in PR #6222:
URL: https://github.com/apache/iceberg/pull/6222#discussion_r1066600385


##########
core/src/main/java/org/apache/iceberg/BaseFilesTable.java:
##########
@@ -223,34 +226,37 @@ ManifestFile manifest() {
   static class ContentFileStructWithMetrics implements StructLike {
     private final StructLike fileAsStruct;
     private final MetricsUtil.ReadableMetricsStruct readableMetrics;
-    private final int expectedSize;
+    private final int columnCount;
+    private final int metricsPosition;
 
     ContentFileStructWithMetrics(
-        int expectedSize,
+        int columnCount,
+        int metricsPosition,
         StructLike fileAsStruct,
         MetricsUtil.ReadableMetricsStruct readableMetrics) {
       this.fileAsStruct = fileAsStruct;
       this.readableMetrics = readableMetrics;
-      this.expectedSize = expectedSize;
+      this.columnCount = columnCount;
+      this.metricsPosition = metricsPosition;
     }
 
     @Override
     public int size() {
-      return expectedSize;
+      return columnCount;
     }
 
     @Override
     public <T> T get(int pos, Class<T> javaClass) {
-      int lastExpectedIndex = expectedSize - 1;
-      if (pos < lastExpectedIndex) {
+      if (pos < metricsPosition) {
         return fileAsStruct.get(pos, javaClass);
-      } else if (pos == lastExpectedIndex) {
+      } else if (pos == metricsPosition) {
         return javaClass.cast(readableMetrics);
       } else {
-        throw new IllegalArgumentException(
-            String.format(
-                "Illegal position access for ContentFileStructWithMetrics: %d, max allowed is %d",
-                pos, lastExpectedIndex));
+        // columnCount = fileAsStruct column count + the readable metrics field.
+        // When pos is greater than metricsPosition, the actual position of the field in
+        // fileAsStruct should be
+        // subtracted by 1.

Review Comment:
   done



##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java:
##########
@@ -149,15 +151,28 @@ public Catalog catalog() {
     return icebergCatalog;
   }
 
-  private Namespace toNamespace(String database) {
+  /** Append a new level to the base namespace */
+  private static Namespace appendLevel(Namespace baseNamespace, String newLevel) {
     String[] namespace = new String[baseNamespace.levels().length + 1];
     System.arraycopy(baseNamespace.levels(), 0, namespace, 0, baseNamespace.levels().length);
-    namespace[baseNamespace.levels().length] = database;
+    namespace[baseNamespace.levels().length] = newLevel;
     return Namespace.of(namespace);
   }
 
   TableIdentifier toIdentifier(ObjectPath path) {
-    return TableIdentifier.of(toNamespace(path.getDatabaseName()), path.getObjectName());
+    String objectName = path.getObjectName();
+    List<String> tableName = Splitter.on('$').splitToList(objectName);
+
+    if (tableName.size() == 1) {
+      return TableIdentifier.of(
+          appendLevel(baseNamespace, path.getDatabaseName()), path.getObjectName());
+    } else if (tableName.size() == 2 && MetadataTableType.from(tableName.get(1)) != null) {
+      return TableIdentifier.of(
+          appendLevel(appendLevel(baseNamespace, path.getDatabaseName()), tableName.get(0)),
+          tableName.get(1));
+    } else {
+      throw new IllegalArgumentException("Illegal table name");

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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] stevenzwu commented on a diff in pull request #6222: Flink: Support inspecting table

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on code in PR #6222:
URL: https://github.com/apache/iceberg/pull/6222#discussion_r1042777490


##########
flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java:
##########
@@ -295,6 +299,161 @@ private static void assertEquals(
     }
   }
 
+  public static void assertEqualsSafe(
+      Schema schema, List<GenericData.Record> recs, List<Row> rows) {

Review Comment:
   ok. these new `assertEqualsSafe` methods seem only used inside this class. should they be private?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] szehon-ho commented on a diff in pull request #6222: Flink: Support inspecting table

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on code in PR #6222:
URL: https://github.com/apache/iceberg/pull/6222#discussion_r1042568978


##########
core/src/main/java/org/apache/iceberg/BaseFilesTable.java:
##########
@@ -223,34 +225,28 @@ ManifestFile manifest() {
   static class ContentFileStructWithMetrics implements StructLike {
     private final StructLike fileAsStruct;
     private final MetricsUtil.ReadableMetricsStruct readableMetrics;
-    private final int expectedSize;
+    private final int position;

Review Comment:
   Makes sense 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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] hililiwei commented on a diff in pull request #6222: Flink: Support inspecting table

Posted by GitBox <gi...@apache.org>.
hililiwei commented on code in PR #6222:
URL: https://github.com/apache/iceberg/pull/6222#discussion_r1043413775


##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java:
##########
@@ -0,0 +1,340 @@
+/*
+ * 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.Duration;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.OffsetDateTime;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.function.BiConsumer;
+import java.util.function.Function;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericMapData;
+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.flink.types.RowKind;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+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.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.ByteBuffers;
+
+public class StructRowData implements RowData {
+  private final Types.StructType type;
+  private RowKind kind;
+  private StructLike struct;
+
+  public StructRowData(Types.StructType type) {
+    this(type, RowKind.INSERT);
+  }
+
+  public StructRowData(Types.StructType type, RowKind kind) {
+    this(type, null, kind);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct) {
+    this(type, struct, RowKind.INSERT);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct, RowKind kind) {
+    this.type = type;
+    this.struct = struct;
+    this.kind = kind;
+  }
+
+  public StructRowData setStruct(StructLike newStruct) {
+    this.struct = newStruct;
+    return this;
+  }
+
+  @Override
+  public int getArity() {
+    return struct.size();
+  }
+
+  @Override
+  public RowKind getRowKind() {
+    return kind;
+  }
+
+  @Override
+  public void setRowKind(RowKind newKind) {
+    Preconditions.checkNotNull(newKind, "kind can not be null");
+    this.kind = newKind;
+  }
+
+  @Override
+  public boolean isNullAt(int pos) {
+    return struct.get(pos, Object.class) == null;
+  }
+
+  @Override
+  public boolean getBoolean(int pos) {
+    return struct.get(pos, Boolean.class);
+  }
+
+  @Override
+  public byte getByte(int pos) {
+    return (byte) (int) struct.get(pos, Integer.class);
+  }
+
+  @Override
+  public short getShort(int pos) {
+    return (short) (int) struct.get(pos, Integer.class);
+  }
+
+  @Override
+  public int getInt(int pos) {
+    Object integer = struct.get(pos, Object.class);
+
+    if (integer instanceof Integer) {
+      return (int) integer;
+    } else if (integer instanceof LocalDate) {
+      return (int) ((LocalDate) integer).toEpochDay();
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for int field. Type name: " + integer.getClass().getName());
+    }
+  }
+
+  @Override
+  public long getLong(int pos) {
+    Object longVal = struct.get(pos, Object.class);
+
+    if (longVal instanceof Long) {
+      return (long) longVal;
+    } else if (longVal instanceof OffsetDateTime) {
+      return Duration.between(Instant.EPOCH, (OffsetDateTime) longVal).toNanos() / 1000;
+    } else if (longVal instanceof LocalDate) {
+      return ((LocalDate) longVal).toEpochDay();
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for long field. Type name: " + longVal.getClass().getName());
+    }
+  }
+
+  @Override
+  public float getFloat(int pos) {
+    return struct.get(pos, Float.class);
+  }
+
+  @Override
+  public double getDouble(int pos) {
+    return struct.get(pos, Double.class);
+  }
+
+  @Override
+  public StringData getString(int pos) {
+    return isNullAt(pos) ? null : getStringDataInternal(pos);
+  }
+
+  private StringData getStringDataInternal(int pos) {
+    CharSequence seq = struct.get(pos, CharSequence.class);
+    return StringData.fromString(seq.toString());
+  }
+
+  @Override
+  public DecimalData getDecimal(int pos, int precision, int scale) {
+    return isNullAt(pos)
+        ? null
+        : DecimalData.fromBigDecimal(getDecimalInternal(pos), precision, scale);
+  }
+
+  private BigDecimal getDecimalInternal(int pos) {
+    return struct.get(pos, BigDecimal.class);
+  }
+
+  @Override
+  public TimestampData getTimestamp(int pos, int precision) {
+    long timeLong = getLong(pos);

Review Comment:
   Thank you for your patience.
   It is only the Iceberg Timestamp type.
   
   
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] stevenzwu commented on a diff in pull request #6222: Flink: Support inspecting table

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on code in PR #6222:
URL: https://github.com/apache/iceberg/pull/6222#discussion_r1042794414


##########
core/src/main/java/org/apache/iceberg/BaseFilesTable.java:
##########
@@ -177,14 +177,17 @@ public CloseableIterable<StructLike> rows() {
         Set<Integer> readableMetricsIds = TypeUtil.getProjectedIds(readableMetricsField.type());
         Schema fileProjection = TypeUtil.selectNot(projection, readableMetricsIds);
 
+        int position = projection.columns().indexOf(readableMetricsField);

Review Comment:
   ah. didn't see the if-else before 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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] stevenzwu commented on a diff in pull request #6222: Flink: Support inspecting table

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on code in PR #6222:
URL: https://github.com/apache/iceberg/pull/6222#discussion_r1043944439


##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java:
##########
@@ -0,0 +1,286 @@
+/*
+ * 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.Duration;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.OffsetDateTime;
+import java.util.List;
+import java.util.Map;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericMapData;
+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.flink.types.RowKind;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.ByteBuffers;
+
+@Internal
+public class StructRowData implements RowData {
+  private final Types.StructType type;
+  private RowKind kind;
+  private StructLike struct;
+
+  public StructRowData(Types.StructType type) {
+    this(type, RowKind.INSERT);
+  }
+
+  public StructRowData(Types.StructType type, RowKind kind) {
+    this(type, null, kind);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct) {
+    this(type, struct, RowKind.INSERT);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct, RowKind kind) {
+    this.type = type;
+    this.struct = struct;
+    this.kind = kind;
+  }
+
+  public StructRowData setStruct(StructLike newStruct) {
+    this.struct = newStruct;
+    return this;
+  }
+
+  @Override
+  public int getArity() {
+    return struct.size();
+  }
+
+  @Override
+  public RowKind getRowKind() {
+    return kind;
+  }
+
+  @Override
+  public void setRowKind(RowKind newKind) {
+    Preconditions.checkNotNull(newKind, "kind can not be null");
+    this.kind = newKind;
+  }
+
+  @Override
+  public boolean isNullAt(int pos) {
+    return struct.get(pos, Object.class) == null;
+  }
+
+  @Override
+  public boolean getBoolean(int pos) {
+    return struct.get(pos, Boolean.class);
+  }
+
+  @Override
+  public byte getByte(int pos) {
+    return (byte) (int) struct.get(pos, Integer.class);
+  }
+
+  @Override
+  public short getShort(int pos) {
+    return (short) (int) struct.get(pos, Integer.class);
+  }
+
+  @Override
+  public int getInt(int pos) {
+    Object integer = struct.get(pos, Object.class);
+
+    if (integer instanceof Integer) {
+      return (int) integer;
+    } else if (integer instanceof LocalDate) {
+      return (int) ((LocalDate) integer).toEpochDay();
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for int field. Type name: " + integer.getClass().getName());
+    }
+  }
+
+  @Override
+  public long getLong(int pos) {
+    Object longVal = struct.get(pos, Object.class);
+
+    if (longVal instanceof Long) {
+      return (long) longVal;
+    } else if (longVal instanceof OffsetDateTime) {
+      return Duration.between(Instant.EPOCH, (OffsetDateTime) longVal).toNanos() / 1000;
+    } else if (longVal instanceof LocalDate) {
+      return ((LocalDate) longVal).toEpochDay();
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for long field. Type name: " + longVal.getClass().getName());
+    }
+  }
+
+  @Override
+  public float getFloat(int pos) {
+    return struct.get(pos, Float.class);
+  }
+
+  @Override
+  public double getDouble(int pos) {
+    return struct.get(pos, Double.class);
+  }
+
+  @Override
+  public StringData getString(int pos) {
+    return isNullAt(pos) ? null : getStringDataInternal(pos);
+  }
+
+  private StringData getStringDataInternal(int pos) {
+    CharSequence seq = struct.get(pos, CharSequence.class);
+    return StringData.fromString(seq.toString());
+  }
+
+  @Override
+  public DecimalData getDecimal(int pos, int precision, int scale) {
+    return isNullAt(pos)
+        ? null
+        : DecimalData.fromBigDecimal(getDecimalInternal(pos), precision, scale);
+  }
+
+  private BigDecimal getDecimalInternal(int pos) {
+    return struct.get(pos, BigDecimal.class);
+  }
+
+  @Override
+  public TimestampData getTimestamp(int pos, int precision) {
+    long timeLong = getLong(pos);
+    return TimestampData.fromEpochMillis(timeLong / 1000, (int) (timeLong % 1000) * 1000);
+  }
+
+  @Override
+  public <T> RawValueData<T> getRawValue(int pos) {
+    throw new UnsupportedOperationException("Not supported yet.");
+  }
+
+  @Override
+  public byte[] getBinary(int pos) {
+    return isNullAt(pos) ? null : getBinaryInternal(pos);
+  }
+
+  private byte[] getBinaryInternal(int pos) {
+    Object bytes = struct.get(pos, Object.class);
+
+    // should only be either ByteBuffer or byte[]
+    if (bytes instanceof ByteBuffer) {
+      return ByteBuffers.toByteArray((ByteBuffer) bytes);
+    } else if (bytes instanceof byte[]) {
+      return (byte[]) bytes;
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for binary field. Type name: " + bytes.getClass().getName());
+    }
+  }
+
+  @Override
+  public ArrayData getArray(int pos) {
+    return isNullAt(pos)
+        ? null
+        : (ArrayData)
+            covertFromStructToRowData(
+                type.fields().get(pos).type().asListType(), struct.get(pos, List.class));
+  }
+
+  @Override
+  public MapData getMap(int pos) {
+    return isNullAt(pos)
+        ? null
+        : (MapData)
+            covertFromStructToRowData(
+                type.fields().get(pos).type().asMapType(), struct.get(pos, Map.class));
+  }
+
+  @Override
+  public RowData getRow(int pos, int numFields) {
+    return isNullAt(pos) ? null : getStructRowData(pos, numFields);
+  }
+
+  private StructRowData getStructRowData(int pos, int numFields) {
+    return new StructRowData(
+        type.fields().get(pos).type().asStructType(), struct.get(pos, StructLike.class));
+  }
+
+  private Object covertFromStructToRowData(Type elementType, Object value) {

Review Comment:
   this is not actually converting from struct to RowData. It is `convertingIcebergValueToFlinkRowDataValue` or maybe simpler form as `convertValue`



##########
api/src/main/java/org/apache/iceberg/types/ReassignDoc.java:
##########
@@ -0,0 +1,104 @@
+/*
+ * 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.types;
+
+import java.util.List;
+import java.util.function.Supplier;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+public class ReassignDoc extends TypeUtil.CustomOrderSchemaVisitor<Type> {
+  private final Schema docSourceSchema;
+
+  public ReassignDoc(Schema docSourceSchema) {
+    this.docSourceSchema = docSourceSchema;
+  }
+
+  @Override
+  public Type schema(Schema schema, Supplier<Type> future) {
+    return future.get();
+  }
+
+  @Override
+  public Type struct(Types.StructType struct, Iterable<Type> fieldTypes) {
+    List<Types.NestedField> fields = struct.fields();
+    int length = fields.size();
+
+    List<Type> types = Lists.newArrayList(fieldTypes);
+    List<Types.NestedField> newFields = Lists.newArrayListWithExpectedSize(length);
+    for (int i = 0; i < length; i += 1) {
+      Types.NestedField field = fields.get(i);
+      int fieldId = field.fieldId();
+      Types.NestedField docField = docSourceSchema.findField(fieldId);
+
+      if (docField == null) {

Review Comment:
   I think Iceberg coding style is to use `Preconditions.check` for cases like this.



##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java:
##########
@@ -0,0 +1,286 @@
+/*
+ * 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.Duration;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.OffsetDateTime;
+import java.util.List;
+import java.util.Map;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericMapData;
+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.flink.types.RowKind;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.ByteBuffers;
+
+@Internal
+public class StructRowData implements RowData {
+  private final Types.StructType type;
+  private RowKind kind;
+  private StructLike struct;
+
+  public StructRowData(Types.StructType type) {
+    this(type, RowKind.INSERT);
+  }
+
+  public StructRowData(Types.StructType type, RowKind kind) {
+    this(type, null, kind);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct) {
+    this(type, struct, RowKind.INSERT);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct, RowKind kind) {
+    this.type = type;
+    this.struct = struct;
+    this.kind = kind;
+  }
+
+  public StructRowData setStruct(StructLike newStruct) {
+    this.struct = newStruct;
+    return this;
+  }
+
+  @Override
+  public int getArity() {
+    return struct.size();
+  }
+
+  @Override
+  public RowKind getRowKind() {
+    return kind;
+  }
+
+  @Override
+  public void setRowKind(RowKind newKind) {
+    Preconditions.checkNotNull(newKind, "kind can not be null");
+    this.kind = newKind;
+  }
+
+  @Override
+  public boolean isNullAt(int pos) {
+    return struct.get(pos, Object.class) == null;
+  }
+
+  @Override
+  public boolean getBoolean(int pos) {
+    return struct.get(pos, Boolean.class);
+  }
+
+  @Override
+  public byte getByte(int pos) {
+    return (byte) (int) struct.get(pos, Integer.class);
+  }
+
+  @Override
+  public short getShort(int pos) {
+    return (short) (int) struct.get(pos, Integer.class);
+  }
+
+  @Override
+  public int getInt(int pos) {
+    Object integer = struct.get(pos, Object.class);
+
+    if (integer instanceof Integer) {
+      return (int) integer;
+    } else if (integer instanceof LocalDate) {
+      return (int) ((LocalDate) integer).toEpochDay();
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for int field. Type name: " + integer.getClass().getName());
+    }
+  }
+
+  @Override
+  public long getLong(int pos) {
+    Object longVal = struct.get(pos, Object.class);
+
+    if (longVal instanceof Long) {
+      return (long) longVal;
+    } else if (longVal instanceof OffsetDateTime) {
+      return Duration.between(Instant.EPOCH, (OffsetDateTime) longVal).toNanos() / 1000;
+    } else if (longVal instanceof LocalDate) {
+      return ((LocalDate) longVal).toEpochDay();
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for long field. Type name: " + longVal.getClass().getName());
+    }
+  }
+
+  @Override
+  public float getFloat(int pos) {
+    return struct.get(pos, Float.class);
+  }
+
+  @Override
+  public double getDouble(int pos) {
+    return struct.get(pos, Double.class);
+  }
+
+  @Override
+  public StringData getString(int pos) {
+    return isNullAt(pos) ? null : getStringDataInternal(pos);
+  }
+
+  private StringData getStringDataInternal(int pos) {
+    CharSequence seq = struct.get(pos, CharSequence.class);
+    return StringData.fromString(seq.toString());
+  }
+
+  @Override
+  public DecimalData getDecimal(int pos, int precision, int scale) {
+    return isNullAt(pos)
+        ? null
+        : DecimalData.fromBigDecimal(getDecimalInternal(pos), precision, scale);
+  }
+
+  private BigDecimal getDecimalInternal(int pos) {
+    return struct.get(pos, BigDecimal.class);
+  }
+
+  @Override
+  public TimestampData getTimestamp(int pos, int precision) {
+    long timeLong = getLong(pos);
+    return TimestampData.fromEpochMillis(timeLong / 1000, (int) (timeLong % 1000) * 1000);
+  }
+
+  @Override
+  public <T> RawValueData<T> getRawValue(int pos) {
+    throw new UnsupportedOperationException("Not supported yet.");
+  }
+
+  @Override
+  public byte[] getBinary(int pos) {
+    return isNullAt(pos) ? null : getBinaryInternal(pos);
+  }
+
+  private byte[] getBinaryInternal(int pos) {
+    Object bytes = struct.get(pos, Object.class);
+
+    // should only be either ByteBuffer or byte[]
+    if (bytes instanceof ByteBuffer) {
+      return ByteBuffers.toByteArray((ByteBuffer) bytes);
+    } else if (bytes instanceof byte[]) {
+      return (byte[]) bytes;
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for binary field. Type name: " + bytes.getClass().getName());
+    }
+  }
+
+  @Override
+  public ArrayData getArray(int pos) {
+    return isNullAt(pos)
+        ? null
+        : (ArrayData)
+            covertFromStructToRowData(
+                type.fields().get(pos).type().asListType(), struct.get(pos, List.class));
+  }
+
+  @Override
+  public MapData getMap(int pos) {
+    return isNullAt(pos)
+        ? null
+        : (MapData)
+            covertFromStructToRowData(
+                type.fields().get(pos).type().asMapType(), struct.get(pos, Map.class));
+  }
+
+  @Override
+  public RowData getRow(int pos, int numFields) {
+    return isNullAt(pos) ? null : getStructRowData(pos, numFields);
+  }
+
+  private StructRowData getStructRowData(int pos, int numFields) {
+    return new StructRowData(
+        type.fields().get(pos).type().asStructType(), struct.get(pos, StructLike.class));
+  }
+
+  private Object covertFromStructToRowData(Type elementType, Object value) {
+    switch (elementType.typeId()) {
+      case BOOLEAN:
+      case INTEGER:
+      case DATE:
+      case TIME:
+      case LONG:
+      case FLOAT:
+      case DOUBLE:
+      case DECIMAL:
+        return value;
+      case TIMESTAMP:
+        return TimestampData.fromEpochMillis(
+            (Long) value / 1000, (int) ((Long) value % 1000) * 1000);

Review Comment:
   nit: `Long` to `long` to avoid extra boxing
   
   Might be a little easier to read if it is broken down.
   ```
   // Iceberg timestamp field has micro second precision
   long millisecond = (long) value / 1000;
   int nanoOfMillisecond =  (int) ((Long) value % 1000) * 1000;
   return TimestampData.fromEpochMillis(millisecond, nanoOfMillisecond);
   ```



##########
flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java:
##########
@@ -295,6 +299,161 @@ private static void assertEquals(
     }
   }
 
+  public static void assertEqualsSafe(

Review Comment:
   what does `Safe` mean here? should they just be called `assertEquals`?
   
   nit: Iceberg coding style probably would name `recs` as full names `records`



##########
api/src/main/java/org/apache/iceberg/types/TypeUtil.java:
##########
@@ -299,6 +299,28 @@ public static Schema reassignIds(Schema schema, Schema idSourceSchema) {
     return reassignIds(schema, idSourceSchema, true);
   }
 
+  /**
+   * Reassigns doc in a schema from another schema.
+   *
+   * <p>Doc are determined by field id. If a field in the schema cannot be found in the source
+   * schema, this will throw IllegalArgumentException.
+   *
+   * <p>This will not alter a schema's structure, nullability, or types.
+   *
+   * @param schema the schema to have doc reassigned
+   * @param docSourceSchema the schema from which field doc will be used
+   * @return an structurally identical schema with field ids matching the source schema
+   * @throws IllegalArgumentException if a field cannot be found (by id) in the source schema
+   */
+  public static Schema reassignDoc(Schema schema, Schema docSourceSchema) {

Review Comment:
   like to get more feedback for adding a new public method in the api module. cc @rdblue @aokolnychyi @RussellSpitzer @szehon-ho 
   
   Here is the context why we need this. The converter in `FlinkSchemaUtil` creates Iceberg `Schema` without doc attributes. Hence @hililiwei implemented the util class and method to carry over the doc attribute from `docSourceSchema` to `schema` (converted from Flink `TableSchema`).
   ```
     public static Schema convert(TableSchema schema) {
     }
   ```
   
   I suggested moving it into `TypeUtil` class in api module as it is very similar to the `reassignIds` method in the `TypeUtil` class.



##########
api/src/main/java/org/apache/iceberg/types/ReassignDoc.java:
##########
@@ -0,0 +1,104 @@
+/*
+ * 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.types;
+
+import java.util.List;
+import java.util.function.Supplier;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+public class ReassignDoc extends TypeUtil.CustomOrderSchemaVisitor<Type> {

Review Comment:
   this doesn't need to be public



##########
flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java:
##########
@@ -295,6 +299,161 @@ private static void assertEquals(
     }
   }
 
+  public static void assertEqualsSafe(
+      Schema schema, List<GenericData.Record> recs, List<Row> rows) {
+    Streams.forEachPair(
+        recs.stream(), rows.stream(), (rec, row) -> assertEqualsSafe(schema, rec, row));
+  }
+
+  public static void assertEqualsSafe(Schema schema, GenericData.Record rec, Row row) {
+    List<Types.NestedField> fields = schema.asStruct().fields();
+    RowType rowType = FlinkSchemaUtil.convert(schema);
+    for (int i = 0; i < fields.size(); i += 1) {
+      Type fieldType = fields.get(i).type();
+      Object expectedValue = rec.get(i);
+      Object actualValue = row.getField(i);
+      LogicalType logicalType = rowType.getTypeAt(i);
+      assertAvroEquals(fieldType, logicalType, expectedValue, actualValue);
+    }
+  }
+
+  private static void assertEqualsSafe(Types.StructType struct, GenericData.Record rec, Row row) {
+    List<Types.NestedField> fields = struct.fields();
+    for (int i = 0; i < fields.size(); i += 1) {
+      Type fieldType = fields.get(i).type();
+      Object expectedValue = rec.get(i);
+      Object actualValue = row.getField(i);
+      assertAvroEquals(fieldType, null, expectedValue, actualValue);
+    }
+  }
+
+  private static void assertAvroEquals(
+      Type type, LogicalType logicalType, Object expected, Object actual) {
+
+    if (expected == null && actual == null) {
+      return;
+    }
+
+    Assert.assertTrue(
+        "expected and actual should be both null or not null", expected != null && actual != null);
+
+    switch (type.typeId()) {
+      case BOOLEAN:

Review Comment:
   nit: seems that some primitive types can be collapsed without repetitions



##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java:
##########
@@ -0,0 +1,286 @@
+/*
+ * 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.Duration;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.OffsetDateTime;
+import java.util.List;
+import java.util.Map;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericMapData;
+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.flink.types.RowKind;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.ByteBuffers;
+
+@Internal
+public class StructRowData implements RowData {
+  private final Types.StructType type;
+  private RowKind kind;
+  private StructLike struct;
+
+  public StructRowData(Types.StructType type) {
+    this(type, RowKind.INSERT);
+  }
+
+  public StructRowData(Types.StructType type, RowKind kind) {
+    this(type, null, kind);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct) {
+    this(type, struct, RowKind.INSERT);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct, RowKind kind) {
+    this.type = type;
+    this.struct = struct;
+    this.kind = kind;
+  }
+
+  public StructRowData setStruct(StructLike newStruct) {
+    this.struct = newStruct;
+    return this;
+  }
+
+  @Override
+  public int getArity() {
+    return struct.size();
+  }
+
+  @Override
+  public RowKind getRowKind() {
+    return kind;
+  }
+
+  @Override
+  public void setRowKind(RowKind newKind) {
+    Preconditions.checkNotNull(newKind, "kind can not be null");
+    this.kind = newKind;
+  }
+
+  @Override
+  public boolean isNullAt(int pos) {
+    return struct.get(pos, Object.class) == null;
+  }
+
+  @Override
+  public boolean getBoolean(int pos) {
+    return struct.get(pos, Boolean.class);
+  }
+
+  @Override
+  public byte getByte(int pos) {
+    return (byte) (int) struct.get(pos, Integer.class);
+  }
+
+  @Override
+  public short getShort(int pos) {
+    return (short) (int) struct.get(pos, Integer.class);
+  }
+
+  @Override
+  public int getInt(int pos) {
+    Object integer = struct.get(pos, Object.class);
+
+    if (integer instanceof Integer) {
+      return (int) integer;
+    } else if (integer instanceof LocalDate) {
+      return (int) ((LocalDate) integer).toEpochDay();
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for int field. Type name: " + integer.getClass().getName());
+    }
+  }
+
+  @Override
+  public long getLong(int pos) {
+    Object longVal = struct.get(pos, Object.class);
+
+    if (longVal instanceof Long) {
+      return (long) longVal;
+    } else if (longVal instanceof OffsetDateTime) {
+      return Duration.between(Instant.EPOCH, (OffsetDateTime) longVal).toNanos() / 1000;
+    } else if (longVal instanceof LocalDate) {
+      return ((LocalDate) longVal).toEpochDay();
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for long field. Type name: " + longVal.getClass().getName());
+    }
+  }
+
+  @Override
+  public float getFloat(int pos) {
+    return struct.get(pos, Float.class);
+  }
+
+  @Override
+  public double getDouble(int pos) {
+    return struct.get(pos, Double.class);
+  }
+
+  @Override
+  public StringData getString(int pos) {
+    return isNullAt(pos) ? null : getStringDataInternal(pos);
+  }
+
+  private StringData getStringDataInternal(int pos) {
+    CharSequence seq = struct.get(pos, CharSequence.class);
+    return StringData.fromString(seq.toString());
+  }
+
+  @Override
+  public DecimalData getDecimal(int pos, int precision, int scale) {
+    return isNullAt(pos)
+        ? null
+        : DecimalData.fromBigDecimal(getDecimalInternal(pos), precision, scale);
+  }
+
+  private BigDecimal getDecimalInternal(int pos) {
+    return struct.get(pos, BigDecimal.class);
+  }
+
+  @Override
+  public TimestampData getTimestamp(int pos, int precision) {
+    long timeLong = getLong(pos);
+    return TimestampData.fromEpochMillis(timeLong / 1000, (int) (timeLong % 1000) * 1000);
+  }
+
+  @Override
+  public <T> RawValueData<T> getRawValue(int pos) {
+    throw new UnsupportedOperationException("Not supported yet.");
+  }
+
+  @Override
+  public byte[] getBinary(int pos) {
+    return isNullAt(pos) ? null : getBinaryInternal(pos);
+  }
+
+  private byte[] getBinaryInternal(int pos) {
+    Object bytes = struct.get(pos, Object.class);
+
+    // should only be either ByteBuffer or byte[]
+    if (bytes instanceof ByteBuffer) {
+      return ByteBuffers.toByteArray((ByteBuffer) bytes);
+    } else if (bytes instanceof byte[]) {
+      return (byte[]) bytes;
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for binary field. Type name: " + bytes.getClass().getName());
+    }
+  }
+
+  @Override
+  public ArrayData getArray(int pos) {
+    return isNullAt(pos)
+        ? null
+        : (ArrayData)
+            covertFromStructToRowData(
+                type.fields().get(pos).type().asListType(), struct.get(pos, List.class));
+  }
+
+  @Override
+  public MapData getMap(int pos) {
+    return isNullAt(pos)
+        ? null
+        : (MapData)
+            covertFromStructToRowData(
+                type.fields().get(pos).type().asMapType(), struct.get(pos, Map.class));
+  }
+
+  @Override
+  public RowData getRow(int pos, int numFields) {
+    return isNullAt(pos) ? null : getStructRowData(pos, numFields);
+  }
+
+  private StructRowData getStructRowData(int pos, int numFields) {
+    return new StructRowData(
+        type.fields().get(pos).type().asStructType(), struct.get(pos, StructLike.class));
+  }
+
+  private Object covertFromStructToRowData(Type elementType, Object value) {
+    switch (elementType.typeId()) {
+      case BOOLEAN:
+      case INTEGER:
+      case DATE:
+      case TIME:
+      case LONG:
+      case FLOAT:
+      case DOUBLE:
+      case DECIMAL:
+        return value;
+      case TIMESTAMP:
+        return TimestampData.fromEpochMillis(
+            (Long) value / 1000, (int) ((Long) value % 1000) * 1000);
+      case STRING:
+        return StringData.fromString(value.toString());
+      case FIXED:
+      case BINARY:
+        return ByteBuffers.toByteArray((ByteBuffer) value);
+      case STRUCT:
+        return new StructRowData(elementType.asStructType(), (StructLike) value);
+      case LIST:
+        List<?> list = (List<?>) value;
+        Object[] array = new Object[list.size()];

Review Comment:
   technically, since we know the list element type, we can build the accurate array as `T[]`, if the list element type is `T`



##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java:
##########
@@ -0,0 +1,286 @@
+/*
+ * 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.Duration;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.OffsetDateTime;
+import java.util.List;
+import java.util.Map;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericMapData;
+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.flink.types.RowKind;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.ByteBuffers;
+
+@Internal
+public class StructRowData implements RowData {
+  private final Types.StructType type;
+  private RowKind kind;
+  private StructLike struct;
+
+  public StructRowData(Types.StructType type) {
+    this(type, RowKind.INSERT);
+  }
+
+  public StructRowData(Types.StructType type, RowKind kind) {
+    this(type, null, kind);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct) {
+    this(type, struct, RowKind.INSERT);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct, RowKind kind) {
+    this.type = type;
+    this.struct = struct;
+    this.kind = kind;
+  }
+
+  public StructRowData setStruct(StructLike newStruct) {
+    this.struct = newStruct;
+    return this;
+  }
+
+  @Override
+  public int getArity() {
+    return struct.size();
+  }
+
+  @Override
+  public RowKind getRowKind() {
+    return kind;
+  }
+
+  @Override
+  public void setRowKind(RowKind newKind) {
+    Preconditions.checkNotNull(newKind, "kind can not be null");
+    this.kind = newKind;
+  }
+
+  @Override
+  public boolean isNullAt(int pos) {
+    return struct.get(pos, Object.class) == null;
+  }
+
+  @Override
+  public boolean getBoolean(int pos) {
+    return struct.get(pos, Boolean.class);
+  }
+
+  @Override
+  public byte getByte(int pos) {
+    return (byte) (int) struct.get(pos, Integer.class);
+  }
+
+  @Override
+  public short getShort(int pos) {
+    return (short) (int) struct.get(pos, Integer.class);
+  }
+
+  @Override
+  public int getInt(int pos) {
+    Object integer = struct.get(pos, Object.class);
+
+    if (integer instanceof Integer) {
+      return (int) integer;
+    } else if (integer instanceof LocalDate) {
+      return (int) ((LocalDate) integer).toEpochDay();
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for int field. Type name: " + integer.getClass().getName());
+    }
+  }
+
+  @Override
+  public long getLong(int pos) {
+    Object longVal = struct.get(pos, Object.class);
+
+    if (longVal instanceof Long) {
+      return (long) longVal;
+    } else if (longVal instanceof OffsetDateTime) {
+      return Duration.between(Instant.EPOCH, (OffsetDateTime) longVal).toNanos() / 1000;
+    } else if (longVal instanceof LocalDate) {
+      return ((LocalDate) longVal).toEpochDay();
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for long field. Type name: " + longVal.getClass().getName());
+    }
+  }
+
+  @Override
+  public float getFloat(int pos) {
+    return struct.get(pos, Float.class);
+  }
+
+  @Override
+  public double getDouble(int pos) {
+    return struct.get(pos, Double.class);
+  }
+
+  @Override
+  public StringData getString(int pos) {
+    return isNullAt(pos) ? null : getStringDataInternal(pos);
+  }
+
+  private StringData getStringDataInternal(int pos) {
+    CharSequence seq = struct.get(pos, CharSequence.class);
+    return StringData.fromString(seq.toString());
+  }
+
+  @Override
+  public DecimalData getDecimal(int pos, int precision, int scale) {
+    return isNullAt(pos)
+        ? null
+        : DecimalData.fromBigDecimal(getDecimalInternal(pos), precision, scale);
+  }
+
+  private BigDecimal getDecimalInternal(int pos) {
+    return struct.get(pos, BigDecimal.class);
+  }
+
+  @Override
+  public TimestampData getTimestamp(int pos, int precision) {
+    long timeLong = getLong(pos);
+    return TimestampData.fromEpochMillis(timeLong / 1000, (int) (timeLong % 1000) * 1000);
+  }
+
+  @Override
+  public <T> RawValueData<T> getRawValue(int pos) {
+    throw new UnsupportedOperationException("Not supported yet.");
+  }
+
+  @Override
+  public byte[] getBinary(int pos) {
+    return isNullAt(pos) ? null : getBinaryInternal(pos);
+  }
+
+  private byte[] getBinaryInternal(int pos) {
+    Object bytes = struct.get(pos, Object.class);
+
+    // should only be either ByteBuffer or byte[]
+    if (bytes instanceof ByteBuffer) {
+      return ByteBuffers.toByteArray((ByteBuffer) bytes);
+    } else if (bytes instanceof byte[]) {
+      return (byte[]) bytes;
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for binary field. Type name: " + bytes.getClass().getName());
+    }
+  }
+
+  @Override
+  public ArrayData getArray(int pos) {
+    return isNullAt(pos)
+        ? null
+        : (ArrayData)
+            covertFromStructToRowData(
+                type.fields().get(pos).type().asListType(), struct.get(pos, List.class));
+  }
+
+  @Override
+  public MapData getMap(int pos) {
+    return isNullAt(pos)
+        ? null
+        : (MapData)
+            covertFromStructToRowData(
+                type.fields().get(pos).type().asMapType(), struct.get(pos, Map.class));
+  }
+
+  @Override
+  public RowData getRow(int pos, int numFields) {
+    return isNullAt(pos) ? null : getStructRowData(pos, numFields);
+  }
+
+  private StructRowData getStructRowData(int pos, int numFields) {
+    return new StructRowData(
+        type.fields().get(pos).type().asStructType(), struct.get(pos, StructLike.class));
+  }
+
+  private Object covertFromStructToRowData(Type elementType, Object value) {
+    switch (elementType.typeId()) {
+      case BOOLEAN:
+      case INTEGER:
+      case DATE:
+      case TIME:
+      case LONG:
+      case FLOAT:
+      case DOUBLE:
+      case DECIMAL:
+        return value;
+      case TIMESTAMP:
+        return TimestampData.fromEpochMillis(
+            (Long) value / 1000, (int) ((Long) value % 1000) * 1000);
+      case STRING:
+        return StringData.fromString(value.toString());
+      case FIXED:
+      case BINARY:
+        return ByteBuffers.toByteArray((ByteBuffer) value);
+      case STRUCT:
+        return new StructRowData(elementType.asStructType(), (StructLike) value);
+      case LIST:
+        List<?> list = (List<?>) value;
+        Object[] array = new Object[list.size()];

Review Comment:
   Java arrays are covariant. Hence `Object[]` can hold references to any type. But I am wondering if it will cause problem for `equals` e.g. comparing btw `Object[]` and `Integer[]`. we will see once we write up the unit test (in a separate PR).



##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java:
##########
@@ -0,0 +1,286 @@
+/*
+ * 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.Duration;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.OffsetDateTime;
+import java.util.List;
+import java.util.Map;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericMapData;
+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.flink.types.RowKind;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.ByteBuffers;
+
+@Internal
+public class StructRowData implements RowData {
+  private final Types.StructType type;
+  private RowKind kind;
+  private StructLike struct;
+
+  public StructRowData(Types.StructType type) {
+    this(type, RowKind.INSERT);
+  }
+
+  public StructRowData(Types.StructType type, RowKind kind) {
+    this(type, null, kind);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct) {
+    this(type, struct, RowKind.INSERT);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct, RowKind kind) {
+    this.type = type;
+    this.struct = struct;
+    this.kind = kind;
+  }
+
+  public StructRowData setStruct(StructLike newStruct) {
+    this.struct = newStruct;
+    return this;
+  }
+
+  @Override
+  public int getArity() {
+    return struct.size();
+  }
+
+  @Override
+  public RowKind getRowKind() {
+    return kind;
+  }
+
+  @Override
+  public void setRowKind(RowKind newKind) {
+    Preconditions.checkNotNull(newKind, "kind can not be null");
+    this.kind = newKind;
+  }
+
+  @Override
+  public boolean isNullAt(int pos) {
+    return struct.get(pos, Object.class) == null;
+  }
+
+  @Override
+  public boolean getBoolean(int pos) {
+    return struct.get(pos, Boolean.class);
+  }
+
+  @Override
+  public byte getByte(int pos) {
+    return (byte) (int) struct.get(pos, Integer.class);
+  }
+
+  @Override
+  public short getShort(int pos) {
+    return (short) (int) struct.get(pos, Integer.class);
+  }
+
+  @Override
+  public int getInt(int pos) {
+    Object integer = struct.get(pos, Object.class);
+
+    if (integer instanceof Integer) {
+      return (int) integer;
+    } else if (integer instanceof LocalDate) {
+      return (int) ((LocalDate) integer).toEpochDay();
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for int field. Type name: " + integer.getClass().getName());
+    }
+  }
+
+  @Override
+  public long getLong(int pos) {
+    Object longVal = struct.get(pos, Object.class);
+
+    if (longVal instanceof Long) {
+      return (long) longVal;
+    } else if (longVal instanceof OffsetDateTime) {
+      return Duration.between(Instant.EPOCH, (OffsetDateTime) longVal).toNanos() / 1000;
+    } else if (longVal instanceof LocalDate) {
+      return ((LocalDate) longVal).toEpochDay();
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for long field. Type name: " + longVal.getClass().getName());
+    }
+  }
+
+  @Override
+  public float getFloat(int pos) {
+    return struct.get(pos, Float.class);
+  }
+
+  @Override
+  public double getDouble(int pos) {
+    return struct.get(pos, Double.class);
+  }
+
+  @Override
+  public StringData getString(int pos) {
+    return isNullAt(pos) ? null : getStringDataInternal(pos);
+  }
+
+  private StringData getStringDataInternal(int pos) {
+    CharSequence seq = struct.get(pos, CharSequence.class);
+    return StringData.fromString(seq.toString());
+  }
+
+  @Override
+  public DecimalData getDecimal(int pos, int precision, int scale) {
+    return isNullAt(pos)
+        ? null
+        : DecimalData.fromBigDecimal(getDecimalInternal(pos), precision, scale);
+  }
+
+  private BigDecimal getDecimalInternal(int pos) {
+    return struct.get(pos, BigDecimal.class);
+  }
+
+  @Override
+  public TimestampData getTimestamp(int pos, int precision) {
+    long timeLong = getLong(pos);
+    return TimestampData.fromEpochMillis(timeLong / 1000, (int) (timeLong % 1000) * 1000);
+  }
+
+  @Override
+  public <T> RawValueData<T> getRawValue(int pos) {
+    throw new UnsupportedOperationException("Not supported yet.");
+  }
+
+  @Override
+  public byte[] getBinary(int pos) {
+    return isNullAt(pos) ? null : getBinaryInternal(pos);
+  }
+
+  private byte[] getBinaryInternal(int pos) {
+    Object bytes = struct.get(pos, Object.class);
+
+    // should only be either ByteBuffer or byte[]
+    if (bytes instanceof ByteBuffer) {
+      return ByteBuffers.toByteArray((ByteBuffer) bytes);
+    } else if (bytes instanceof byte[]) {
+      return (byte[]) bytes;
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for binary field. Type name: " + bytes.getClass().getName());
+    }
+  }
+
+  @Override
+  public ArrayData getArray(int pos) {
+    return isNullAt(pos)
+        ? null
+        : (ArrayData)
+            covertFromStructToRowData(
+                type.fields().get(pos).type().asListType(), struct.get(pos, List.class));
+  }
+
+  @Override
+  public MapData getMap(int pos) {
+    return isNullAt(pos)
+        ? null
+        : (MapData)
+            covertFromStructToRowData(
+                type.fields().get(pos).type().asMapType(), struct.get(pos, Map.class));
+  }
+
+  @Override
+  public RowData getRow(int pos, int numFields) {
+    return isNullAt(pos) ? null : getStructRowData(pos, numFields);
+  }
+
+  private StructRowData getStructRowData(int pos, int numFields) {
+    return new StructRowData(
+        type.fields().get(pos).type().asStructType(), struct.get(pos, StructLike.class));
+  }
+
+  private Object covertFromStructToRowData(Type elementType, Object value) {
+    switch (elementType.typeId()) {
+      case BOOLEAN:
+      case INTEGER:
+      case DATE:
+      case TIME:
+      case LONG:
+      case FLOAT:
+      case DOUBLE:
+      case DECIMAL:
+        return value;
+      case TIMESTAMP:
+        return TimestampData.fromEpochMillis(
+            (Long) value / 1000, (int) ((Long) value % 1000) * 1000);
+      case STRING:
+        return StringData.fromString(value.toString());
+      case FIXED:
+      case BINARY:
+        return ByteBuffers.toByteArray((ByteBuffer) value);
+      case STRUCT:
+        return new StructRowData(elementType.asStructType(), (StructLike) value);
+      case LIST:
+        List<?> list = (List<?>) value;
+        Object[] array = new Object[list.size()];
+
+        int index = 0;
+        for (Object element : list) {
+          if (element == null) {
+            array[index] = null;
+          } else {
+            array[index] =
+                covertFromStructToRowData(elementType.asListType().elementType(), element);
+          }
+
+          index += 1;
+        }
+        return new GenericArrayData(array);
+      case MAP:
+        Types.MapType mapType = elementType.asMapType();
+        // make a defensive copy to ensure entries do not change

Review Comment:
   is this necessary? if we need defensive copy for map, we should do it for every fields, right?



##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java:
##########
@@ -0,0 +1,286 @@
+/*
+ * 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.Duration;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.OffsetDateTime;
+import java.util.List;
+import java.util.Map;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericMapData;
+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.flink.types.RowKind;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.ByteBuffers;
+
+@Internal
+public class StructRowData implements RowData {
+  private final Types.StructType type;
+  private RowKind kind;
+  private StructLike struct;
+
+  public StructRowData(Types.StructType type) {
+    this(type, RowKind.INSERT);
+  }
+
+  public StructRowData(Types.StructType type, RowKind kind) {
+    this(type, null, kind);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct) {
+    this(type, struct, RowKind.INSERT);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct, RowKind kind) {
+    this.type = type;
+    this.struct = struct;
+    this.kind = kind;
+  }
+
+  public StructRowData setStruct(StructLike newStruct) {
+    this.struct = newStruct;
+    return this;
+  }
+
+  @Override
+  public int getArity() {
+    return struct.size();
+  }
+
+  @Override
+  public RowKind getRowKind() {
+    return kind;
+  }
+
+  @Override
+  public void setRowKind(RowKind newKind) {
+    Preconditions.checkNotNull(newKind, "kind can not be null");
+    this.kind = newKind;
+  }
+
+  @Override
+  public boolean isNullAt(int pos) {
+    return struct.get(pos, Object.class) == null;
+  }
+
+  @Override
+  public boolean getBoolean(int pos) {
+    return struct.get(pos, Boolean.class);
+  }
+
+  @Override
+  public byte getByte(int pos) {
+    return (byte) (int) struct.get(pos, Integer.class);
+  }
+
+  @Override
+  public short getShort(int pos) {
+    return (short) (int) struct.get(pos, Integer.class);
+  }
+
+  @Override
+  public int getInt(int pos) {
+    Object integer = struct.get(pos, Object.class);
+
+    if (integer instanceof Integer) {
+      return (int) integer;
+    } else if (integer instanceof LocalDate) {
+      return (int) ((LocalDate) integer).toEpochDay();
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for int field. Type name: " + integer.getClass().getName());
+    }
+  }
+
+  @Override
+  public long getLong(int pos) {
+    Object longVal = struct.get(pos, Object.class);
+
+    if (longVal instanceof Long) {
+      return (long) longVal;
+    } else if (longVal instanceof OffsetDateTime) {
+      return Duration.between(Instant.EPOCH, (OffsetDateTime) longVal).toNanos() / 1000;
+    } else if (longVal instanceof LocalDate) {
+      return ((LocalDate) longVal).toEpochDay();
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for long field. Type name: " + longVal.getClass().getName());
+    }
+  }
+
+  @Override
+  public float getFloat(int pos) {
+    return struct.get(pos, Float.class);
+  }
+
+  @Override
+  public double getDouble(int pos) {
+    return struct.get(pos, Double.class);
+  }
+
+  @Override
+  public StringData getString(int pos) {
+    return isNullAt(pos) ? null : getStringDataInternal(pos);
+  }
+
+  private StringData getStringDataInternal(int pos) {
+    CharSequence seq = struct.get(pos, CharSequence.class);
+    return StringData.fromString(seq.toString());
+  }
+
+  @Override
+  public DecimalData getDecimal(int pos, int precision, int scale) {
+    return isNullAt(pos)
+        ? null
+        : DecimalData.fromBigDecimal(getDecimalInternal(pos), precision, scale);
+  }
+
+  private BigDecimal getDecimalInternal(int pos) {
+    return struct.get(pos, BigDecimal.class);
+  }
+
+  @Override
+  public TimestampData getTimestamp(int pos, int precision) {
+    long timeLong = getLong(pos);
+    return TimestampData.fromEpochMillis(timeLong / 1000, (int) (timeLong % 1000) * 1000);
+  }
+
+  @Override
+  public <T> RawValueData<T> getRawValue(int pos) {
+    throw new UnsupportedOperationException("Not supported yet.");
+  }
+
+  @Override
+  public byte[] getBinary(int pos) {
+    return isNullAt(pos) ? null : getBinaryInternal(pos);
+  }
+
+  private byte[] getBinaryInternal(int pos) {
+    Object bytes = struct.get(pos, Object.class);
+
+    // should only be either ByteBuffer or byte[]
+    if (bytes instanceof ByteBuffer) {
+      return ByteBuffers.toByteArray((ByteBuffer) bytes);
+    } else if (bytes instanceof byte[]) {
+      return (byte[]) bytes;
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for binary field. Type name: " + bytes.getClass().getName());
+    }
+  }
+
+  @Override
+  public ArrayData getArray(int pos) {
+    return isNullAt(pos)
+        ? null
+        : (ArrayData)
+            covertFromStructToRowData(
+                type.fields().get(pos).type().asListType(), struct.get(pos, List.class));
+  }
+
+  @Override
+  public MapData getMap(int pos) {
+    return isNullAt(pos)
+        ? null
+        : (MapData)
+            covertFromStructToRowData(
+                type.fields().get(pos).type().asMapType(), struct.get(pos, Map.class));
+  }
+
+  @Override
+  public RowData getRow(int pos, int numFields) {
+    return isNullAt(pos) ? null : getStructRowData(pos, numFields);
+  }
+
+  private StructRowData getStructRowData(int pos, int numFields) {
+    return new StructRowData(
+        type.fields().get(pos).type().asStructType(), struct.get(pos, StructLike.class));
+  }
+
+  private Object covertFromStructToRowData(Type elementType, Object value) {
+    switch (elementType.typeId()) {
+      case BOOLEAN:
+      case INTEGER:
+      case DATE:
+      case TIME:
+      case LONG:
+      case FLOAT:
+      case DOUBLE:
+      case DECIMAL:
+        return value;
+      case TIMESTAMP:
+        return TimestampData.fromEpochMillis(
+            (Long) value / 1000, (int) ((Long) value % 1000) * 1000);
+      case STRING:
+        return StringData.fromString(value.toString());
+      case FIXED:
+      case BINARY:
+        return ByteBuffers.toByteArray((ByteBuffer) value);
+      case STRUCT:
+        return new StructRowData(elementType.asStructType(), (StructLike) value);
+      case LIST:
+        List<?> list = (List<?>) value;
+        Object[] array = new Object[list.size()];
+
+        int index = 0;
+        for (Object element : list) {
+          if (element == null) {
+            array[index] = null;
+          } else {
+            array[index] =
+                covertFromStructToRowData(elementType.asListType().elementType(), element);
+          }
+
+          index += 1;
+        }
+        return new GenericArrayData(array);
+      case MAP:
+        Types.MapType mapType = elementType.asMapType();
+        // make a defensive copy to ensure entries do not change
+        List<Map.Entry<?, ?>> entries = ImmutableList.copyOf(((Map<?, ?>) value).entrySet());
+
+        Map<Object, Object> result = Maps.newHashMap();
+        for (Map.Entry<?, ?> entry : entries) {
+          final Object keyValue = covertFromStructToRowData(mapType.keyType(), entry.getKey());
+          final Object valueValue =
+              covertFromStructToRowData(mapType.valueType(), entry.getValue());
+          result.put(keyValue, valueValue);
+        }
+        return new GenericMapData(result);
+      default:
+        throw new UnsupportedOperationException("Unsupported array element type: " + elementType);

Review Comment:
   is the error msg `array element type` correct?



##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java:
##########
@@ -0,0 +1,286 @@
+/*
+ * 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.Duration;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.OffsetDateTime;
+import java.util.List;
+import java.util.Map;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericMapData;
+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.flink.types.RowKind;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.ByteBuffers;
+
+@Internal
+public class StructRowData implements RowData {
+  private final Types.StructType type;
+  private RowKind kind;
+  private StructLike struct;
+
+  public StructRowData(Types.StructType type) {
+    this(type, RowKind.INSERT);
+  }
+
+  public StructRowData(Types.StructType type, RowKind kind) {
+    this(type, null, kind);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct) {
+    this(type, struct, RowKind.INSERT);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct, RowKind kind) {
+    this.type = type;
+    this.struct = struct;
+    this.kind = kind;
+  }
+
+  public StructRowData setStruct(StructLike newStruct) {
+    this.struct = newStruct;
+    return this;
+  }
+
+  @Override
+  public int getArity() {
+    return struct.size();
+  }
+
+  @Override
+  public RowKind getRowKind() {
+    return kind;
+  }
+
+  @Override
+  public void setRowKind(RowKind newKind) {
+    Preconditions.checkNotNull(newKind, "kind can not be null");
+    this.kind = newKind;
+  }
+
+  @Override
+  public boolean isNullAt(int pos) {
+    return struct.get(pos, Object.class) == null;
+  }
+
+  @Override
+  public boolean getBoolean(int pos) {
+    return struct.get(pos, Boolean.class);
+  }
+
+  @Override
+  public byte getByte(int pos) {
+    return (byte) (int) struct.get(pos, Integer.class);
+  }
+
+  @Override
+  public short getShort(int pos) {
+    return (short) (int) struct.get(pos, Integer.class);
+  }
+
+  @Override
+  public int getInt(int pos) {
+    Object integer = struct.get(pos, Object.class);
+
+    if (integer instanceof Integer) {
+      return (int) integer;
+    } else if (integer instanceof LocalDate) {
+      return (int) ((LocalDate) integer).toEpochDay();
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for int field. Type name: " + integer.getClass().getName());
+    }
+  }
+
+  @Override
+  public long getLong(int pos) {
+    Object longVal = struct.get(pos, Object.class);
+
+    if (longVal instanceof Long) {
+      return (long) longVal;
+    } else if (longVal instanceof OffsetDateTime) {
+      return Duration.between(Instant.EPOCH, (OffsetDateTime) longVal).toNanos() / 1000;
+    } else if (longVal instanceof LocalDate) {
+      return ((LocalDate) longVal).toEpochDay();
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for long field. Type name: " + longVal.getClass().getName());
+    }
+  }
+
+  @Override
+  public float getFloat(int pos) {
+    return struct.get(pos, Float.class);
+  }
+
+  @Override
+  public double getDouble(int pos) {
+    return struct.get(pos, Double.class);
+  }
+
+  @Override
+  public StringData getString(int pos) {
+    return isNullAt(pos) ? null : getStringDataInternal(pos);
+  }
+
+  private StringData getStringDataInternal(int pos) {
+    CharSequence seq = struct.get(pos, CharSequence.class);
+    return StringData.fromString(seq.toString());
+  }
+
+  @Override
+  public DecimalData getDecimal(int pos, int precision, int scale) {
+    return isNullAt(pos)
+        ? null
+        : DecimalData.fromBigDecimal(getDecimalInternal(pos), precision, scale);
+  }
+
+  private BigDecimal getDecimalInternal(int pos) {
+    return struct.get(pos, BigDecimal.class);
+  }
+
+  @Override
+  public TimestampData getTimestamp(int pos, int precision) {
+    long timeLong = getLong(pos);
+    return TimestampData.fromEpochMillis(timeLong / 1000, (int) (timeLong % 1000) * 1000);
+  }
+
+  @Override
+  public <T> RawValueData<T> getRawValue(int pos) {
+    throw new UnsupportedOperationException("Not supported yet.");
+  }
+
+  @Override
+  public byte[] getBinary(int pos) {
+    return isNullAt(pos) ? null : getBinaryInternal(pos);
+  }
+
+  private byte[] getBinaryInternal(int pos) {
+    Object bytes = struct.get(pos, Object.class);
+
+    // should only be either ByteBuffer or byte[]
+    if (bytes instanceof ByteBuffer) {
+      return ByteBuffers.toByteArray((ByteBuffer) bytes);
+    } else if (bytes instanceof byte[]) {
+      return (byte[]) bytes;
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for binary field. Type name: " + bytes.getClass().getName());
+    }
+  }
+
+  @Override
+  public ArrayData getArray(int pos) {
+    return isNullAt(pos)
+        ? null
+        : (ArrayData)
+            covertFromStructToRowData(
+                type.fields().get(pos).type().asListType(), struct.get(pos, List.class));
+  }
+
+  @Override
+  public MapData getMap(int pos) {
+    return isNullAt(pos)
+        ? null
+        : (MapData)
+            covertFromStructToRowData(
+                type.fields().get(pos).type().asMapType(), struct.get(pos, Map.class));
+  }
+
+  @Override
+  public RowData getRow(int pos, int numFields) {
+    return isNullAt(pos) ? null : getStructRowData(pos, numFields);
+  }
+
+  private StructRowData getStructRowData(int pos, int numFields) {
+    return new StructRowData(
+        type.fields().get(pos).type().asStructType(), struct.get(pos, StructLike.class));
+  }
+
+  private Object covertFromStructToRowData(Type elementType, Object value) {
+    switch (elementType.typeId()) {
+      case BOOLEAN:
+      case INTEGER:
+      case DATE:
+      case TIME:
+      case LONG:
+      case FLOAT:
+      case DOUBLE:
+      case DECIMAL:
+        return value;
+      case TIMESTAMP:
+        return TimestampData.fromEpochMillis(
+            (Long) value / 1000, (int) ((Long) value % 1000) * 1000);
+      case STRING:
+        return StringData.fromString(value.toString());
+      case FIXED:
+      case BINARY:
+        return ByteBuffers.toByteArray((ByteBuffer) value);
+      case STRUCT:
+        return new StructRowData(elementType.asStructType(), (StructLike) value);
+      case LIST:
+        List<?> list = (List<?>) value;
+        Object[] array = new Object[list.size()];
+

Review Comment:
   nit: empty line not need here



##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java:
##########
@@ -0,0 +1,286 @@
+/*
+ * 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.Duration;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.OffsetDateTime;
+import java.util.List;
+import java.util.Map;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericMapData;
+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.flink.types.RowKind;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.ByteBuffers;
+
+@Internal
+public class StructRowData implements RowData {
+  private final Types.StructType type;
+  private RowKind kind;
+  private StructLike struct;
+
+  public StructRowData(Types.StructType type) {
+    this(type, RowKind.INSERT);
+  }
+
+  public StructRowData(Types.StructType type, RowKind kind) {
+    this(type, null, kind);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct) {
+    this(type, struct, RowKind.INSERT);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct, RowKind kind) {
+    this.type = type;
+    this.struct = struct;
+    this.kind = kind;
+  }
+
+  public StructRowData setStruct(StructLike newStruct) {
+    this.struct = newStruct;
+    return this;
+  }
+
+  @Override
+  public int getArity() {
+    return struct.size();
+  }
+
+  @Override
+  public RowKind getRowKind() {
+    return kind;
+  }
+
+  @Override
+  public void setRowKind(RowKind newKind) {
+    Preconditions.checkNotNull(newKind, "kind can not be null");
+    this.kind = newKind;
+  }
+
+  @Override
+  public boolean isNullAt(int pos) {
+    return struct.get(pos, Object.class) == null;
+  }
+
+  @Override
+  public boolean getBoolean(int pos) {
+    return struct.get(pos, Boolean.class);
+  }
+
+  @Override
+  public byte getByte(int pos) {
+    return (byte) (int) struct.get(pos, Integer.class);
+  }
+
+  @Override
+  public short getShort(int pos) {
+    return (short) (int) struct.get(pos, Integer.class);
+  }
+
+  @Override
+  public int getInt(int pos) {
+    Object integer = struct.get(pos, Object.class);
+
+    if (integer instanceof Integer) {
+      return (int) integer;
+    } else if (integer instanceof LocalDate) {
+      return (int) ((LocalDate) integer).toEpochDay();
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for int field. Type name: " + integer.getClass().getName());
+    }
+  }
+
+  @Override
+  public long getLong(int pos) {
+    Object longVal = struct.get(pos, Object.class);
+
+    if (longVal instanceof Long) {
+      return (long) longVal;
+    } else if (longVal instanceof OffsetDateTime) {
+      return Duration.between(Instant.EPOCH, (OffsetDateTime) longVal).toNanos() / 1000;
+    } else if (longVal instanceof LocalDate) {
+      return ((LocalDate) longVal).toEpochDay();
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for long field. Type name: " + longVal.getClass().getName());
+    }
+  }
+
+  @Override
+  public float getFloat(int pos) {
+    return struct.get(pos, Float.class);
+  }
+
+  @Override
+  public double getDouble(int pos) {
+    return struct.get(pos, Double.class);
+  }
+
+  @Override
+  public StringData getString(int pos) {
+    return isNullAt(pos) ? null : getStringDataInternal(pos);
+  }
+
+  private StringData getStringDataInternal(int pos) {
+    CharSequence seq = struct.get(pos, CharSequence.class);
+    return StringData.fromString(seq.toString());
+  }
+
+  @Override
+  public DecimalData getDecimal(int pos, int precision, int scale) {
+    return isNullAt(pos)
+        ? null
+        : DecimalData.fromBigDecimal(getDecimalInternal(pos), precision, scale);
+  }
+
+  private BigDecimal getDecimalInternal(int pos) {
+    return struct.get(pos, BigDecimal.class);
+  }
+
+  @Override
+  public TimestampData getTimestamp(int pos, int precision) {
+    long timeLong = getLong(pos);
+    return TimestampData.fromEpochMillis(timeLong / 1000, (int) (timeLong % 1000) * 1000);
+  }
+
+  @Override
+  public <T> RawValueData<T> getRawValue(int pos) {
+    throw new UnsupportedOperationException("Not supported yet.");
+  }
+
+  @Override
+  public byte[] getBinary(int pos) {
+    return isNullAt(pos) ? null : getBinaryInternal(pos);
+  }
+
+  private byte[] getBinaryInternal(int pos) {
+    Object bytes = struct.get(pos, Object.class);
+
+    // should only be either ByteBuffer or byte[]
+    if (bytes instanceof ByteBuffer) {
+      return ByteBuffers.toByteArray((ByteBuffer) bytes);
+    } else if (bytes instanceof byte[]) {
+      return (byte[]) bytes;
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for binary field. Type name: " + bytes.getClass().getName());
+    }
+  }
+
+  @Override
+  public ArrayData getArray(int pos) {
+    return isNullAt(pos)
+        ? null
+        : (ArrayData)
+            covertFromStructToRowData(
+                type.fields().get(pos).type().asListType(), struct.get(pos, List.class));
+  }
+
+  @Override
+  public MapData getMap(int pos) {
+    return isNullAt(pos)
+        ? null
+        : (MapData)
+            covertFromStructToRowData(
+                type.fields().get(pos).type().asMapType(), struct.get(pos, Map.class));
+  }
+
+  @Override
+  public RowData getRow(int pos, int numFields) {
+    return isNullAt(pos) ? null : getStructRowData(pos, numFields);
+  }
+
+  private StructRowData getStructRowData(int pos, int numFields) {
+    return new StructRowData(
+        type.fields().get(pos).type().asStructType(), struct.get(pos, StructLike.class));
+  }
+
+  private Object covertFromStructToRowData(Type elementType, Object value) {
+    switch (elementType.typeId()) {
+      case BOOLEAN:
+      case INTEGER:
+      case DATE:
+      case TIME:
+      case LONG:
+      case FLOAT:
+      case DOUBLE:
+      case DECIMAL:
+        return value;
+      case TIMESTAMP:
+        return TimestampData.fromEpochMillis(
+            (Long) value / 1000, (int) ((Long) value % 1000) * 1000);
+      case STRING:
+        return StringData.fromString(value.toString());
+      case FIXED:
+      case BINARY:
+        return ByteBuffers.toByteArray((ByteBuffer) value);
+      case STRUCT:
+        return new StructRowData(elementType.asStructType(), (StructLike) value);
+      case LIST:
+        List<?> list = (List<?>) value;
+        Object[] array = new Object[list.size()];
+
+        int index = 0;
+        for (Object element : list) {
+          if (element == null) {
+            array[index] = null;
+          } else {
+            array[index] =
+                covertFromStructToRowData(elementType.asListType().elementType(), element);
+          }
+
+          index += 1;
+        }
+        return new GenericArrayData(array);
+      case MAP:
+        Types.MapType mapType = elementType.asMapType();
+        // make a defensive copy to ensure entries do not change
+        List<Map.Entry<?, ?>> entries = ImmutableList.copyOf(((Map<?, ?>) value).entrySet());
+

Review Comment:
   nit: no empty line needed here.



##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java:
##########
@@ -0,0 +1,286 @@
+/*
+ * 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.Duration;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.OffsetDateTime;
+import java.util.List;
+import java.util.Map;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericMapData;
+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.flink.types.RowKind;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.ByteBuffers;
+
+@Internal
+public class StructRowData implements RowData {
+  private final Types.StructType type;
+  private RowKind kind;
+  private StructLike struct;
+
+  public StructRowData(Types.StructType type) {
+    this(type, RowKind.INSERT);
+  }
+
+  public StructRowData(Types.StructType type, RowKind kind) {
+    this(type, null, kind);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct) {
+    this(type, struct, RowKind.INSERT);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct, RowKind kind) {
+    this.type = type;
+    this.struct = struct;
+    this.kind = kind;
+  }
+
+  public StructRowData setStruct(StructLike newStruct) {
+    this.struct = newStruct;
+    return this;
+  }
+
+  @Override
+  public int getArity() {
+    return struct.size();
+  }
+
+  @Override
+  public RowKind getRowKind() {
+    return kind;
+  }
+
+  @Override
+  public void setRowKind(RowKind newKind) {
+    Preconditions.checkNotNull(newKind, "kind can not be null");
+    this.kind = newKind;
+  }
+
+  @Override
+  public boolean isNullAt(int pos) {
+    return struct.get(pos, Object.class) == null;
+  }
+
+  @Override
+  public boolean getBoolean(int pos) {
+    return struct.get(pos, Boolean.class);
+  }
+
+  @Override
+  public byte getByte(int pos) {
+    return (byte) (int) struct.get(pos, Integer.class);
+  }
+
+  @Override
+  public short getShort(int pos) {
+    return (short) (int) struct.get(pos, Integer.class);
+  }
+
+  @Override
+  public int getInt(int pos) {
+    Object integer = struct.get(pos, Object.class);
+
+    if (integer instanceof Integer) {
+      return (int) integer;
+    } else if (integer instanceof LocalDate) {
+      return (int) ((LocalDate) integer).toEpochDay();
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for int field. Type name: " + integer.getClass().getName());
+    }
+  }
+
+  @Override
+  public long getLong(int pos) {
+    Object longVal = struct.get(pos, Object.class);
+
+    if (longVal instanceof Long) {
+      return (long) longVal;
+    } else if (longVal instanceof OffsetDateTime) {
+      return Duration.between(Instant.EPOCH, (OffsetDateTime) longVal).toNanos() / 1000;
+    } else if (longVal instanceof LocalDate) {
+      return ((LocalDate) longVal).toEpochDay();
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for long field. Type name: " + longVal.getClass().getName());
+    }
+  }
+
+  @Override
+  public float getFloat(int pos) {
+    return struct.get(pos, Float.class);
+  }
+
+  @Override
+  public double getDouble(int pos) {
+    return struct.get(pos, Double.class);
+  }
+
+  @Override
+  public StringData getString(int pos) {
+    return isNullAt(pos) ? null : getStringDataInternal(pos);
+  }
+
+  private StringData getStringDataInternal(int pos) {
+    CharSequence seq = struct.get(pos, CharSequence.class);
+    return StringData.fromString(seq.toString());
+  }
+
+  @Override
+  public DecimalData getDecimal(int pos, int precision, int scale) {
+    return isNullAt(pos)
+        ? null
+        : DecimalData.fromBigDecimal(getDecimalInternal(pos), precision, scale);
+  }
+
+  private BigDecimal getDecimalInternal(int pos) {
+    return struct.get(pos, BigDecimal.class);
+  }
+
+  @Override
+  public TimestampData getTimestamp(int pos, int precision) {
+    long timeLong = getLong(pos);
+    return TimestampData.fromEpochMillis(timeLong / 1000, (int) (timeLong % 1000) * 1000);
+  }
+
+  @Override
+  public <T> RawValueData<T> getRawValue(int pos) {
+    throw new UnsupportedOperationException("Not supported yet.");
+  }
+
+  @Override
+  public byte[] getBinary(int pos) {
+    return isNullAt(pos) ? null : getBinaryInternal(pos);
+  }
+
+  private byte[] getBinaryInternal(int pos) {
+    Object bytes = struct.get(pos, Object.class);
+
+    // should only be either ByteBuffer or byte[]
+    if (bytes instanceof ByteBuffer) {
+      return ByteBuffers.toByteArray((ByteBuffer) bytes);
+    } else if (bytes instanceof byte[]) {
+      return (byte[]) bytes;
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for binary field. Type name: " + bytes.getClass().getName());
+    }
+  }
+
+  @Override
+  public ArrayData getArray(int pos) {
+    return isNullAt(pos)
+        ? null
+        : (ArrayData)
+            covertFromStructToRowData(
+                type.fields().get(pos).type().asListType(), struct.get(pos, List.class));
+  }
+
+  @Override
+  public MapData getMap(int pos) {
+    return isNullAt(pos)
+        ? null
+        : (MapData)
+            covertFromStructToRowData(
+                type.fields().get(pos).type().asMapType(), struct.get(pos, Map.class));
+  }
+
+  @Override
+  public RowData getRow(int pos, int numFields) {
+    return isNullAt(pos) ? null : getStructRowData(pos, numFields);
+  }
+
+  private StructRowData getStructRowData(int pos, int numFields) {
+    return new StructRowData(
+        type.fields().get(pos).type().asStructType(), struct.get(pos, StructLike.class));
+  }
+
+  private Object covertFromStructToRowData(Type elementType, Object value) {
+    switch (elementType.typeId()) {
+      case BOOLEAN:
+      case INTEGER:
+      case DATE:
+      case TIME:
+      case LONG:
+      case FLOAT:
+      case DOUBLE:
+      case DECIMAL:
+        return value;
+      case TIMESTAMP:
+        return TimestampData.fromEpochMillis(
+            (Long) value / 1000, (int) ((Long) value % 1000) * 1000);
+      case STRING:
+        return StringData.fromString(value.toString());
+      case FIXED:
+      case BINARY:
+        return ByteBuffers.toByteArray((ByteBuffer) value);
+      case STRUCT:
+        return new StructRowData(elementType.asStructType(), (StructLike) value);
+      case LIST:
+        List<?> list = (List<?>) value;
+        Object[] array = new Object[list.size()];
+
+        int index = 0;
+        for (Object element : list) {
+          if (element == null) {
+            array[index] = null;
+          } else {
+            array[index] =
+                covertFromStructToRowData(elementType.asListType().elementType(), element);
+          }
+
+          index += 1;
+        }

Review Comment:
   nit: Iceberg coding style requires empty line after control block.



##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java:
##########
@@ -0,0 +1,286 @@
+/*
+ * 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.Duration;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.OffsetDateTime;
+import java.util.List;
+import java.util.Map;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericMapData;
+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.flink.types.RowKind;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.ByteBuffers;
+
+@Internal
+public class StructRowData implements RowData {
+  private final Types.StructType type;
+  private RowKind kind;
+  private StructLike struct;
+
+  public StructRowData(Types.StructType type) {
+    this(type, RowKind.INSERT);
+  }
+
+  public StructRowData(Types.StructType type, RowKind kind) {
+    this(type, null, kind);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct) {
+    this(type, struct, RowKind.INSERT);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct, RowKind kind) {
+    this.type = type;
+    this.struct = struct;
+    this.kind = kind;
+  }
+
+  public StructRowData setStruct(StructLike newStruct) {
+    this.struct = newStruct;
+    return this;
+  }
+
+  @Override
+  public int getArity() {
+    return struct.size();
+  }
+
+  @Override
+  public RowKind getRowKind() {
+    return kind;
+  }
+
+  @Override
+  public void setRowKind(RowKind newKind) {
+    Preconditions.checkNotNull(newKind, "kind can not be null");
+    this.kind = newKind;
+  }
+
+  @Override
+  public boolean isNullAt(int pos) {
+    return struct.get(pos, Object.class) == null;
+  }
+
+  @Override
+  public boolean getBoolean(int pos) {
+    return struct.get(pos, Boolean.class);
+  }
+
+  @Override
+  public byte getByte(int pos) {
+    return (byte) (int) struct.get(pos, Integer.class);
+  }
+
+  @Override
+  public short getShort(int pos) {
+    return (short) (int) struct.get(pos, Integer.class);
+  }
+
+  @Override
+  public int getInt(int pos) {
+    Object integer = struct.get(pos, Object.class);
+
+    if (integer instanceof Integer) {
+      return (int) integer;
+    } else if (integer instanceof LocalDate) {
+      return (int) ((LocalDate) integer).toEpochDay();
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for int field. Type name: " + integer.getClass().getName());
+    }
+  }
+
+  @Override
+  public long getLong(int pos) {
+    Object longVal = struct.get(pos, Object.class);
+
+    if (longVal instanceof Long) {
+      return (long) longVal;
+    } else if (longVal instanceof OffsetDateTime) {
+      return Duration.between(Instant.EPOCH, (OffsetDateTime) longVal).toNanos() / 1000;
+    } else if (longVal instanceof LocalDate) {
+      return ((LocalDate) longVal).toEpochDay();
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for long field. Type name: " + longVal.getClass().getName());
+    }
+  }
+
+  @Override
+  public float getFloat(int pos) {
+    return struct.get(pos, Float.class);
+  }
+
+  @Override
+  public double getDouble(int pos) {
+    return struct.get(pos, Double.class);
+  }
+
+  @Override
+  public StringData getString(int pos) {
+    return isNullAt(pos) ? null : getStringDataInternal(pos);
+  }
+
+  private StringData getStringDataInternal(int pos) {
+    CharSequence seq = struct.get(pos, CharSequence.class);
+    return StringData.fromString(seq.toString());
+  }
+
+  @Override
+  public DecimalData getDecimal(int pos, int precision, int scale) {
+    return isNullAt(pos)
+        ? null
+        : DecimalData.fromBigDecimal(getDecimalInternal(pos), precision, scale);
+  }
+
+  private BigDecimal getDecimalInternal(int pos) {
+    return struct.get(pos, BigDecimal.class);
+  }
+
+  @Override
+  public TimestampData getTimestamp(int pos, int precision) {
+    long timeLong = getLong(pos);
+    return TimestampData.fromEpochMillis(timeLong / 1000, (int) (timeLong % 1000) * 1000);
+  }
+
+  @Override
+  public <T> RawValueData<T> getRawValue(int pos) {
+    throw new UnsupportedOperationException("Not supported yet.");
+  }
+
+  @Override
+  public byte[] getBinary(int pos) {
+    return isNullAt(pos) ? null : getBinaryInternal(pos);
+  }
+
+  private byte[] getBinaryInternal(int pos) {
+    Object bytes = struct.get(pos, Object.class);
+
+    // should only be either ByteBuffer or byte[]
+    if (bytes instanceof ByteBuffer) {
+      return ByteBuffers.toByteArray((ByteBuffer) bytes);
+    } else if (bytes instanceof byte[]) {
+      return (byte[]) bytes;
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for binary field. Type name: " + bytes.getClass().getName());
+    }
+  }
+
+  @Override
+  public ArrayData getArray(int pos) {
+    return isNullAt(pos)
+        ? null
+        : (ArrayData)
+            covertFromStructToRowData(
+                type.fields().get(pos).type().asListType(), struct.get(pos, List.class));
+  }
+
+  @Override
+  public MapData getMap(int pos) {
+    return isNullAt(pos)
+        ? null
+        : (MapData)
+            covertFromStructToRowData(
+                type.fields().get(pos).type().asMapType(), struct.get(pos, Map.class));
+  }
+
+  @Override
+  public RowData getRow(int pos, int numFields) {
+    return isNullAt(pos) ? null : getStructRowData(pos, numFields);
+  }
+
+  private StructRowData getStructRowData(int pos, int numFields) {
+    return new StructRowData(
+        type.fields().get(pos).type().asStructType(), struct.get(pos, StructLike.class));
+  }
+
+  private Object covertFromStructToRowData(Type elementType, Object value) {
+    switch (elementType.typeId()) {
+      case BOOLEAN:
+      case INTEGER:
+      case DATE:
+      case TIME:
+      case LONG:
+      case FLOAT:
+      case DOUBLE:
+      case DECIMAL:
+        return value;
+      case TIMESTAMP:
+        return TimestampData.fromEpochMillis(
+            (Long) value / 1000, (int) ((Long) value % 1000) * 1000);
+      case STRING:
+        return StringData.fromString(value.toString());
+      case FIXED:
+      case BINARY:
+        return ByteBuffers.toByteArray((ByteBuffer) value);
+      case STRUCT:
+        return new StructRowData(elementType.asStructType(), (StructLike) value);
+      case LIST:
+        List<?> list = (List<?>) value;
+        Object[] array = new Object[list.size()];
+
+        int index = 0;
+        for (Object element : list) {
+          if (element == null) {
+            array[index] = null;
+          } else {
+            array[index] =
+                covertFromStructToRowData(elementType.asListType().elementType(), element);
+          }
+
+          index += 1;
+        }
+        return new GenericArrayData(array);
+      case MAP:
+        Types.MapType mapType = elementType.asMapType();
+        // make a defensive copy to ensure entries do not change
+        List<Map.Entry<?, ?>> entries = ImmutableList.copyOf(((Map<?, ?>) value).entrySet());
+
+        Map<Object, Object> result = Maps.newHashMap();
+        for (Map.Entry<?, ?> entry : entries) {
+          final Object keyValue = covertFromStructToRowData(mapType.keyType(), entry.getKey());
+          final Object valueValue =
+              covertFromStructToRowData(mapType.valueType(), entry.getValue());
+          result.put(keyValue, valueValue);
+        }

Review Comment:
   nit: iceberg coding style needs an empty line here (after control block).



##########
flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java:
##########
@@ -295,6 +299,161 @@ private static void assertEquals(
     }
   }
 
+  public static void assertEqualsSafe(
+      Schema schema, List<GenericData.Record> recs, List<Row> rows) {
+    Streams.forEachPair(
+        recs.stream(), rows.stream(), (rec, row) -> assertEqualsSafe(schema, rec, row));
+  }
+
+  public static void assertEqualsSafe(Schema schema, GenericData.Record rec, Row row) {
+    List<Types.NestedField> fields = schema.asStruct().fields();
+    RowType rowType = FlinkSchemaUtil.convert(schema);
+    for (int i = 0; i < fields.size(); i += 1) {

Review Comment:
   nit: use `++i`?



##########
flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java:
##########
@@ -295,6 +299,161 @@ private static void assertEquals(
     }
   }
 
+  public static void assertEqualsSafe(
+      Schema schema, List<GenericData.Record> recs, List<Row> rows) {
+    Streams.forEachPair(
+        recs.stream(), rows.stream(), (rec, row) -> assertEqualsSafe(schema, rec, row));
+  }
+
+  public static void assertEqualsSafe(Schema schema, GenericData.Record rec, Row row) {
+    List<Types.NestedField> fields = schema.asStruct().fields();

Review Comment:
   maybe first do an equals assertion on the size for the record and row



##########
flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMetaDataTable.java:
##########
@@ -0,0 +1,836 @@
+/*
+ * 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.source;
+
+import java.io.IOException;
+import java.time.Instant;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.avro.generic.GenericData;
+import org.apache.commons.collections.ListUtils;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.CoreOptions;
+import org.apache.flink.table.api.TableEnvironment;
+import org.apache.flink.types.Row;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileContent;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Files;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.MetadataTableUtils;
+import org.apache.iceberg.MetricsUtil;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.data.FileHelpers;
+import org.apache.iceberg.data.GenericRecord;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.flink.FlinkCatalogTestBase;
+import org.apache.iceberg.flink.TestHelpers;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.util.SnapshotUtil;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Assume;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class TestFlinkMetaDataTable extends FlinkCatalogTestBase {
+  private static final String TABLE_NAME = "test_table";
+  private final FileFormat format = FileFormat.AVRO;
+  private static final TemporaryFolder TEMP = new TemporaryFolder();
+  private final boolean isPartition;
+
+  public TestFlinkMetaDataTable(String catalogName, Namespace baseNamespace, Boolean isPartition) {
+    super(catalogName, baseNamespace);
+    this.isPartition = isPartition;
+  }
+
+  @Parameterized.Parameters(name = "catalogName={0}, baseNamespace={1}, isPartition={2}")
+  public static Iterable<Object[]> parameters() {
+    List<Object[]> parameters = Lists.newArrayList();
+
+    for (Boolean isPartition : new Boolean[] {true, false}) {
+      String catalogName = "testhadoop";
+      Namespace baseNamespace = Namespace.of("default");
+      parameters.add(new Object[] {catalogName, baseNamespace, isPartition});
+    }
+    return parameters;
+  }
+
+  @Override
+  protected TableEnvironment getTableEnv() {
+    Configuration configuration = super.getTableEnv().getConfig().getConfiguration();
+    configuration.set(CoreOptions.DEFAULT_PARALLELISM, 1);
+    return super.getTableEnv();
+  }
+
+  @Before
+  public void before() {
+    super.before();
+    sql("USE CATALOG %s", catalogName);
+    sql("CREATE DATABASE %s", flinkDatabase);
+    sql("USE %s", DATABASE);
+    if (isPartition) {
+      sql(
+          "CREATE TABLE %s (id INT, data VARCHAR,d DOUBLE) PARTITIONED BY (data) WITH ('format-version'='2', 'write.format.default'='%s')",
+          TABLE_NAME, format.name());
+

Review Comment:
   nit: no need for empty line



##########
flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java:
##########
@@ -295,6 +299,161 @@ private static void assertEquals(
     }
   }
 
+  public static void assertEqualsSafe(
+      Schema schema, List<GenericData.Record> recs, List<Row> rows) {
+    Streams.forEachPair(
+        recs.stream(), rows.stream(), (rec, row) -> assertEqualsSafe(schema, rec, row));
+  }
+
+  public static void assertEqualsSafe(Schema schema, GenericData.Record rec, Row row) {
+    List<Types.NestedField> fields = schema.asStruct().fields();
+    RowType rowType = FlinkSchemaUtil.convert(schema);
+    for (int i = 0; i < fields.size(); i += 1) {
+      Type fieldType = fields.get(i).type();
+      Object expectedValue = rec.get(i);
+      Object actualValue = row.getField(i);
+      LogicalType logicalType = rowType.getTypeAt(i);
+      assertAvroEquals(fieldType, logicalType, expectedValue, actualValue);
+    }
+  }
+
+  private static void assertEqualsSafe(Types.StructType struct, GenericData.Record rec, Row row) {
+    List<Types.NestedField> fields = struct.fields();
+    for (int i = 0; i < fields.size(); i += 1) {
+      Type fieldType = fields.get(i).type();
+      Object expectedValue = rec.get(i);
+      Object actualValue = row.getField(i);
+      assertAvroEquals(fieldType, null, expectedValue, actualValue);
+    }
+  }
+
+  private static void assertAvroEquals(
+      Type type, LogicalType logicalType, Object expected, Object actual) {
+
+    if (expected == null && actual == null) {
+      return;
+    }
+
+    Assert.assertTrue(
+        "expected and actual should be both null or not null", expected != null && actual != null);
+
+    switch (type.typeId()) {
+      case BOOLEAN:
+        Assert.assertEquals("boolean value should be equal", expected, actual);
+        break;
+      case INTEGER:
+        Assert.assertEquals("int value should be equal", expected, actual);
+        break;
+      case LONG:
+        Assert.assertEquals("long value should be equal", expected, actual);
+        break;
+      case FLOAT:
+        Assert.assertEquals("float value should be equal", expected, actual);
+        break;
+      case DOUBLE:
+        Assert.assertEquals("double value should be equal", expected, actual);
+        break;
+      case STRING:
+        Assertions.assertThat(expected)
+            .as("Should expect a CharSequence")
+            .isInstanceOf(CharSequence.class);
+        Assert.assertEquals("string should be equal", String.valueOf(expected), actual.toString());
+        break;
+      case DATE:
+        Assertions.assertThat(expected).as("Should expect a Date").isInstanceOf(LocalDate.class);

Review Comment:
   @hililiwei  not sure how we are going to test the `assertEquals` change in this PR. For the test data generator PR #6377 , the internal version also has Avro GenericRecord support. I removed it in PR #6377 since I thought upstream doesn't need Avro Record. Should I add Avro GenericRecord to PR #6377 ?



##########
flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java:
##########
@@ -295,6 +299,161 @@ private static void assertEquals(
     }
   }
 
+  public static void assertEqualsSafe(
+      Schema schema, List<GenericData.Record> recs, List<Row> rows) {
+    Streams.forEachPair(
+        recs.stream(), rows.stream(), (rec, row) -> assertEqualsSafe(schema, rec, row));
+  }
+
+  public static void assertEqualsSafe(Schema schema, GenericData.Record rec, Row row) {
+    List<Types.NestedField> fields = schema.asStruct().fields();
+    RowType rowType = FlinkSchemaUtil.convert(schema);
+    for (int i = 0; i < fields.size(); i += 1) {
+      Type fieldType = fields.get(i).type();
+      Object expectedValue = rec.get(i);
+      Object actualValue = row.getField(i);
+      LogicalType logicalType = rowType.getTypeAt(i);
+      assertAvroEquals(fieldType, logicalType, expectedValue, actualValue);
+    }
+  }
+
+  private static void assertEqualsSafe(Types.StructType struct, GenericData.Record rec, Row row) {
+    List<Types.NestedField> fields = struct.fields();
+    for (int i = 0; i < fields.size(); i += 1) {
+      Type fieldType = fields.get(i).type();
+      Object expectedValue = rec.get(i);
+      Object actualValue = row.getField(i);
+      assertAvroEquals(fieldType, null, expectedValue, actualValue);
+    }
+  }
+
+  private static void assertAvroEquals(
+      Type type, LogicalType logicalType, Object expected, Object actual) {
+
+    if (expected == null && actual == null) {
+      return;
+    }
+
+    Assert.assertTrue(
+        "expected and actual should be both null or not null", expected != null && actual != null);
+
+    switch (type.typeId()) {
+      case BOOLEAN:
+        Assert.assertEquals("boolean value should be equal", expected, actual);
+        break;
+      case INTEGER:
+        Assert.assertEquals("int value should be equal", expected, actual);
+        break;
+      case LONG:
+        Assert.assertEquals("long value should be equal", expected, actual);
+        break;
+      case FLOAT:
+        Assert.assertEquals("float value should be equal", expected, actual);
+        break;
+      case DOUBLE:
+        Assert.assertEquals("double value should be equal", expected, actual);
+        break;
+      case STRING:
+        Assertions.assertThat(expected)
+            .as("Should expect a CharSequence")
+            .isInstanceOf(CharSequence.class);
+        Assert.assertEquals("string should be equal", String.valueOf(expected), actual.toString());

Review Comment:
   nit: `String.valueOf(expected)` -> `expected.toString()`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] stevenzwu commented on pull request #6222: Flink: Support inspecting table

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on PR #6222:
URL: https://github.com/apache/iceberg/pull/6222#issuecomment-1382153049

   thanks @hililiwei for contributing this major feature


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] hililiwei commented on a diff in pull request #6222: Flink: Support inspecting table

Posted by GitBox <gi...@apache.org>.
hililiwei commented on code in PR #6222:
URL: https://github.com/apache/iceberg/pull/6222#discussion_r1039683554


##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java:
##########
@@ -0,0 +1,340 @@
+/*
+ * 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.Duration;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.OffsetDateTime;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.function.BiConsumer;
+import java.util.function.Function;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericMapData;
+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.flink.types.RowKind;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+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.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.ByteBuffers;
+
+public class StructRowData implements RowData {
+  private final Types.StructType type;
+  private RowKind kind;
+  private StructLike struct;
+
+  public StructRowData(Types.StructType type) {
+    this(type, RowKind.INSERT);
+  }
+
+  public StructRowData(Types.StructType type, RowKind kind) {
+    this(type, null, kind);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct) {
+    this(type, struct, RowKind.INSERT);
+  }
+
+  private StructRowData(Types.StructType type, StructLike struct, RowKind kind) {
+    this.type = type;
+    this.struct = struct;
+    this.kind = kind;
+  }
+
+  public StructRowData setStruct(StructLike newStruct) {
+    this.struct = newStruct;
+    return this;
+  }
+
+  @Override
+  public int getArity() {
+    return struct.size();
+  }
+
+  @Override
+  public RowKind getRowKind() {
+    return kind;
+  }
+
+  @Override
+  public void setRowKind(RowKind newKind) {
+    Preconditions.checkNotNull(newKind, "kind can not be null");
+    this.kind = newKind;
+  }
+
+  @Override
+  public boolean isNullAt(int pos) {
+    return struct.get(pos, Object.class) == null;
+  }
+
+  @Override
+  public boolean getBoolean(int pos) {
+    return struct.get(pos, Boolean.class);
+  }
+
+  @Override
+  public byte getByte(int pos) {
+    return (byte) (int) struct.get(pos, Integer.class);
+  }
+
+  @Override
+  public short getShort(int pos) {
+    return (short) (int) struct.get(pos, Integer.class);
+  }
+
+  @Override
+  public int getInt(int pos) {
+    Object integer = struct.get(pos, Object.class);
+
+    if (integer instanceof Integer) {
+      return (int) integer;
+    } else if (integer instanceof LocalDate) {
+      return (int) ((LocalDate) integer).toEpochDay();
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for int field. Type name: " + integer.getClass().getName());
+    }
+  }
+
+  @Override
+  public long getLong(int pos) {
+    Object longVal = struct.get(pos, Object.class);
+
+    if (longVal instanceof Long) {
+      return (long) longVal;
+    } else if (longVal instanceof OffsetDateTime) {
+      return Duration.between(Instant.EPOCH, (OffsetDateTime) longVal).toNanos() / 1000;
+    } else if (longVal instanceof LocalDate) {
+      return ((LocalDate) longVal).toEpochDay();
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for long field. Type name: " + longVal.getClass().getName());
+    }
+  }
+
+  @Override
+  public float getFloat(int pos) {
+    return struct.get(pos, Float.class);
+  }
+
+  @Override
+  public double getDouble(int pos) {
+    return struct.get(pos, Double.class);
+  }
+
+  @Override
+  public StringData getString(int pos) {
+    return isNullAt(pos) ? null : getStringDataInternal(pos);
+  }
+
+  private StringData getStringDataInternal(int pos) {
+    CharSequence seq = struct.get(pos, CharSequence.class);
+    return StringData.fromString(seq.toString());
+  }
+
+  @Override
+  public DecimalData getDecimal(int pos, int precision, int scale) {
+    return isNullAt(pos)
+        ? null
+        : DecimalData.fromBigDecimal(getDecimalInternal(pos), precision, scale);
+  }
+
+  private BigDecimal getDecimalInternal(int pos) {
+    return struct.get(pos, BigDecimal.class);
+  }
+
+  @Override
+  public TimestampData getTimestamp(int pos, int precision) {
+    long timeLong = getLong(pos);
+    if (precision == 6) {
+      int nanosOfMillisecond = (int) (timeLong % 1000);
+      return TimestampData.fromEpochMillis(timeLong / 1000, nanosOfMillisecond);
+    } else {
+      return TimestampData.fromEpochMillis(timeLong);
+    }
+  }
+
+  @Override
+  public <T> RawValueData<T> getRawValue(int pos) {
+    return RawValueData.fromBytes(getBinary(pos));
+  }
+
+  @Override
+  public byte[] getBinary(int pos) {
+    return isNullAt(pos) ? null : getBinaryInternal(pos);
+  }
+
+  private byte[] getBinaryInternal(int pos) {
+    Object bytes = struct.get(pos, Object.class);
+
+    // should only be either ByteBuffer or byte[]
+    if (bytes instanceof ByteBuffer) {
+      return ByteBuffers.toByteArray((ByteBuffer) bytes);
+    } else if (bytes instanceof byte[]) {
+      return (byte[]) bytes;
+    } else {
+      throw new IllegalStateException(
+          "Unknown type for binary field. Type name: " + bytes.getClass().getName());
+    }
+  }
+
+  @Override
+  public ArrayData getArray(int pos) {
+    return isNullAt(pos) ? null : getArrayInternal(pos);
+  }
+
+  private ArrayData getArrayInternal(int pos) {
+    return collectionToArrayData(
+        type.fields().get(pos).type().asListType().elementType(),
+        struct.get(pos, Collection.class));
+  }
+
+  @Override
+  public MapData getMap(int pos) {
+    return isNullAt(pos) ? null : getMapInternal(pos);
+  }
+
+  private MapData getMapInternal(int pos) {
+    return mapToMapData(type.fields().get(pos).type().asMapType(), struct.get(pos, Map.class));
+  }
+
+  @Override
+  public RowData getRow(int pos, int numFields) {
+    return isNullAt(pos) ? null : getStructRowData(pos, numFields);
+  }
+
+  private StructRowData getStructRowData(int ordinal, int numFields) {
+    return new StructRowData(
+        type.fields().get(ordinal).type().asStructType(), struct.get(ordinal, StructLike.class));
+  }
+
+  private MapData mapToMapData(Types.MapType mapType, Map<?, ?> map) {
+    // make a defensive copy to ensure entries do not change

Review Comment:
   This ensures that it does not change when the  sub-query has a complex transformation, such as `select map['key']+ 1, map['key']+2`. Please correct me if I am mistaken.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] hililiwei commented on a diff in pull request #6222: Flink: Support inspecting table

Posted by GitBox <gi...@apache.org>.
hililiwei commented on code in PR #6222:
URL: https://github.com/apache/iceberg/pull/6222#discussion_r1042019390


##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java:
##########
@@ -104,11 +105,38 @@ public static Schema convert(Schema baseSchema, TableSchema flinkSchema) {
     Types.StructType struct = convert(flinkSchema).asStruct();
     // reassign ids to match the base schema
     Schema schema = TypeUtil.reassignIds(new Schema(struct.fields()), baseSchema);
+    // reassign doc to match the base schema
+    schema = reassignDoc(schema, baseSchema);

Review Comment:
   The reason I added this logic is:
   https://github.com/apache/iceberg/pull/5376/files#diff-db9787684ca50d6d4fdfe8ee927613d2e8e5c29eaa894535d39147b4f631cb17R152-R157
   
   If the doc of the field is different, it will not pass the validation. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] stevenzwu commented on a diff in pull request #6222: Flink: Support inspecting table

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on code in PR #6222:
URL: https://github.com/apache/iceberg/pull/6222#discussion_r1042761243


##########
core/src/main/java/org/apache/iceberg/BaseFilesTable.java:
##########
@@ -223,34 +225,28 @@ ManifestFile manifest() {
   static class ContentFileStructWithMetrics implements StructLike {
     private final StructLike fileAsStruct;
     private final MetricsUtil.ReadableMetricsStruct readableMetrics;
-    private final int expectedSize;
+    private final int position;
 
     ContentFileStructWithMetrics(
-        int expectedSize,
-        StructLike fileAsStruct,
-        MetricsUtil.ReadableMetricsStruct readableMetrics) {
+        int position, StructLike fileAsStruct, MetricsUtil.ReadableMetricsStruct readableMetrics) {
       this.fileAsStruct = fileAsStruct;
       this.readableMetrics = readableMetrics;
-      this.expectedSize = expectedSize;
+      this.position = position;
     }
 
     @Override
     public int size() {
-      return expectedSize;
+      return position;
     }
 
     @Override
     public <T> T get(int pos, Class<T> javaClass) {
-      int lastExpectedIndex = expectedSize - 1;
-      if (pos < lastExpectedIndex) {
+      if (pos < position) {
         return fileAsStruct.get(pos, javaClass);
-      } else if (pos == lastExpectedIndex) {
+      } else if (pos == position) {
         return javaClass.cast(readableMetrics);
       } else {
-        throw new IllegalArgumentException(

Review Comment:
   I think we should just add a Preconditions check
   ```
   Preconditions.check(pos < expectedSize, ...)
   if (pos == metricsPosition) {
     return javaClass.cast...
   } else {
    return  fileAsStruct.get(pos, javaClass);
   }
   ```
   
   Then just check if the (pos == metricsPos



##########
core/src/main/java/org/apache/iceberg/BaseFilesTable.java:
##########
@@ -223,34 +225,28 @@ ManifestFile manifest() {
   static class ContentFileStructWithMetrics implements StructLike {
     private final StructLike fileAsStruct;
     private final MetricsUtil.ReadableMetricsStruct readableMetrics;
-    private final int expectedSize;
+    private final int position;
 
     ContentFileStructWithMetrics(
-        int expectedSize,
-        StructLike fileAsStruct,
-        MetricsUtil.ReadableMetricsStruct readableMetrics) {
+        int position, StructLike fileAsStruct, MetricsUtil.ReadableMetricsStruct readableMetrics) {
       this.fileAsStruct = fileAsStruct;
       this.readableMetrics = readableMetrics;
-      this.expectedSize = expectedSize;
+      this.position = position;
     }
 
     @Override
     public int size() {
-      return expectedSize;
+      return position;
     }
 
     @Override
     public <T> T get(int pos, Class<T> javaClass) {
-      int lastExpectedIndex = expectedSize - 1;
-      if (pos < lastExpectedIndex) {
+      if (pos < position) {
         return fileAsStruct.get(pos, javaClass);
-      } else if (pos == lastExpectedIndex) {
+      } else if (pos == position) {
         return javaClass.cast(readableMetrics);
       } else {
-        throw new IllegalArgumentException(

Review Comment:
   I think we should just add a Preconditions check
   ```
   Preconditions.check(pos < expectedSize, ...)
   if (pos == metricsPosition) {
     return javaClass.cast...
   } else {
    return  fileAsStruct.get(pos, javaClass);
   }
   ```
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] stevenzwu commented on a diff in pull request #6222: Flink: Support inspecting table

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on code in PR #6222:
URL: https://github.com/apache/iceberg/pull/6222#discussion_r1042762795


##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java:
##########
@@ -140,15 +142,25 @@ public Catalog catalog() {
     return icebergCatalog;
   }
 
-  private Namespace toNamespace(String database) {
+  /** Append a new level to the base namespace */
+  private static Namespace appendLevel(Namespace baseNamespace, String newLevel) {
     String[] namespace = new String[baseNamespace.levels().length + 1];
     System.arraycopy(baseNamespace.levels(), 0, namespace, 0, baseNamespace.levels().length);
-    namespace[baseNamespace.levels().length] = database;
+    namespace[baseNamespace.levels().length] = newLevel;
     return Namespace.of(namespace);
   }
 
   TableIdentifier toIdentifier(ObjectPath path) {
-    return TableIdentifier.of(toNamespace(path.getDatabaseName()), path.getObjectName());
+    String objectName = path.getObjectName();
+    List<String> tableName = Splitter.on('$').splitToList(objectName);
+    if (tableName.size() == 2 && MetadataTableType.from(tableName.get(1)) != null) {

Review Comment:
   after split, the list should have size of either 1 (regular table without $ char) or 2 (metadata table with one $ char). `$` shouldn't be a valid char in other names.
   
   if the list size is 3 or any other numbers, it should be an illegal case.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] hililiwei commented on a diff in pull request #6222: Flink: Support inspecting table

Posted by GitBox <gi...@apache.org>.
hililiwei commented on code in PR #6222:
URL: https://github.com/apache/iceberg/pull/6222#discussion_r1042975474


##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java:
##########
@@ -104,11 +105,38 @@ public static Schema convert(Schema baseSchema, TableSchema flinkSchema) {
     Types.StructType struct = convert(flinkSchema).asStruct();
     // reassign ids to match the base schema
     Schema schema = TypeUtil.reassignIds(new Schema(struct.fields()), baseSchema);
+    // reassign doc to match the base schema
+    schema = reassignDoc(schema, baseSchema);

Review Comment:
   `reassignDoc`  doesn't seem like a common problem for other engines, so I just put it in flink. But I think it's okay to put it in TypeUtil.
   
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] pvary commented on pull request #6222: Flink: Support inspecting table

Posted by GitBox <gi...@apache.org>.
pvary commented on PR #6222:
URL: https://github.com/apache/iceberg/pull/6222#issuecomment-1335610198

   > > Do we want to introduce a different naming convention for Flink? How widespread is the usage of the SELECT * FROM prod.db.table$history like SQL?
   > 
   > @pvary, thank you for your feedback. I personally like `SELECT * FROM prod.db.table.history` too, but unfortunately, flink does not support this syntax, and it always discards `.history`. I tried for a long time and couldn't get it to work properly. Eventually, I had to choose the same way as flink-table-store.
   
   Thanks @hililiwei!


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] pvary commented on pull request #6222: Flink: Support inspecting table

Posted by GitBox <gi...@apache.org>.
pvary commented on PR #6222:
URL: https://github.com/apache/iceberg/pull/6222#issuecomment-1336098519

   Missclicked 😢 
   
   What I was trying to comment:
   Sadly I don't have enough time to properly review the PR, but I remember that we had issues with the partition evolution in the past when @szlta tested the Hive metadata tables. IIRC the issues surfaced when we changed from (a, b) partitioning to (a, c).
   See the relevant PRs: https://github.com/apache/iceberg/pulls?q=is%3Apr+szlta+partition
   
   The issues are handled on Core level, but it might worth to add some checks for edge cases too. Like:
   - Removed columns
   - Readded columns
   - Partition evolution
   - Schema evolution


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] stevenzwu commented on a diff in pull request #6222: Flink: Support inspecting table

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on code in PR #6222:
URL: https://github.com/apache/iceberg/pull/6222#discussion_r1039841272


##########
docs/flink-getting-started.md:
##########
@@ -712,9 +712,188 @@ INSERT INTO tableName /*+ OPTIONS('upsert-enabled'='true') */
 | compression-strategy   | Table write.orc.compression-strategy       | Overrides this table's compression strategy for ORC tables for this write                                  |
 
 
-## Inspecting tables.
+## Inspecting tables
 
-Iceberg does not support inspecting table in flink sql now, we need to use [iceberg's Java API](../api) to read iceberg's meta data to get those table information.
+To inspect a table's history, snapshots, and other metadata, Iceberg supports metadata tables.
+
+Metadata tables are identified by adding the metadata table name after the original table name. For example, history for `db.table` is read using `db.table$history`.
+
+### History
+
+To show table history:
+
+```sql
+SELECT * FROM prod.db.table$history;
+```
+
+| made_current_at         | snapshot_id         | parent_id           | is_current_ancestor |
+| ----------------------- | ------------------- | ------------------- | ------------------- |
+| 2019-02-08 03:29:51.215 | 5781947118336215154 | NULL                | true                |
+| 2019-02-08 03:47:55.948 | 5179299526185056830 | 5781947118336215154 | true                |
+| 2019-02-09 16:24:30.13  | 296410040247533544  | 5179299526185056830 | false               |
+| 2019-02-09 16:32:47.336 | 2999875608062437330 | 5179299526185056830 | true                |
+| 2019-02-09 19:42:03.919 | 8924558786060583479 | 2999875608062437330 | true                |
+| 2019-02-09 19:49:16.343 | 6536733823181975045 | 8924558786060583479 | true                |
+
+{{< hint info >}}
+**This shows a commit that was rolled back.** The example has two snapshots with the same parent, and one is *not* an ancestor of the current table state.

Review Comment:
   maybe more explicit?
   
   In this example, snapshot 296410040247533544 and 2999875608062437330 have the same parent snapshot 5179299526185056830. Snapshot 296410040247533544 was rolled back and is *not* an ancestor of the current table state.
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] stevenzwu commented on pull request #6222: Flink: Support inspecting table

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on PR #6222:
URL: https://github.com/apache/iceberg/pull/6222#issuecomment-1338683010

   @hililiwei we should add comprehensive unit test for `StructRowData`. 
   
   I have some internal DataGenerators for unit test code with very comprehensive coverage all field types (including complex nested types). Maybe I will submit a separate PR for that, which will cover Flink `RowData` and Iceberg `GenericRecord`. You can expand it with support for Iceberg `StructLike`. With that, we can write unit test/assertions compare the actual and expected.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] stevenzwu commented on a diff in pull request #6222: Flink: Support inspecting table

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on code in PR #6222:
URL: https://github.com/apache/iceberg/pull/6222#discussion_r1043891097


##########
core/src/main/java/org/apache/iceberg/BaseFilesTable.java:
##########
@@ -223,34 +225,28 @@ ManifestFile manifest() {
   static class ContentFileStructWithMetrics implements StructLike {
     private final StructLike fileAsStruct;
     private final MetricsUtil.ReadableMetricsStruct readableMetrics;
-    private final int expectedSize;
+    private final int position;
 
     ContentFileStructWithMetrics(
-        int expectedSize,
-        StructLike fileAsStruct,
-        MetricsUtil.ReadableMetricsStruct readableMetrics) {
+        int position, StructLike fileAsStruct, MetricsUtil.ReadableMetricsStruct readableMetrics) {
       this.fileAsStruct = fileAsStruct;
       this.readableMetrics = readableMetrics;
-      this.expectedSize = expectedSize;
+      this.position = position;
     }
 
     @Override
     public int size() {
-      return expectedSize;
+      return position;
     }
 
     @Override
     public <T> T get(int pos, Class<T> javaClass) {
-      int lastExpectedIndex = expectedSize - 1;
-      if (pos < lastExpectedIndex) {
+      if (pos < position) {
         return fileAsStruct.get(pos, javaClass);
-      } else if (pos == lastExpectedIndex) {
+      } else if (pos == position) {
         return javaClass.cast(readableMetrics);
       } else {
-        throw new IllegalArgumentException(

Review Comment:
   this else condition is essentially where `pos > metricsPosition`. why do we need to do ` - 1`  in `fileAsStruct.get(pos - 1, javaClass);`? why not just `fileAsStruct.get(pos, javaClass);`
   
   Isn't this logic simpler and sufficient?
   ```
   if (pos == metricsPosition) {
     return javaClass.cast...
   } else {
    return  fileAsStruct.get(pos, javaClass);
   }
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] stevenzwu commented on a diff in pull request #6222: Flink: Support inspecting table

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on code in PR #6222:
URL: https://github.com/apache/iceberg/pull/6222#discussion_r1043887176


##########
core/src/main/java/org/apache/iceberg/BaseFilesTable.java:
##########
@@ -200,13 +202,14 @@ private CloseableIterable<? extends ContentFile<?>> files(Schema fileProjection)
       }
     }
 
-    private StructLike withReadableMetrics(ContentFile<?> file) {
-      int expectedSize = projection.columns().size();
+    private StructLike withReadableMetrics(ContentFile<?> file, int metricsPosition) {
+      int columnSize = projection.columns().size();

Review Comment:
   nit: `fieldCount` or `columnCount` is probably more clear. `columnSize` can be read as the size of one particular column.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] stevenzwu commented on a diff in pull request #6222: Flink: Support inspecting table

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on code in PR #6222:
URL: https://github.com/apache/iceberg/pull/6222#discussion_r1043891097


##########
core/src/main/java/org/apache/iceberg/BaseFilesTable.java:
##########
@@ -223,34 +225,28 @@ ManifestFile manifest() {
   static class ContentFileStructWithMetrics implements StructLike {
     private final StructLike fileAsStruct;
     private final MetricsUtil.ReadableMetricsStruct readableMetrics;
-    private final int expectedSize;
+    private final int position;
 
     ContentFileStructWithMetrics(
-        int expectedSize,
-        StructLike fileAsStruct,
-        MetricsUtil.ReadableMetricsStruct readableMetrics) {
+        int position, StructLike fileAsStruct, MetricsUtil.ReadableMetricsStruct readableMetrics) {
       this.fileAsStruct = fileAsStruct;
       this.readableMetrics = readableMetrics;
-      this.expectedSize = expectedSize;
+      this.position = position;
     }
 
     @Override
     public int size() {
-      return expectedSize;
+      return position;
     }
 
     @Override
     public <T> T get(int pos, Class<T> javaClass) {
-      int lastExpectedIndex = expectedSize - 1;
-      if (pos < lastExpectedIndex) {
+      if (pos < position) {
         return fileAsStruct.get(pos, javaClass);
-      } else if (pos == lastExpectedIndex) {
+      } else if (pos == position) {
         return javaClass.cast(readableMetrics);
       } else {
-        throw new IllegalArgumentException(

Review Comment:
   this else condition is essentially where `pos > metricsPosition`. why do we need to do minus ` of `fileAsStruct.get(pos - 1, javaClass);`? why not just `fileAsStruct.get(pos, javaClass);`
   
   Isn't this logic simpler and sufficient?
   ```
   if (pos == metricsPosition) {
     return javaClass.cast...
   } else {
    return  fileAsStruct.get(pos, javaClass);
   }
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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