Skip to content

feat: support V0 iceberg_tables schema for SqlCatalog #2380

Open
rchowell wants to merge 2 commits intoapache:mainfrom
rchowell:iceberg-type
Open

feat: support V0 iceberg_tables schema for SqlCatalog #2380
rchowell wants to merge 2 commits intoapache:mainfrom
rchowell:iceberg-type

Conversation

@rchowell
Copy link
Copy Markdown
Contributor

@rchowell rchowell commented Apr 28, 2026

Which issue does this PR close?

What changes are included in this PR?

This PR adds support for using a V0 SqlCatalog from other implementations like iceberg-python or iceberg-java, and it follows the iceberg-java behavior of checking an explicit schema-version property and migrating to V1 only-if the user requested this. I also added logging at the same level as iceberg-java.

Probe to see if we have a V0 or V1 table, then add the iceberg_type column if it does not exist. More details in apache/iceberg-python#3263

Are these changes tested?

  • Unit tests with migration path

Comment thread crates/catalog/sql/src/catalog.rs Outdated
@rchowell rchowell changed the title fix: automatically migrate V0 SqlCatalog catalog tables feat: support V0 iceberg_tables schema for SqlCatalog May 1, 2026
@rchowell
Copy link
Copy Markdown
Contributor Author

rchowell commented May 1, 2026

Ok @blackmwk thanks again for reviewing, I was mistaken and have fixed this PR to match iceberg-java much more closely. I also updated the PR title and description to be more accurate since this is more of a feature to support reading V0 tables in iceberg-rust. Thanks!

Comment on lines 43 to 46
pub const SQL_CATALOG_PROP_BIND_STYLE: &str = "sql_bind_style";
/// catalog schema version, setting to "V1" will migrate from V0 to V1 schema
pub const SQL_CATALOG_PROP_SCHEMA_VERSION: &str = "sql.schema-version";

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Seeing these together makes me think sql_schema_version yet iceberg-java has jdbc.schema-version. I do prefer consistency for something like sql.<kebab>.

))
.execute(&pool)
.await
.is_ok();
Copy link
Copy Markdown
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

What if the SELECT failed here but the schema is actually V1?

I assume in that case it would pass silently here and fail later when trying to update it. This would be confusing to users

With above said, I don't have a better idea to probe the schema type... at least we should check the error type? any thoughts?

let tbl_metadata_location_str = tbl_metadata_location.to_string();
self.execute(&format!(
"INSERT INTO {CATALOG_TABLE_NAME}
({CATALOG_FIELD_CATALOG_NAME}, {CATALOG_FIELD_TABLE_NAMESPACE}, {CATALOG_FIELD_TABLE_NAME}, {CATALOG_FIELD_METADATA_LOCATION_PROP}, {CATALOG_FIELD_RECORD_TYPE})
Copy link
Copy Markdown
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Should we consider V0 schema here as well?

Copy link
Copy Markdown
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

In iceberg-java, it looks like we only support creating V0 table. So I think it's fine for us to leave it here.

But still quite curious about the case when TableCreation has V1 schema.

https://github.com/apache/iceberg/blob/bb37293484468f0502de9986955860505aef9776/core/src/main/java/org/apache/iceberg/jdbc/JdbcUtil.java#L128

let requested: SchemaVersion = config
.props
.get(SQL_CATALOG_PROP_SCHEMA_VERSION)
.and_then(|v| v.parse().ok())
Copy link
Copy Markdown
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We should fail on parse error here to avoid silent fallback for misinputs

Similar to sql_bind_style parsing: https://github.com/apache/iceberg-rust/blob/main/crates/catalog/sql/src/catalog.rs#L183

//! };
//!
//! #[tokio::main]
//! #[tokio::main(flavor = "current_thread")]
Copy link
Copy Markdown
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

why do we need to change this?

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

column "iceberg_type" does not exist

3 participants