delta-rs
delta-rs copied to clipboard
deltalake.PyDeltaTableError: Invalid JSON in log record: missing field `minValues` at line 1 column 16
Environment
Local Python + Azure Spark Delta-rs version: unkonwn Binding: Python (latest installation from pip) Environment:
- Cloud provider: Azure
- OS: Linux
- Other: Tested in Local Linux
Bug
What happened:
os.environ['AZURE_STORAGE_ACCOUNT_NAME'] = 'acctname'
os.environ['AZURE_STORAGE_ACCOUNT_KEY'] = 'key'
res_adls = "adls2://feathrazuretest3storage/feathrazuretest3fs/demo_data/output_44_8_deltalake"
from deltalake import DeltaTable
delta = DeltaTable(res_adls)
result_df = delta.to_pyarrow_table().to_pandas()
After the above lines were executed, the error will be like below:
File "/Users/xiazhu/Documents/GitHub/feathr/feathr_project/feathr/job_utils.py", line 39, in get_result_df
result_df = delta.to_pyarrow_table().to_pandas()
File "/opt/miniconda3/lib/python3.9/site-packages/deltalake/table.py", line 303, in to_pyarrow_table
return self.to_pyarrow_dataset(
File "/opt/miniconda3/lib/python3.9/site-packages/deltalake/table.py", line 284, in to_pyarrow_dataset
for file, part_expression in self._table.dataset_partitions(partitions)
deltalake.PyDeltaTableError: Invalid JSON in log record: missing field `minValues` at line 1 column 16
What you expected to happen: Function correctly
How to reproduce it: It should be reproduced easily. Account name and key can be provided at request.
More details: The result is generated by a spark job and can be loaded in spark correctly. However in delta-rs or the python binding, it was not loaded correctly
It sounds like Spark wrote a bad statistics in the Delta log. Two possible (and not mutually exclusive) solutions:
(1) If there is an error processing statistics, we could ignore the stats. (Maybe we have an option to error on bad stats, if people want that.) (2) We could adapt the statistics JSON schema in delta-rs to handle the behavior of this writer.
Do you have information on the version of Spark and the Delta writer this table was written with?
It sounds like Spark wrote a bad statistics in the Delta log. Two possible (and not mutually exclusive) solutions:
(1) If there is an error processing statistics, we could ignore the stats. (Maybe we have an option to error on bad stats, if people want that.) (2) We could adapt the statistics JSON schema in delta-rs to handle the behavior of this writer.
Do you have information on the version of Spark and the Delta writer this table was written with?
Thanks! https://docs.microsoft.com/en-us/azure/synapse-analytics/spark/apache-spark-3-runtime Here should have all the details on spark and Delta. Specifically, delta-core_2.12-1.0.0.2b.jar
and spark-core_2.12-3.1.2.5.0-50849917.jar
.
Let me know if you need anything else from me!
But the weird thing is Spark can read the table, so not sure if there are still stats errors
Thanks @xiaoyongzhu for the report :)
I think this is probably caused by an ambiguity in the delta spec. In https://github.com/delta-io/delta/blob/master/PROTOCOL.md#per-file-statistics, there is no mention of whether these stats fields are optional or not.
Right now all the stats fields are modeled as required in https://github.com/delta-io/delta-rs/blob/346f51a277c23be09eca373cd6d775b745d59c05/rust/src/action.rs#L126. So I think we just need to wrap these types with an Option like what @wjones127 said.
We should probably also send a PR to https://github.com/delta-io to clarify the behavior too.
Thanks @xiaoyongzhu for the report :)
I think this is probably caused by an ambiguity in the delta spec. In https://github.com/delta-io/delta/blob/master/PROTOCOL.md#per-file-statistics, there is no mention of whether these stats fields are optional or not.
Right now all the stats fields are modeled as required in
https://github.com/delta-io/delta-rs/blob/346f51a277c23be09eca373cd6d775b745d59c05/rust/src/action.rs#L126
. So I think we just need to wrap these types with an Option like what @wjones127 said. We should probably also send a PR to https://github.com/delta-io to clarify the behavior too.
Thanks @houqp for the response! Ya marking those as optional should be good. Maybe I can expect a fix (and a release) soon?
Let's see what what @wjones127 has to say about this approach first. You are welcome submit a quick fix PR as well if you wants to get your hands rusty ;)
Yes, adding the option seems like it should solve this issue.
To be defensive, I'm inclined to also add parsing logic to ignore the stats if they come back with this error. What do you think of that @houqp?
We should go over and make a clarification to the delta protocol.
To be defensive, I'm inclined to also add parsing logic to ignore the stats if they come back with this error. What do you think of that @houqp?
My main concern about this is what happens if it contains other stats like null count and max val, but only missing min value? In this case, we still want to load the other stats right?
My main concern about this is what happens if it contains other stats like null count and max val, but only missing min value? In this case, we still want to load the other stats right?
Yes, agreed. My main point is that parsing these stats could return an Option, not a Result, so that even if the stats were completely corrupt (say invalid JSON, or used a custom format we didn't recognize), we could still load the table.
Ha I see, that sounds reasonable. But I think it would be good to log the parsing error as well so we are not silently ignoring the error.
Thanks @houqp and @wjones127 ! I'm not a rust developer so might not be able to help here, but definitely open to help testing it out!
This problem currently prevents use of delta-rs with "Microsoft Fabric Data Warehouse" ( https://twitter.com/mim_djo/status/1692855422364041428 ). I'm also told that disabling column statistics in Spark results in a log entry which has numRecords but not the other members.