-
Notifications
You must be signed in to change notification settings - Fork 255
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: don't eagerly materialize fields that the user hasn't asked for #3442
Merged
Merged
Changes from 7 commits
Commits
Show all changes
9 commits
Select commit
Hold shift + click to select a range
83f8027
Don't eagerly materialize fields that the user hasn't asked for
westonpace fe80ac7
Fix unit tests
westonpace 9bd13f7
Fix lint
westonpace d047224
Fix python lint
westonpace 43dbc9d
Fix java failures (maybe)
westonpace fe585a6
Fix java compile
westonpace 4e924c0
Java formatting
westonpace 9a2da8c
Fix Java unit test
westonpace b3df0b2
Java formatting
westonpace File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -189,6 +189,7 @@ impl MaterializationStyle { | |
} | ||
|
||
/// Filter for filtering rows | ||
#[derive(Debug)] | ||
pub enum LanceFilter { | ||
/// The filter is an SQL string | ||
Sql(String), | ||
|
@@ -1027,11 +1028,22 @@ impl Scanner { | |
Ok(concat_batches(&schema, &batches)?) | ||
} | ||
|
||
/// Scan and return the number of matching rows | ||
#[instrument(skip_all)] | ||
pub fn count_rows(&self) -> BoxFuture<Result<u64>> { | ||
fn create_count_plan(&self) -> BoxFuture<Result<Arc<dyn ExecutionPlan>>> { | ||
// Future intentionally boxed here to avoid large futures on the stack | ||
async move { | ||
if !self.projection_plan.physical_schema.fields.is_empty() { | ||
return Err(Error::invalid_input( | ||
"count_rows should not be called on a plan selecting columns".to_string(), | ||
location!(), | ||
)); | ||
} | ||
|
||
if self.limit.is_some() || self.offset.is_some() { | ||
log::warn!( | ||
"count_rows called with limit or offset which could have surprising results" | ||
); | ||
} | ||
|
||
let plan = self.create_plan().await?; | ||
// Datafusion interprets COUNT(*) as COUNT(1) | ||
let one = Arc::new(Literal::new(ScalarValue::UInt8(Some(1)))); | ||
|
@@ -1046,14 +1058,27 @@ impl Scanner { | |
let count_expr = builder.build()?; | ||
|
||
let plan_schema = plan.schema(); | ||
let count_plan = Arc::new(AggregateExec::try_new( | ||
Ok(Arc::new(AggregateExec::try_new( | ||
AggregateMode::Single, | ||
PhysicalGroupBy::new_single(Vec::new()), | ||
vec![Arc::new(count_expr)], | ||
vec![None], | ||
plan, | ||
plan_schema, | ||
)?); | ||
)?) as Arc<dyn ExecutionPlan>) | ||
} | ||
.boxed() | ||
} | ||
|
||
/// Scan and return the number of matching rows | ||
/// | ||
/// Note: calling [`Dataset::count_rows`] can be more efficient than calling this method | ||
/// especially if there is no filter. | ||
#[instrument(skip_all)] | ||
pub fn count_rows(&self) -> BoxFuture<Result<u64>> { | ||
// Future intentionally boxed here to avoid large futures on the stack | ||
async move { | ||
let count_plan = self.create_count_plan().await?; | ||
let mut stream = execute_plan(count_plan, LanceExecutionOptions::default())?; | ||
|
||
// A count plan will always return a single batch with a single row. | ||
|
@@ -1127,15 +1152,25 @@ impl Scanner { | |
} | ||
} | ||
|
||
fn calc_eager_columns(&self, filter_plan: &FilterPlan) -> Result<Arc<Schema>> { | ||
let columns = filter_plan.refine_columns(); | ||
// If we are going to filter on `filter_plan`, then which columns are so small it is | ||
// cheaper to read the entire column and filter in memory. | ||
// | ||
// Note: only add columns that we actually need to read | ||
fn calc_eager_columns( | ||
&self, | ||
filter_plan: &FilterPlan, | ||
desired_schema: &Schema, | ||
) -> Result<Arc<Schema>> { | ||
let filter_columns = filter_plan.refine_columns(); | ||
let early_schema = self | ||
.dataset | ||
.empty_projection() | ||
// We need the filter columns | ||
.union_columns(columns, OnMissing::Error)? | ||
// And also any columns that are eager | ||
.union_predicate(|f| self.is_early_field(f)) | ||
// Start with the desired schema | ||
.union_schema(desired_schema) | ||
// Subtract columns that are expensive | ||
.subtract_predicate(|f| !self.is_early_field(f)) | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. do "early" and "eager" mean the same thing in the vocabulary? |
||
// Add back columns that we need for filtering | ||
.union_columns(filter_columns, OnMissing::Error)? | ||
.into_schema_ref(); | ||
|
||
if early_schema.fields.iter().any(|f| !f.is_default_storage()) { | ||
|
@@ -1340,7 +1375,10 @@ impl Scanner { | |
(Some(index_query), Some(_)) => { | ||
// If there is a filter then just load the eager columns and | ||
// "take" the other columns later. | ||
let eager_schema = self.calc_eager_columns(&filter_plan)?; | ||
let eager_schema = self.calc_eager_columns( | ||
&filter_plan, | ||
self.projection_plan.physical_schema.as_ref(), | ||
)?; | ||
self.scalar_indexed_scan(&eager_schema, index_query).await? | ||
} | ||
(None, Some(_)) if use_stats && self.batch_size.is_none() => { | ||
|
@@ -1352,7 +1390,10 @@ impl Scanner { | |
let eager_schema = if filter_plan.has_refine() { | ||
// If there is a filter then only load the filter columns in the | ||
// initial scan. We will `take` the remaining columns later | ||
self.calc_eager_columns(&filter_plan)? | ||
self.calc_eager_columns( | ||
&filter_plan, | ||
self.projection_plan.physical_schema.as_ref(), | ||
)? | ||
} else { | ||
// If there is no filter we eagerly load everything | ||
self.projection_plan.physical_schema.clone() | ||
|
@@ -3913,14 +3954,11 @@ mod test { | |
.unwrap(); | ||
|
||
let dataset = Dataset::open(test_uri).await.unwrap(); | ||
assert_eq!(32, dataset.scan().count_rows().await.unwrap()); | ||
assert_eq!(32, dataset.count_rows(None).await.unwrap()); | ||
assert_eq!( | ||
16, | ||
dataset | ||
.scan() | ||
.filter("`Filter_me` > 15") | ||
.unwrap() | ||
.count_rows() | ||
.count_rows(Some("`Filter_me` > 15".to_string())) | ||
.await | ||
.unwrap() | ||
); | ||
|
@@ -3948,7 +3986,7 @@ mod test { | |
.unwrap(); | ||
|
||
let dataset = Dataset::open(test_uri).await.unwrap(); | ||
assert_eq!(32, dataset.scan().count_rows().await.unwrap()); | ||
assert_eq!(dataset.count_rows(None).await.unwrap(), 32); | ||
|
||
let mut scanner = dataset.scan(); | ||
|
||
|
@@ -3996,7 +4034,7 @@ mod test { | |
.unwrap(); | ||
|
||
let dataset = Dataset::open(test_uri).await.unwrap(); | ||
assert_eq!(32, dataset.scan().count_rows().await.unwrap()); | ||
assert_eq!(dataset.count_rows(None).await.unwrap(), 32); | ||
|
||
let mut scanner = dataset.scan(); | ||
|
||
|
@@ -4519,20 +4557,13 @@ mod test { | |
} | ||
} | ||
|
||
/// Assert that the plan when formatted matches the expected string. | ||
/// | ||
/// Within expected, you can use `...` to match any number of characters. | ||
async fn assert_plan_equals( | ||
dataset: &Dataset, | ||
plan: impl Fn(&mut Scanner) -> Result<&mut Scanner>, | ||
async fn assert_plan_node_equals( | ||
plan_node: Arc<dyn ExecutionPlan>, | ||
expected: &str, | ||
) -> Result<()> { | ||
let mut scan = dataset.scan(); | ||
plan(&mut scan)?; | ||
let exec_plan = scan.create_plan().await?; | ||
let plan_desc = format!( | ||
"{}", | ||
datafusion::physical_plan::displayable(exec_plan.as_ref()).indent(true) | ||
datafusion::physical_plan::displayable(plan_node.as_ref()).indent(true) | ||
); | ||
|
||
let to_match = expected.split("...").collect::<Vec<_>>(); | ||
|
@@ -4559,6 +4590,71 @@ mod test { | |
Ok(()) | ||
} | ||
|
||
/// Assert that the plan when formatted matches the expected string. | ||
/// | ||
/// Within expected, you can use `...` to match any number of characters. | ||
async fn assert_plan_equals( | ||
dataset: &Dataset, | ||
plan: impl Fn(&mut Scanner) -> Result<&mut Scanner>, | ||
expected: &str, | ||
) -> Result<()> { | ||
let mut scan = dataset.scan(); | ||
plan(&mut scan)?; | ||
let exec_plan = scan.create_plan().await?; | ||
assert_plan_node_equals(exec_plan, expected).await | ||
} | ||
|
||
#[tokio::test] | ||
async fn test_count_plan() { | ||
// A count rows operation should load the minimal amount of data | ||
let dim = 256; | ||
let fixture = TestVectorDataset::new_with_dimension(LanceFileVersion::Stable, true, dim) | ||
.await | ||
.unwrap(); | ||
|
||
// By default, all columns are returned, this is bad for a count_rows op | ||
let err = fixture | ||
.dataset | ||
.scan() | ||
.create_count_plan() | ||
.await | ||
.unwrap_err(); | ||
assert!(matches!(err, Error::InvalidInput { .. })); | ||
|
||
let mut scan = fixture.dataset.scan(); | ||
scan.project(&Vec::<String>::default()).unwrap(); | ||
|
||
// with_row_id needs to be specified | ||
let err = scan.create_count_plan().await.unwrap_err(); | ||
assert!(matches!(err, Error::InvalidInput { .. })); | ||
|
||
scan.with_row_id(); | ||
|
||
let plan = scan.create_count_plan().await.unwrap(); | ||
|
||
assert_plan_node_equals( | ||
plan, | ||
"AggregateExec: mode=Single, gby=[], aggr=[count_rows] | ||
LanceScan: uri=..., projection=[], row_id=true, row_addr=false, ordered=true", | ||
) | ||
.await | ||
.unwrap(); | ||
|
||
scan.filter("s == ''").unwrap(); | ||
|
||
let plan = scan.create_count_plan().await.unwrap(); | ||
|
||
assert_plan_node_equals( | ||
plan, | ||
"AggregateExec: mode=Single, gby=[], aggr=[count_rows] | ||
ProjectionExec: expr=[_rowid@1 as _rowid] | ||
FilterExec: s@0 = | ||
LanceScan: uri=..., projection=[s], row_id=true, row_addr=false, ordered=true", | ||
) | ||
.await | ||
.unwrap(); | ||
} | ||
|
||
#[rstest] | ||
#[tokio::test] | ||
async fn test_late_materialization( | ||
|
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
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'm a little torn on this error. Ideally, we would just silently blank out the projection plan and then create the count plan. However, to do that we either have to clone the scan, which is a pretty big thing to be cloning, or modify the scanner, which would maybe not be what users would expect from
count_rows
.For now, I want to get something out soon, so I'm just raising an error, with the assumption that
Scanner::count_rows
is a mostly internal method anyways (users should useDataset::count_rows
).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 think you're right that is pretty internal. Plus easy to work around.