-
Notifications
You must be signed in to change notification settings - Fork 1.3k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Writing a Dataframe to a partitioned table ignores partitioning #7860
Comments
Thanks for the report @theelderbeever! Partitioned writes are not supported yet, so you should have received an error on the I just so happened to open a PR adding support for partitioned writes recently (#7801), which should resolve this issue as a consequence. If you are so inclined, feel free to try out #7801 to see if you get the expected behavior with respect to partitioned writes. #7801 does not add support for writing the compression codec in the file suffix. I think it would be a good idea to open an additional issue to track that for future work. |
@devinjdangelo Nice thats awesome on the timing! Ran against your branch and the only problem was lack of timestamp support. Coercing to a
|
How did you coerce? If you define your table so that the partition columns are of type STRING like below and also convert incoming timestamps to strings it should work. Basically the DDL of the table and the schema of the data to be written must both be string for any partition column currently. Some auto casting between common partitioning types would definitely be a nice improvement in the future. CREATE EXTERNAL TABLE
partitioned_insert_test_pq(a string, b bigint)
STORED AS parquet
LOCATION 'test_files/scratch/insert_to_external/insert_to_partitioned_pq/'
PARTITIONED BY (a)
OPTIONS(
create_local_path 'true',
insert_mode 'append_new_files',
);
INSERT INTO partitioned_insert_test_pq values (1, 2), (3, 4), (5, 6), (1, 2), (3, 4), (5, 6);
----
6
select * from partitioned_insert_test_pq order by a ASC, b ASC
----
1 2
1 2
3 4
3 4
5 6
5 6 |
@devinjdangelo The clunky workaround though is to register the table with the timestamp partition column as a Utf8 when writing and re-register it with the correct datatypes when reading. Also... Not sure if this was introduced on your branch or not but, the number of fields isn't being determined correctly when I have a Map column. I get this
|
@devinjdangelo jinx... but yeah defining the table to write to using strings then just manually casting the timestamp worked. let schema = Schema::new(VectorMetric::fields());
let ctx = SessionContext::new();
ctx.register_parquet(
"metrics",
"data/metrics",
ParquetReadOptions::default()
.table_partition_cols(vec![(
"time_bucket".to_string(),
// DataType::Timestamp(TimeUnit::Second, None),
DataType::Utf8,
)])
.schema(&schema),
)
.await
.unwrap();
ctx.register_batch("batch", batch).unwrap();
let write_options = DataFrameWriteOptions::default()
.with_compression(CompressionTypeVariant::ZSTD)
.with_single_file_output(false);
let df = ctx
.sql(
r#"
SELECT *, DATE_TRUNC('DAY', timestamp)::TEXT AS time_bucket FROM batch
"#,
)
.await
.unwrap()
.write_table("metrics", write_options)
.await
.unwrap(); |
@theelderbeever agreed that that workaround is clunky... Could you share a create table statement or example schema definition that is giving you issues, including with the map type? I can try to replicate and see if I can fix and possibly handle the type casting transparently. |
@devinjdangelo Sorry in advance for how gross this is... Its a sandbox right now Clone this -> https://github.com/theelderbeever/datafusion-testing The run The Cargo.toml is already pointed at your branch. Commenting out L140 and L183 will make the write successful. Having them uncommented results in the error. |
@theelderbeever thank you for this reproducible example! I was able to replicate the error and identify the root cause. It was indeed a flaw in how I was handling nested columns. I just pushed up a fix, which now handles nested columns correctly. Let me know if the latest state of the branch is working for you a bit more smoothly now. I did not get to improving auto type casting to string type, but I plan to cut an issue for that if the branch is merged before I get to it (cc @alamb). |
Describe the bug
After registering a parquet table with a partition column I write a dataframe to that table then attempt to read back the data from the table. This results in an error
index out of bounds: the len is 0 but the index is 0
. Upon inspecting the table path there is a written parquet file. This parquet file is readable although when being read directly it still contains the column for partitioning which to my understanding shouldn't be there if hive_partitioning is used. Additionally, the directory for the table contains no partitions and the parquet is at the top level.Also two things ofof note:
To Reproduce
This code is missing some of the supporting methods but is the gist of where the problem is.
Expected behavior
Additional context
I can sanitize data and send the rest of the example if needed.
The text was updated successfully, but these errors were encountered: