Skip to content

Commit

Permalink
iceberg/compat: Force new and promoted fields to nullable
Browse files Browse the repository at this point in the history
We don't allow adding required fields to a struct, but a "required" field in
iceberg land doesn't precisely map to a "required" field in avro/proto land.

Instead of trying to smash the two abstractions together, we can basically force
any and all fields in a schema diff into nullability, rather than rejecting
such changes outright. This will not affect how the core serialization rules
are applied by the schema language, but it allows us to work around language
idiosyncrasies without adding much complexity.

e.g. In Avro, the only way to add an "optional" field is to add a _required_
union field, which becomes a struct with two optional subfields and a defualt
value. If we transform one of these into a non-required struct w/ a null
default, the Avro stack can parse new records in the correct way while a
conformant query engine can treat the row as nullable, thereby rendering an
otherwise compat-breaking schema change backwards compatible.
  • Loading branch information
oleiman committed Jan 17, 2025
1 parent 624e02c commit 7d0f340
Showing 1 changed file with 6 additions and 2 deletions.
8 changes: 6 additions & 2 deletions src/v/iceberg/compatibility.cc
Original file line number Diff line number Diff line change
Expand Up @@ -353,7 +353,9 @@ struct validate_transform_visitor {
if (
src.required == field_required::no
&& f_->required == field_required::yes) {
return schema_evolution_errc::new_required_field;
// force the field nullable. we can just treat these columns as
// nullable for the purposes of iceberg schema accounting
f_->required = field_required::no;
} else if (src.required != f_->required) {
promoted = true;
}
Expand All @@ -364,7 +366,9 @@ struct validate_transform_visitor {

schema_errc_result operator()(const nested_field::is_new&) {
if (f_->required == field_required::yes) {
return schema_evolution_errc::new_required_field;
// force the field nullable. we can just treat these columns as
// nullable for the purposes of iceberg schema accounting
f_->required = field_required::no;
}
*state_ += schema_transform_state{.n_added = 1};
return std::nullopt;
Expand Down

0 comments on commit 7d0f340

Please sign in to comment.