kbendick commented on a change in pull request #3240:
URL: https://github.com/apache/iceberg/pull/3240#discussion_r733968859



##########
File path: 
flink/src/test/java/org/apache/iceberg/flink/data/TestRowDataProjection.java
##########
@@ -0,0 +1,332 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.flink.data;
+
+import java.util.Iterator;
+import org.apache.flink.table.data.RowData;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.data.RandomGenericData;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.flink.TestHelpers;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.StructProjection;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestRowDataProjection {
+
+  @Test
+  public void testFullProjection() {
+    Schema schema = new Schema(
+        Types.NestedField.required(0, "id", Types.LongType.get()),
+        Types.NestedField.optional(1, "data", Types.StringType.get())
+    );
+
+    generateAndValidate(schema, schema);
+  }
+
+  @Test
+  public void testReorderedFullProjection() {
+    Schema schema = new Schema(
+        Types.NestedField.required(0, "id", Types.LongType.get()),
+        Types.NestedField.optional(1, "data", Types.StringType.get())
+    );
+
+    Schema reordered = new Schema(
+        Types.NestedField.optional(1, "data", Types.StringType.get()),
+        Types.NestedField.required(0, "id", Types.LongType.get())
+    );
+
+    generateAndValidate(schema, reordered);
+  }
+
+  @Test
+  public void testBasicProjection() {
+    Schema schema = new Schema(
+        Types.NestedField.required(0, "id", Types.LongType.get()),
+        Types.NestedField.optional(1, "data", Types.StringType.get())
+    );
+    Schema id = new Schema(
+        Types.NestedField.required(0, "id", Types.LongType.get())
+    );
+    Schema data = new Schema(
+        Types.NestedField.optional(1, "data", Types.StringType.get())
+    );
+    generateAndValidate(schema, id);
+    generateAndValidate(schema, data);
+  }
+
+  @Test
+  public void testEmptyProjection() {
+    Schema schema = new Schema(
+        Types.NestedField.required(0, "id", Types.LongType.get()),
+        Types.NestedField.optional(1, "data", Types.StringType.get())
+    );
+    generateAndValidate(schema, schema.select());
+  }
+
+  @Test
+  public void testRename() {
+    Schema schema = new Schema(
+        Types.NestedField.required(0, "id", Types.LongType.get()),
+        Types.NestedField.optional(1, "data", Types.StringType.get())
+    );
+
+    Schema renamed = new Schema(
+        Types.NestedField.required(0, "id", Types.LongType.get()),
+        Types.NestedField.optional(1, "renamed", Types.StringType.get())
+    );
+    generateAndValidate(schema, renamed);
+  }
+
+  @Test
+  public void testNestedProjection() {
+    Schema schema = new Schema(
+        Types.NestedField.required(0, "id", Types.LongType.get()),
+        Types.NestedField.optional(3, "location", Types.StructType.of(
+            Types.NestedField.required(1, "lat", Types.FloatType.get()),
+            Types.NestedField.required(2, "long", Types.FloatType.get())
+        ))
+    );
+
+    // Project id only.
+    Schema idOnly = new Schema(
+        Types.NestedField.required(0, "id", Types.LongType.get())
+    );
+    generateAndValidate(schema, idOnly);
+
+    // Project lat only.
+    Schema latOnly = new Schema(
+        Types.NestedField.optional(3, "location", Types.StructType.of(
+            Types.NestedField.required(1, "lat", Types.FloatType.get())
+        ))
+    );
+    generateAndValidate(schema, latOnly);
+
+    // Project long only.
+    Schema longOnly = new Schema(
+        Types.NestedField.optional(3, "location", Types.StructType.of(
+            Types.NestedField.required(2, "long", Types.FloatType.get())
+        ))
+    );
+    generateAndValidate(schema, longOnly);
+
+    // Project location.
+    Schema locationOnly = schema.select("location");
+    generateAndValidate(schema, locationOnly);
+  }
+
+  @Test
+  public void testPrimitiveTypeProjection() {
+    Schema schema = new Schema(
+        Types.NestedField.required(0, "id", Types.LongType.get()),
+        Types.NestedField.optional(1, "data", Types.StringType.get()),
+        Types.NestedField.required(2, "b", Types.BooleanType.get()),
+        Types.NestedField.optional(3, "i", Types.IntegerType.get()),
+        Types.NestedField.required(4, "l", Types.LongType.get()),
+        Types.NestedField.optional(5, "f", Types.FloatType.get()),
+        Types.NestedField.required(6, "d", Types.DoubleType.get()),
+        Types.NestedField.optional(7, "date", Types.DateType.get()),
+        Types.NestedField.optional(8, "time", Types.TimeType.get()),
+        Types.NestedField.required(9, "ts", Types.TimestampType.withoutZone()),
+        Types.NestedField.required(10, "ts_tz", 
Types.TimestampType.withZone()),
+        Types.NestedField.required(11, "s", Types.StringType.get()),
+        Types.NestedField.required(12, "fixed", Types.FixedType.ofLength(7)),
+        Types.NestedField.optional(13, "bytes", Types.BinaryType.get()),
+        Types.NestedField.required(14, "dec_9_0", Types.DecimalType.of(9, 0)),
+        Types.NestedField.required(15, "dec_11_2", Types.DecimalType.of(11, 
2)),
+        Types.NestedField.required(16, "dec_38_10", Types.DecimalType.of(38, 
10))// maximum precision
+    );
+
+    generateAndValidate(schema, schema);
+  }
+
+  @Test
+  public void testPrimitiveMapTypeProjection() {
+    Schema schema = new Schema(
+        Types.NestedField.required(0, "id", Types.LongType.get()),
+        Types.NestedField.optional(3, "map", Types.MapType.ofOptional(
+            1, 2, Types.IntegerType.get(), Types.StringType.get()
+        ))
+    );
+
+    // Project id only.
+    Schema idOnly = schema.select("id");
+    generateAndValidate(schema, idOnly);
+
+    // Project map only.
+    Schema mapOnly = schema.select("map");
+    generateAndValidate(schema, mapOnly);
+
+    // Project all.
+    generateAndValidate(schema, schema);
+  }
+
+  @Test
+  public void testNestedMapTypeProjection() {
+    Schema schema = new Schema(
+        Types.NestedField.required(0, "id", Types.LongType.get()),
+        Types.NestedField.optional(7, "map", Types.MapType.ofOptional(
+            5, 6,
+            Types.StructType.of(
+                Types.NestedField.required(1, "key", Types.LongType.get()),
+                Types.NestedField.required(2, "keyData", Types.LongType.get())
+            ),
+            Types.StructType.of(
+                Types.NestedField.required(3, "value", Types.LongType.get()),
+                Types.NestedField.required(4, "valueData", 
Types.LongType.get())
+            )
+        ))
+    );
+
+    // Project id only.
+    Schema idOnly = schema.select("id");
+    generateAndValidate(schema, idOnly);
+
+    // Project map only.
+    Schema mapOnly = schema.select("map");
+    generateAndValidate(schema, mapOnly);
+
+    // Project all.
+    generateAndValidate(schema, schema);
+
+    // Project partial map key.
+    Schema partialMapKey = new Schema(
+        Types.NestedField.optional(7, "map", Types.MapType.ofOptional(
+            5, 6,
+            Types.StructType.of(
+                Types.NestedField.required(1, "key", Types.LongType.get())
+            ),
+            Types.StructType.of(
+                Types.NestedField.required(3, "value", Types.LongType.get()),
+                Types.NestedField.required(4, "valueData", 
Types.LongType.get())
+            )
+        ))
+    );
+    AssertHelpers.assertThrows("Should be error because cannot project a 
partial nested map key.",
+        IllegalArgumentException.class, "Cannot project a partial map key or 
value",
+        () -> generateAndValidate(schema, partialMapKey)
+    );
+
+    // Project partial map key.
+    Schema partialMapValue = new Schema(
+        Types.NestedField.optional(7, "map", Types.MapType.ofOptional(
+            5, 6,
+            Types.StructType.of(
+                Types.NestedField.required(1, "key", Types.LongType.get()),
+                Types.NestedField.required(2, "keyData", Types.LongType.get())
+            ),
+            Types.StructType.of(
+                Types.NestedField.required(3, "value", Types.LongType.get())
+            )
+        ))
+    );
+    AssertHelpers.assertThrows("Should be error because cannot project a 
partial nested map value.",
+        IllegalArgumentException.class, "Cannot project a partial map key or 
value",
+        () -> generateAndValidate(schema, partialMapValue)
+    );
+  }
+
+  @Test
+  public void testPrimitiveListTypeProjection() {
+    Schema schema = new Schema(
+        Types.NestedField.required(0, "id", Types.LongType.get()),
+        Types.NestedField.optional(2, "list", Types.ListType.ofOptional(
+            1, Types.StringType.get()
+        ))
+    );
+
+    // Project id only.
+    Schema idOnly = schema.select("id");
+    generateAndValidate(schema, idOnly);
+
+    // Project list only.
+    Schema mapOnly = schema.select("list");
+    generateAndValidate(schema, mapOnly);
+
+    // Project all.
+    generateAndValidate(schema, schema);
+  }
+
+  @Test
+  public void testNestedListTypeProjection() {
+    Schema schema = new Schema(
+        Types.NestedField.required(0, "id", Types.LongType.get()),
+        Types.NestedField.optional(5, "list", Types.ListType.ofOptional(
+            4, Types.StructType.of(
+                Types.NestedField.required(1, "nestedListField1", 
Types.LongType.get()),
+                Types.NestedField.required(2, "nestedListField2", 
Types.LongType.get()),
+                Types.NestedField.required(3, "nestedListField3", 
Types.LongType.get())
+            )
+        ))
+    );
+
+    // Project id only.
+    Schema idOnly = schema.select("id");
+    generateAndValidate(schema, idOnly);
+
+    // Project list only.
+    Schema mapOnly = schema.select("list");
+    generateAndValidate(schema, mapOnly);
+
+    // Project all.
+    generateAndValidate(schema, schema);
+
+    // Project partial list value.
+    Schema partialList = new Schema(
+        Types.NestedField.optional(5, "list", Types.ListType.ofOptional(
+            4, Types.StructType.of(
+                Types.NestedField.required(2, "nestedListField2", 
Types.LongType.get())
+            )
+        ))
+    );
+    AssertHelpers.assertThrows("Should be error because cannot project a 
partial nested list element.",

Review comment:
       Nit: This is a little confusing for me at first.
   
   Can we possibly rephrase this as `Should not all users to project onto a 
subset of fields of a struct used in a list type`? That would make what's being 
tested a bit more clear (at least for me) from the get go.

##########
File path: 
flink/src/main/java/org/apache/iceberg/flink/data/RowDataProjection.java
##########
@@ -0,0 +1,210 @@
+/*
+ * 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 org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.MapData;
+import org.apache.flink.table.data.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.table.types.logical.RowType;
+import org.apache.flink.types.RowKind;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.types.Types;
+
+public class RowDataProjection implements RowData {
+
+  private final RowData.FieldGetter[] getters;
+  private RowData rowData;
+
+  public static RowDataProjection create(Schema schema, Schema projectSchema) {
+    return new RowDataProjection(FlinkSchemaUtil.convert(schema), 
schema.asStruct(), projectSchema.asStruct());
+  }
+
+  private RowDataProjection(RowType rowType, Types.StructType rowStruct, 
Types.StructType projectType) {
+    this.getters = new RowData.FieldGetter[projectType.fields().size()];
+    for (int i = 0; i < getters.length; i++) {
+      getters[i] = createFieldGetter(rowType, rowStruct, 
projectType.fields().get(i));
+    }
+  }
+
+  private static RowData.FieldGetter createFieldGetter(RowType rowType,
+                                                       Types.StructType 
rowStruct,
+                                                       Types.NestedField 
projectField) {
+    for (int i = 0; i < rowStruct.fields().size(); i++) {
+      Types.NestedField rowField = rowStruct.fields().get(i);
+      if (rowField.fieldId() == projectField.fieldId()) {
+        Preconditions.checkArgument(rowField.type().typeId() == 
projectField.type().typeId(),
+            String.format("Different iceberg type between row field <%s> and 
project field <%s>",
+                rowField, projectField));
+
+        switch (projectField.type().typeId()) {
+          case STRUCT:
+            RowType nestedRowType = (RowType) rowType.getTypeAt(i);
+            int rowPos = i;
+            return row -> {
+              RowData nestedRow = row.isNullAt(rowPos) ? null : 
row.getRow(rowPos, nestedRowType.getFieldCount());
+              return new RowDataProjection(nestedRowType, 
rowField.type().asStructType(),
+                  projectField.type().asStructType()).wrap(nestedRow);
+            };
+
+          case MAP:
+            Types.MapType projectedMap = projectField.type().asMapType();
+            Types.MapType originalMap = rowField.type().asMapType();
+
+            boolean keyProjectable = !projectedMap.keyType().isNestedType() ||
+                projectedMap.keyType().equals(originalMap.keyType());
+            boolean valueProjectable = 
!projectedMap.valueType().isNestedType() ||
+                projectedMap.valueType().equals(originalMap.valueType());
+            Preconditions.checkArgument(keyProjectable && valueProjectable,
+                "Cannot project a partial map key or value RowData. Trying to 
project %s out of %s",
+                projectField, rowField);
+
+            return RowData.createFieldGetter(rowType.getTypeAt(i), i);
+
+          case LIST:
+            Types.ListType projectedList = projectField.type().asListType();
+            Types.ListType originalList = rowField.type().asListType();
+
+            boolean elementProjectable = 
!projectedList.elementType().isNestedType() ||
+                projectedList.elementType().equals(originalList.elementType());
+            Preconditions.checkArgument(elementProjectable,
+                "Cannot project a partial list element RowData. Trying to 
project %s out of %s",

Review comment:
       See note below about this exception message.

##########
File path: 
flink/src/main/java/org/apache/iceberg/flink/data/RowDataProjection.java
##########
@@ -0,0 +1,210 @@
+/*
+ * 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 org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.MapData;
+import org.apache.flink.table.data.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.table.types.logical.RowType;
+import org.apache.flink.types.RowKind;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.types.Types;
+
+public class RowDataProjection implements RowData {

Review comment:
       Nit: Can you add a Java Doc comment for this? Specifically, is this for 
all projections onto a subset of `RowData` or possibly a small doc comment 
indicating when this is used?
   
   I believe that the checkstyle will emit a warning as this class is public 
and has no doc comment. If it doesn't, I'll leave that up to you 🙂 




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

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]



---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to