-
Notifications
You must be signed in to change notification settings - Fork 1.7k
feat: Support reading CSV files with inconsistent column counts #17553
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
base: main
Are you sure you want to change the base?
feat: Support reading CSV files with inconsistent column counts #17553
Conversation
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Looks like some CI checks to address as well; would suggest running the standard cargo clippy, fmt & test commands locally before pushing so we don't need to wait for CI checks on GitHub to catch these
// Verify we can actually read the data | ||
let results = df.collect().await?; | ||
|
||
// Calculate total rows across all batches | ||
let total_rows: usize = results.iter().map(|batch| batch.num_rows()).sum(); | ||
assert_eq!(total_rows, 6, "Should have 6 total rows across all batches"); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I feel we should assert the actual row contents as well
// Handle files with different numbers of columns by extending the schema | ||
if fields.len() > column_type_possibilities.len() { | ||
// New columns found - extend our tracking structures | ||
for field in fields.iter().skip(column_type_possibilities.len()) { | ||
column_names.push(field.name().clone()); | ||
let mut possibilities = HashSet::new(); | ||
if records_read > 0 { | ||
possibilities.insert(field.data_type().clone()); | ||
} | ||
column_type_possibilities.push(possibilities); | ||
} | ||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
What happens if file A has columns t1, t3 but file B has columns t1, t2, t3?
Do we only allow files having subset of columns of other files in the exact correct order?
AKA we don't support union by name?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The current implementation performs positional union (not union by name). Files must have columns in the same order, with later files potentially adding new columns at the end. This is consistent with CSV format which doesn't have inherent column naming, column names come from headers and are positional. Union by name would require a different approach and is not implemented in this PR.
Addresses all review feedback from PR apache#17553 to improve the CSV schema union implementation that allows reading CSV files with different column counts. Changes based on review: - Moved unit tests from separate tests.rs to bottom of file_format.rs - Updated documentation wording from "now supports" to "can handle" - Removed all println statements from integration test - Added comprehensive assertions for actual row content verification - Simplified HashSet initialization using HashSet::from([...]) syntax - Updated truncated_rows config documentation to reflect expanded purpose - Removed unnecessary min() calculation in column processing loop - Fixed clippy warnings by using enumerate() instead of range loop Technical improvements: - Tests now verify null patterns correctly across union schema - Cleaner iteration logic without redundant bounds checking - Better documentation explaining union schema behavior The feature continues to work as designed: - Creates union schema from all CSV files in a directory - Files with fewer columns have nulls for missing fields - Requires explicit opt-in via truncated_rows(true) - Maintains full backward compatibility
Enable DataFusion to read directories containing CSV files with different numbers of columns by implementing schema union during inference. Changes: - Modified CSV schema inference to create union schema from all files - Extended infer_schema_from_stream to handle varying column counts - Added tests for schema building logic and integration scenarios Requires CsvReadOptions::new().truncated_rows(true) to handle files with fewer columns than the inferred schema. Fixes apache#17516
Addresses all review feedback from PR apache#17553 to improve the CSV schema union implementation that allows reading CSV files with different column counts. Changes based on review: - Moved unit tests from separate tests.rs to bottom of file_format.rs - Updated documentation wording from "now supports" to "can handle" - Removed all println statements from integration test - Added comprehensive assertions for actual row content verification - Simplified HashSet initialization using HashSet::from([...]) syntax - Updated truncated_rows config documentation to reflect expanded purpose - Removed unnecessary min() calculation in column processing loop - Fixed clippy warnings by using enumerate() instead of range loop Technical improvements: - Tests now verify null patterns correctly across union schema - Cleaner iteration logic without redundant bounds checking - Better documentation explaining union schema behavior The feature continues to work as designed: - Creates union schema from all CSV files in a directory - Files with fewer columns have nulls for missing fields - Requires explicit opt-in via truncated_rows(true) - Maintains full backward compatibility
81b2dd9
to
a0ad856
Compare
pub(crate) fn build_schema_helper( | ||
names: Vec<String>, | ||
types: &[HashSet<DataType>], | ||
) -> Schema { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
pub(crate) fn build_schema_helper( | |
names: Vec<String>, | |
types: &[HashSet<DataType>], | |
) -> Schema { | |
fn build_schema_helper(names: Vec<String>, types: &[HashSet<DataType>]) -> Schema { |
// Verify the actual content of the data | ||
// Since we don't know the exact order of rows, just verify the overall structure |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I suggest using assert_snapshot!()
macro to check the contents, see a reference here:
datafusion/datafusion/core/tests/sql/select.rs
Lines 22 to 51 in c1ca3c4
#[tokio::test] | |
async fn test_list_query_parameters() -> Result<()> { | |
let tmp_dir = TempDir::new()?; | |
let partition_count = 4; | |
let ctx = create_ctx_with_partition(&tmp_dir, partition_count).await?; | |
let results = ctx | |
.sql("SELECT * FROM test WHERE c1 = $1") | |
.await? | |
.with_param_values(vec![ScalarValue::from(3i32)])? | |
.collect() | |
.await?; | |
assert_snapshot!(batches_to_sort_string(&results), @r" | |
+----+----+-------+ | |
| c1 | c2 | c3 | | |
+----+----+-------+ | |
| 3 | 1 | false | | |
| 3 | 10 | true | | |
| 3 | 2 | true | | |
| 3 | 3 | false | | |
| 3 | 4 | true | | |
| 3 | 5 | false | | |
| 3 | 6 | true | | |
| 3 | 7 | false | | |
| 3 | 8 | true | | |
| 3 | 9 | false | | |
+----+----+-------+ | |
"); | |
Ok(()) | |
} |
/// Whether to allow CSV files with varying numbers of columns. | ||
/// By default this is set to false and will error if the CSV rows have different lengths. | ||
/// When set to true: | ||
/// - Allows reading multiple CSV files with different column counts | ||
/// - Creates a union schema during inference containing all columns found across files | ||
/// - Files with fewer columns will have missing columns filled with null values |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
/// Whether to allow CSV files with varying numbers of columns. | |
/// By default this is set to false and will error if the CSV rows have different lengths. | |
/// When set to true: | |
/// - Allows reading multiple CSV files with different column counts | |
/// - Creates a union schema during inference containing all columns found across files | |
/// - Files with fewer columns will have missing columns filled with null values | |
/// Whether to allow truncated rows when parsing, both within a single file and across files. | |
/// | |
/// When set to false (default), reading a single CSV file which has rows of different lengths will | |
/// error; if reading multiple CSV files with different number of columns, it will also fail. | |
/// | |
/// When set to true, reading a single CSV file with rows of different lengths will pad the truncated | |
/// rows with null values for the missing columns; if reading multiple CSV files with different number | |
/// of columns, it creates a union schema containing all columns found across the files, and will | |
/// pad any files missing columns with null values for their rows. |
Just to make it more obvious this config has a dual purpose.
column_type_possibilities.iter_mut().zip(&fields).for_each( | ||
|(possibilities, field)| { | ||
possibilities.insert(field.data_type().clone()); | ||
}, | ||
); | ||
// Update type possibilities for columns that exist in this file | ||
// Only process fields that exist in both the current file and our tracking structures | ||
for (field_idx, field) in fields.iter().enumerate() { | ||
if field_idx < column_type_possibilities.len() { | ||
column_type_possibilities[field_idx] | ||
.insert(field.data_type().clone()); | ||
} | ||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
It might be less confusing if we keep the original code:
column_type_possibilities.iter_mut().zip(&fields).for_each(
|(possibilities, field)| {
possibilities.insert(field.data_type().clone());
},
);
But move it before the Handle files with different numbers of columns by extending the schema
check (since zip will terminate on the shortest iterator), so we first extend possibilities for existing columns, then afterwards add the new columns.
Another way could be to do both in a single iteration; something like
for idx, field in enumerate(fields)
if idx < len(possibilities)
add_possibility
else
add_new_field
Thoughts?
if fields.len() != column_type_possibilities.len() { | ||
return exec_err!( | ||
"Encountered unequal lengths between records on CSV file whilst inferring schema. \ | ||
Expected {} fields, found {} fields at record {}", | ||
column_type_possibilities.len(), | ||
fields.len(), | ||
record_number + 1 | ||
); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Would it be beneficial to keep this error message for when truncated_rows
config is false
? What does the error message look like right now if we read CSV files with differing column counts where this new behaviour is not enabled?
Summary
Enables DataFusion to read directories containing CSV files with
different numbers of columns by implementing schema union during
inference.
Previously, attempting to read multiple CSV files with different
column counts would fail with:
Arrow error: Csv error: incorrect number of fields for line 1,
expected 17 got 20
This was particularly problematic for evolving datasets where newer
files include additional columns (e.g., railway services data where
newer files added platform information).
Changes
infer_schema_from_stream
to create union schema from all filesinstead of rejecting files with different column counts
requires explicit opt-in via
truncated_rows(true)
logic and integration test with real CSV scenarios
Usage
Test Results
CSV files
Example
Before this fix:
After this fix:
Closes #17516