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


##########
flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkRowData.java:
##########
@@ -0,0 +1,42 @@
+/*
+ * 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;
+
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.logical.LogicalType;
+
+public class FlinkRowData {
+
+  private FlinkRowData() {}
+
+  public static RowData.FieldGetter createFieldGetter(LogicalType fieldType, 
int fieldPos) {
+    RowData.FieldGetter flinkFieldGetter = 
RowData.createFieldGetter(fieldType, fieldPos);
+    return rowData -> {
+      // Be sure to check for null values, even if the field is required. Flink
+      // RowData.createFieldGetter(..) does not null-check optional / nullable 
types. Without this
+      // explicit null check, the null flag of BinaryRowData will be ignored 
and random bytes will
+      // be parsed as actual values. This will produce incorrect writes 
instead of failing with a
+      // NullPointerException.
+      if (!fieldType.isNullable() && rowData.isNullAt(fieldPos)) {
+        return null;
+      }
+      return flinkFieldGetter.getFieldOrNull(rowData);
+    };
+  }

Review Comment:
   Yes, I think it could be seen as a bug. Filed a ticket here: 
https://issues.apache.org/jira/browse/FLINK-37245



-- 
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-unsubscr...@iceberg.apache.org

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


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

Reply via email to