-
Notifications
You must be signed in to change notification settings - Fork 350
Description
Apache Iceberg version
0.9.1 (latest release)
Please describe the bug 🐞
I've been experimenting with the latest pyiceberg version, 0.10.0 on master b/c I was having some other issue on 0.9.1, and have been getting issues committing table updates. For context I'm using Ray Data as my data engine (Ray Data uses pyIceberg internally), Google BigLake as my REST catalog, and Google Storage as a warehouse.
My code looks something like this:
catalog_kwargs = {
"type": "rest",
"uri": "https://biglake.googleapis.com/iceberg/v1beta/restcatalog",
"warehouse": "gs://some-warehouse-bucket",
"header.x-goog-user-project": "my-project",
"header.X-Iceberg-Access-Delegation": "remote-signing",
"auth": {
"type": "google",
},
}
# load the dataset
dataset = ray.data.read_parquet("gs://some-data-bucket/some-file.parquet")
# hack to get iceberg-compatible schema
refs = dataset.to_arrow_refs()
schema = ray.get(refs[0]).schema
# write
table_identifier = "test_namespace.test_data"
# create the table in iceberg
catalog.create_table_if_not_exists(table_identifier, schema=schema)
dataset.write_iceberg(table_identifier=table_identifier, catalog_kwargs=catalog_kwargs)
The series of requests that Ray Data makes are to:
- Get the table metadata
- Write data to the warehouse
- Commit updates to the table
The issue comes in step 3 where I get a INVALID_ARGUMENT
error from BigLake. I traced the calls and the last payload looks like:
{
"identifier":{
"namespace":["test_namespace"],
"name":"test_data"
},
"requirements":[
{"type":"assert-ref-snapshot-id","ref":"main"},
{"type":"assert-table-uuid","uuid":"689f1a7d-0000-2589-aca7-d4f547fce244"}
],
"updates":[ ... ]
}
The assert-ref-snapshot-id
requirement is missing the snapshot-id
field which should be set to null
, not absent from the requirement. We contacted BigLake/BigQuery support and they said their parser expects that key to exist. The openAPI REST catalog spec says that snapshot-id
is required.
I believe this means that the pyIceberg client implementation here does not meet the spec.
Investigation
Looking into what it would take to make this field serialize to null
in json I:
- made a sample class that just subclasses
BaseModel
and leaves out thevalidate
method - set
exclude=False
on the field and calledmodel_dump_json()
to see the output - it printed
"snapshot-id": null
!
However this experiment fell short b/c all of the pyiceberg assert models subclass ValidatableTableRequirement
, which subclasses IcebergBaseModel
, which overrides the model_dump_json
method to default exclude_none=True
.
Willingness to contribute
- I can contribute a fix for this bug independently
- I would be willing to contribute a fix for this bug with guidance from the Iceberg community
- I cannot contribute a fix for this bug at this time