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

Fix several tests #8

Merged
merged 1 commit into from
Sep 25, 2021
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
9 changes: 7 additions & 2 deletions datafusion/src/physical_plan/filter.rs
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,7 @@ use crate::physical_plan::{
DisplayFormatType, ExecutionPlan, Partitioning, PhysicalExpr,
};

use arrow::array::BooleanArray;
use arrow::array::{BooleanArray, Array};
use arrow::compute::filter::filter_record_batch;
use arrow::datatypes::{DataType, SchemaRef};
use arrow::error::Result as ArrowResult;
Expand All @@ -39,6 +39,7 @@ use arrow::record_batch::RecordBatch;
use async_trait::async_trait;

use futures::stream::{Stream, StreamExt};
use arrow::compute::boolean::{and, is_not_null};

/// FilterExec evaluates a boolean predicate against all input batches to determine which rows to
/// include in its output batches.
Expand Down Expand Up @@ -184,7 +185,11 @@ fn batch_filter(
.into_arrow_external_error()
})
// apply filter array to record batch
.and_then(|filter_array| filter_record_batch(batch, filter_array))
.and_then(|filter_array| {
let is_not_null = is_not_null(filter_array as &dyn Array);
let and_filter = and(&is_not_null, filter_array)?;
filter_record_batch(batch, &and_filter)
Copy link
Collaborator Author

@yjshen yjshen Sep 25, 2021

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

per filter_record_batch doc, we should make filter array null free first:

/// Returns a new [RecordBatch] with arrays containing only values matching the filter.
/// WARNING: the nulls of `filter` are ignored and the value on its slot is considered.
/// Therefore, it is considered undefined behavior to pass `filter` with null values.
pub fn filter_record_batch(
    record_batch: &RecordBatch,
    filter_values: &BooleanArray,
) -> Result<RecordBatch> {

})
})
}

Expand Down
1 change: 0 additions & 1 deletion datafusion/tests/parquet_pruning.rs
Original file line number Diff line number Diff line change
Expand Up @@ -617,7 +617,6 @@ async fn make_test_file(scenario: Scenario) -> NamedTempFile {
};

let schema = batches[0].schema();
eprintln!("----------- schema {:?}", schema);

let options = WriteOptions {
compression: Compression::Uncompressed,
Expand Down
Loading