Skip to content
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

Better construction of RecordBatchOptions #2729

Merged
merged 7 commits into from Sep 16, 2022
Merged
Show file tree
Hide file tree
Changes from 5 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
12 changes: 4 additions & 8 deletions arrow/src/ipc/reader.rs
Expand Up @@ -578,10 +578,7 @@ pub fn read_record_batch(
let mut node_index = 0;
let mut arrays = vec![];

let options = RecordBatchOptions {
row_count: Some(batch.length() as usize),
..Default::default()
askoa marked this conversation as resolved.
Show resolved Hide resolved
};
let options = RecordBatchOptions::new().with_row_count(Some(batch.length() as usize));

if let Some(projection) = projection {
// project fields
Expand Down Expand Up @@ -1692,10 +1689,9 @@ mod tests {
#[test]
fn test_no_columns_batch() {
let schema = Arc::new(Schema::new(vec![]));
let options = RecordBatchOptions {
match_field_names: true,
row_count: Some(10),
};
let options = RecordBatchOptions::new()
askoa marked this conversation as resolved.
Show resolved Hide resolved
.with_match_field_names(true)
.with_row_count(Some(10));
let input_batch =
RecordBatch::try_new_with_options(schema, vec![], &options).unwrap();
let output_batch = roundtrip_ipc_stream(&input_batch);
Expand Down
35 changes: 27 additions & 8 deletions arrow/src/record_batch.rs
Expand Up @@ -80,7 +80,7 @@ impl RecordBatch {
/// # }
/// ```
pub fn try_new(schema: SchemaRef, columns: Vec<ArrayRef>) -> Result<Self> {
let options = RecordBatchOptions::default();
let options = RecordBatchOptions::new();
Self::try_new_impl(schema, columns, &options)
}

Expand Down Expand Up @@ -413,15 +413,29 @@ pub struct RecordBatchOptions {
pub row_count: Option<usize>,
}

impl Default for RecordBatchOptions {
fn default() -> Self {
impl RecordBatchOptions {
pub fn new() -> Self {
Self {
match_field_names: true,
row_count: None,
}
}
//Sets the row_count of RecordBatchOptions and returns self
askoa marked this conversation as resolved.
Show resolved Hide resolved
pub fn with_row_count(mut self, row_count: Option<usize>) -> Self {
self.row_count = row_count;
self
}
//Sets the match_field_names of RecordBatchOptions and returns self
askoa marked this conversation as resolved.
Show resolved Hide resolved
pub fn with_match_field_names(mut self, match_field_names: bool) -> Self {
self.match_field_names = match_field_names;
self
}
}
impl Default for RecordBatchOptions {
fn default() -> Self {
Self::new()
}
}

impl From<&StructArray> for RecordBatch {
/// Create a record batch from struct array, where each field of
/// the `StructArray` becomes a `Field` in the schema.
Expand Down Expand Up @@ -901,10 +915,7 @@ mod tests {
.to_string()
.contains("must either specify a row count or at least one column"));

let options = RecordBatchOptions {
row_count: Some(10),
..Default::default()
};
let options = RecordBatchOptions::new().with_row_count(Some(10));

let ok =
RecordBatch::try_new_with_options(schema.clone(), vec![], &options).unwrap();
Expand All @@ -929,4 +940,12 @@ mod tests {
);
assert_eq!("Invalid argument error: Column 'a' is declared as non-nullable but contains null values", format!("{}", maybe_batch.err().unwrap()));
}
#[test]
fn test_record_batch_options() {
let options = RecordBatchOptions::new()
.with_match_field_names(false)
.with_row_count(Some(20));
assert!(!options.match_field_names);
assert_eq!(options.row_count.unwrap(), 20)
}
}