mxm commented on PR #26100: URL: https://github.com/apache/flink/pull/26100#issuecomment-2643347536
>My intuitive understanding of the mentioned problem in Iceberg is why not fix the nullability of the field? In Iceberg, users can directly provide a stream of RowData, for which the precise schema is only known when writing the stream to the table. Once the table schema is available, Iceberg relies on the Flink FieldGetters to read values from the provided data, but it turns out that the result of the `FieldGetter#getFieldOrNull` cannot be trusted. When a `null` field returns an arbitrary value, depending what random bytes are currently stored for that field with the null flag being set, then the result is plainly wrong. > Accurate nullability is important to the sql system, not only to ensure proper processing on the consumer side (e.g., the field may be used as the primary key of the target table, which requires not null), but also gives the planner more optimization possibilities (e.g., predicate derivation, and also the runtime code can skip the null check overhead for non-null fields, etc.). Thanks for highlighting the value of the non-null concept. I'm definitely not trying to get rid of non-null, but we need to be able to trust the values stored in RowData when using FieldGetters. If there are lower levels that don't want that check, that is fine too, they can directly operate on RowData and not use the FieldGetters. What you say about skipping null checks is not in line with what Flink does. The Flink runtime performs null checks when reading from RowData, regardless of the nullability type. I believe the externally facing API of `RowData#createFieldGetter(..)` should do that as well. I think it would be interesting to check how Flink handles user-provided null values for non-null types. I've seen `table.exec.sink.not-null-enforcer` option, but that only applies to sinks. That wouldn't help in this case because the null value would have already been resurrected in case of a missing `isNullAt(..)` check. -- 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]
