Skip to content

Conversation

@adriangb
Copy link
Contributor

@adriangb adriangb commented Apr 18, 2025

Moves predicate pushdown into parquet being something specialized that ListingTable and Parquet to working for any TableProvider and any file format the implements the APIs. The checks for compatibility also happen all within the parquet data source machinery, instead of leaking implementations via supports_filters_pushdown.

@github-actions github-actions bot added core Core DataFusion crate datasource Changes to the datasource crate labels Apr 18, 2025
Copy link
Contributor Author

@adriangb adriangb left a comment

Choose a reason for hiding this comment

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

pointing out current issues to move forward with implementing parquet filter pushdown via the new APIs we've introduced

cc @alamb @berkaysynnada for ideas

Comment on lines 478 to 455
Arc::new(ParquetSource::default())
todo!() // need access of file schema?
Copy link
Contributor Author

Choose a reason for hiding this comment

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

This poses an issue.

TLDR is that in order to know if it can absorb a filter as exact ParquetSource needs to know not only the filter but also the file schema it's applied to (in particular to get the type of the columns since it can't handle structs).

let remaining_description = if config.execution.parquet.pushdown_filters {
let mut remaining_filters = fd.filters.clone();
for filter in &remaining_filters {
if can_expr_be_pushed_down_with_schemas(filter, &conf.file_schema) {
Copy link
Contributor Author

Choose a reason for hiding this comment

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

This is where we need the file schema

@alamb
Copy link
Contributor

alamb commented Apr 19, 2025

Thanks @adriangb -- I am about to be offline for a week so I will review this when I return

@adriangb adriangb changed the title re-implement filter pushdown for parquet Implement filter pushdown for TopK Apr 19, 2025
@adriangb adriangb changed the title Implement filter pushdown for TopK re-implement filter pushdown for parquet Apr 19, 2025
@github-actions github-actions bot added the proto Related to proto crate label Apr 19, 2025
@adriangb adriangb changed the title re-implement filter pushdown for parquet Implement Parquet filter pushdown via new filter pushdown APIs Apr 19, 2025
@github-actions github-actions bot added the sqllogictest SQL Logic Tests (.slt) label Apr 19, 2025
Comment on lines -985 to -1015
// if we can't push it down completely with only the filename-based/path-based
// column names, then we should check if we can do parquet predicate pushdown
let supports_pushdown = self.options.format.supports_filters_pushdown(
&self.file_schema,
&self.table_schema,
&[filter],
)?;

if supports_pushdown == FilePushdownSupport::Supported {
return Ok(TableProviderFilterPushDown::Exact);
}
Copy link
Contributor Author

Choose a reason for hiding this comment

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

The point of this PR is that this moves from being something specialized that ListingTable does to anything that works for any TableProvider / they don't need to do anything special! The checks for compatibility also happen all within the parquet data source machinery, instead of leaking implementations via supports_filters_pushdown.

Copy link
Contributor

Choose a reason for hiding this comment

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

I have one question: aren't we expecting/preparing for, people to use ListingTable if they read Parquet files? Are we eventually planning to remove all format-specific handlings? Or this is a case only for filter pushdown?

Copy link
Contributor

Choose a reason for hiding this comment

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

If that's the case, why don't we fully remove supports_filters_pushdown() API at all

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I think many users of DataFusion (based on our usage, talks I've seen and examples we have) use custom TableProvider implementations.

I would keep supports_filters_pushdown so that TableProviders can do Exact pruning of filters, e.g. using partition columns.

Copy link
Contributor

Choose a reason for hiding this comment

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

We can justify implementing other TableProviders for Parquet, but still I cannot understand why we need to degrade the capabilities of our ListingTable. Is't it always better pruning/simplifying things at the higher levels as possible?

Copy link
Contributor

Choose a reason for hiding this comment

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

I have one question: aren't we expecting/preparing for, people to use ListingTable if they read Parquet files? Are we eventually planning to remove all format-specific handlings? Or this is a case only for filter pushdown?

For what it is worth, we (InfluxData) doesn't use ListingTable to read parquet files, instead we provide our own equivalent and create the DataSourceExec's directly

I would keep supports_filters_pushdown so that TableProviders can do Exact pruning of filters, e.g. using partition columns.

Yes I think that is important too -- I don't think we should be removing any APIs from ListingTable

Copy link
Contributor

Choose a reason for hiding this comment

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

We can justify implementing other TableProviders for Parquet, but still I cannot understand why we need to degrade the capabilities of our ListingTable. Is't it always better pruning/simplifying things at the higher levels as possible?

I don't think this degrades the capabilities of the current listing table. I think the only implications are for anyone who used a custom FileFormat and impleented supports_filters_pushdown -- I suspect this is not very common and we can likely avoid consternation by mentioning it in the upgrade guide (see comment below)

Comment on lines -151 to +152
source = source.with_predicate(Arc::clone(&file_schema), predicate);
source = source.with_predicate(predicate);
Copy link
Contributor Author

Choose a reason for hiding this comment

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

This seemed like an easy win since I was able to just change this so that the schema is always passed in by the FileSourceConfigBuilder instead of only when with_predicate is called.
This was necessary becasue with_predicate is no longer called to attach a predicate, instaed it happens during an optimization pass so ParquetSource neesd to have it available at that point.
I left with_predicate in there to avoid churn and in case there is a use case for attaching a predicate directly through the scan instad of a as a FilterExec that later gets pushed into the scan.

base_config,
predicate,
pruning_predicate: parquet.pruning_predicate,
pruning_predicate: None, // for backwards compat since `ParquetExec` is only for backwards compat anyway
Copy link
Contributor Author

Choose a reason for hiding this comment

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

Open to other suggestions (i.e. removing it). I felt like this minimizes breakage for folks still using ParquetExec, who are likely the same folks that want to do the least amount of work possible to upgrade.

Copy link
Contributor

Choose a reason for hiding this comment

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

yeah this is fine too in my opinion. It is almost time to remove ParquetExec anyways -- maybe we should just do it in this release 🤔

Copy link
Contributor

Choose a reason for hiding this comment

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

Comment on lines 652 to 656
let table_schema = get_basic_table_schema();

let file_schema = Schema::new(vec![Field::new(
"list_col",
DataType::Struct(Fields::empty()),
true,
)]);
Copy link
Contributor Author

Choose a reason for hiding this comment

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

This test was wrong! It wanted to test that list_col prevents pushdown because it's a nested type. Instead it was prevented because list_col is not in the table / schema!

Comment on lines 562 to 580
let pruning_predicate_string = self
.pruning_predicate
.as_ref()
.map(|pre| {
let mut guarantees = pre
.literal_guarantees()
.iter()
.map(|item| format!("{}", item))
.collect_vec();
guarantees.sort();
format!(
", pruning_predicate={}, required_guarantees=[{}]",
pre.predicate_expr(),
guarantees.join(", ")
)
})
.unwrap_or_default();
Copy link
Contributor Author

Choose a reason for hiding this comment

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

In #15561 (review) Andrew asked me to keep this, but now since the schema isn't even being passed in to with_predicate it's going to be hard to keep these. I suggest we just accept that they won't be present in the physical plans. If that's not okay what I could do is generate them on the fly in fmt_extra or generate them if with_predicate is called with a schema or with_schema is called with a predicate. But I'd like to avoid that unless someone thinks is worth it or has another suggestion.

Copy link
Contributor

Choose a reason for hiding this comment

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

I think it is important to keep these in the physical plans -- in particular what I think is important is to be able to check via the explain plan if pruning is happening by looking at the explain plan

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Hmm okay. I'll see if I can make it happen...

}
}

fn try_pushdown_filters(
Copy link
Contributor Author

Choose a reason for hiding this comment

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

cc @berkaysynnada for this implementation

Comment on lines -112 to -114
/// Check if the specified file format has support for pushing down the provided filters within
/// the given schemas. Added initially to support the Parquet file format's ability to do this.
fn supports_filters_pushdown(
Copy link
Contributor Author

Choose a reason for hiding this comment

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

Binning specialized code that was also leaking parquet stuff through DataSource and into TableProvider 😄

Copy link
Contributor

Choose a reason for hiding this comment

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

yes I agree

Since FileFormat is a pub trait, this is technically a breaking API change, but I do think it was a parquet specific optimization

I recommend we mark this PR as an API change and add a note to the upgrade guide https://github.com/apache/datafusion/blob/main/docs/source/library-user-guide/upgrading.md

I think it should basically say if you implemented FileFormat (which probably no one did) and ListingTable you will have to implement the newly added ExecutionPlan::try_pushdown_filter method into your execution plan directly if you want the filters to be pushed down

/// An enum to distinguish between different states when determining if certain filters can be
/// pushed down to file scanning
#[derive(Debug, PartialEq)]
pub enum FilePushdownSupport {
Copy link
Contributor Author

Choose a reason for hiding this comment

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

Another one of these enums!

Comment on lines -84 to +85
02)--TableScan: t_pushdown projection=[a], full_filters=[t_pushdown.b > Int32(2)]
02)--Projection: t_pushdown.a
03)----Filter: t_pushdown.b > Int32(2)
04)------TableScan: t_pushdown projection=[a, b], partial_filters=[t_pushdown.b > Int32(2)]
Copy link
Contributor Author

Choose a reason for hiding this comment

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

This is because the pushdown no longer happens at the logical level - it happens at the physical level. This makes sense, in part because the checks for suitability of pushdown are better at the physical level (there may be reasons to reject a pushdown at the physical level that are not present at the logical level, e.g. partition columns or encodings).

Copy link
Contributor

Choose a reason for hiding this comment

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

I think it makes sense and is ok that the logical plans show the filter not pushed down

Comment on lines 88 to 92
03)----DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet]]}, projection=[a], file_type=parquet, predicate=b@1 > 2, pruning_predicate=b_null_count@1 != row_count@2 AND b_max@0 > 2, required_guarantees=[]
03)----CoalesceBatchesExec: target_batch_size=8192
04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2
05)--------DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet]]}, projection=[a], file_type=parquet, predicate=b@1 > 2 AND b@1 > 2
Copy link
Contributor Author

Choose a reason for hiding this comment

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

@berkaysynnada any idea why we have extra CoalesceBatchesExec and RepartitionExec now?

Copy link
Contributor

Choose a reason for hiding this comment

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

I've a guess but not proved: CoalesceBatchesExec comes because of RepartitionExec, and RepartitionExec is inserted to satisfy partition count, which is 4. That's required by FilterExec now (which was pushed down at the logical level before), but that FilterExec is pushed down later after EnforceDistribution.

So, this makes me think about the correct order of physical rules. PushdownFilter should probably work before distribution&order satisfiers. But that could also bring some issues, I'm not sure.

Copy link
Contributor Author

@adriangb adriangb Apr 21, 2025

Choose a reason for hiding this comment

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

PushdownFilter should probably work before distribution&order satisfiers

That makes sense to me. It does more "invasive" re-arranging of plans than those do.

Copy link
Contributor

Choose a reason for hiding this comment

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

I agree it is important to remove the coalesce / repartition

Maybe we can make a separate PR to move the filter pushdown code earlier in the physical planning

An alternate could be to update the filter pushdown optimizer pass somehow to remove these -- but I think it would be cleaner / easier to understand if they were never added in the first place

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I opened #15938

@adriangb adriangb marked this pull request as ready for review April 20, 2025 01:30
@adriangb adriangb force-pushed the parquet-filter-pushdown branch from 071aa19 to ff090a7 Compare April 20, 2025 01:31
@adriangb
Copy link
Contributor Author

Thanks @adriangb -- I am about to be offline for a week so I will review this when I return

Enjoy your vacation! I think you'll like this diff:

image

Copy link
Contributor

@berkaysynnada berkaysynnada left a comment

Choose a reason for hiding this comment

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

Thank you @adriangb. I couldn't provide much design suggestions, since I cannot fully understand the need of these changes. If you provide more background information, I can help more maybe.

It seems there are some critical planning changes here, and it's better getting approvals by more people for this PR.

};
let config_pushdown_enabled = config.execution.parquet.pushdown_filters;
let table_pushdown_enabled = self.pushdown_filters();
if table_pushdown_enabled || config_pushdown_enabled {
Copy link
Contributor

Choose a reason for hiding this comment

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

OR'ing this is correct?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

The current behavior is not documented anywhere, I tried to match the existing tests:

# pushdown_filters (currently) defaults to false, but we set it here to be explicit
statement ok
set datafusion.execution.parquet.pushdown_filters = false;
statement ok
CREATE EXTERNAL TABLE t(a varchar, b int, c float) STORED AS PARQUET
LOCATION 'test_files/scratch/parquet_filter_pushdown/parquet_table/';
## Create table with pushdown enabled (pushdown setting is part of the table)
statement ok
set datafusion.execution.parquet.pushdown_filters = true;
## Create table without pushdown
statement ok
CREATE EXTERNAL TABLE t_pushdown(a varchar, b int, c float) STORED AS PARQUET
LOCATION 'test_files/scratch/parquet_filter_pushdown/parquet_table/';
# restore defaults
statement ok
set datafusion.execution.parquet.pushdown_filters = false;
# When filter pushdown is not enabled, ParquetExec only filters based on
# metadata, so a FilterExec is required to filter the
# output of the `ParquetExec`
query T
select a from t where b > 2 ORDER BY a;
----
baz
foo
NULL
NULL
NULL
query TT
EXPLAIN select a from t_pushdown where b > 2 ORDER BY a;
----
logical_plan
01)Sort: t_pushdown.a ASC NULLS LAST
02)--TableScan: t_pushdown projection=[a], full_filters=[t_pushdown.b > Int32(2)]
physical_plan
01)SortPreservingMergeExec: [a@0 ASC NULLS LAST]
02)--SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[true]
03)----DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet]]}, projection=[a], file_type=parquet, predicate=b@1 > 2, pruning_predicate=b_null_count@1 != row_count@2 AND b_max@0 > 2, required_guarantees=[]

let mut conf = self.clone();
let mut allowed_filters = vec![];
let mut remaining_filters = vec![];
for filter in &fd.filters {
Copy link
Contributor

Choose a reason for hiding this comment

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

fd.take_filters() to avoid clone's below

Copy link
Contributor Author

Choose a reason for hiding this comment

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

fd: FilterDescription,
config: &datafusion_common::config::ConfigOptions,
) -> datafusion_common::Result<FilterPushdownResult<Arc<dyn FileSource>>> {
let Some(file_schema) = self.file_schema.clone() else {
Copy link
Contributor

Choose a reason for hiding this comment

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

I'm asking to learn: in which cases ParquetSource doesn't have the schema?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I think they always end up with a schema now, but the current APIs don't require it via the constructor and instead it gets passed in via FileScanConfigBuilder. I tried piping it into the constructor but makes things difficult, there's APIs that rely on ParquetSource::default() and such. So TLDR is it's a bit gross but this is the least chrun way to do it and we can always come back later and clean the rest up.

Copy link
Contributor

Choose a reason for hiding this comment

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

maybe we file a follow on ticket

Copy link
Contributor

Choose a reason for hiding this comment

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

Let's file that follow on ticket

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Comment on lines -985 to -1015
// if we can't push it down completely with only the filename-based/path-based
// column names, then we should check if we can do parquet predicate pushdown
let supports_pushdown = self.options.format.supports_filters_pushdown(
&self.file_schema,
&self.table_schema,
&[filter],
)?;

if supports_pushdown == FilePushdownSupport::Supported {
return Ok(TableProviderFilterPushDown::Exact);
}
Copy link
Contributor

Choose a reason for hiding this comment

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

I have one question: aren't we expecting/preparing for, people to use ListingTable if they read Parquet files? Are we eventually planning to remove all format-specific handlings? Or this is a case only for filter pushdown?

Comment on lines -985 to -1015
// if we can't push it down completely with only the filename-based/path-based
// column names, then we should check if we can do parquet predicate pushdown
let supports_pushdown = self.options.format.supports_filters_pushdown(
&self.file_schema,
&self.table_schema,
&[filter],
)?;

if supports_pushdown == FilePushdownSupport::Supported {
return Ok(TableProviderFilterPushDown::Exact);
}
Copy link
Contributor

Choose a reason for hiding this comment

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

If that's the case, why don't we fully remove supports_filters_pushdown() API at all

pub(crate) file_schema: Option<SchemaRef>,
/// Optional predicate for row filtering during parquet scan
pub(crate) predicate: Option<Arc<dyn PhysicalExpr>>,
/// Optional predicate for pruning row groups (derived from `predicate`)
Copy link
Contributor

Choose a reason for hiding this comment

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

good to see these are unifying

/// The schema of the file.
/// In particular, this is the schema of the table without partition columns,
/// *not* the physical schema of the file.
pub(crate) file_schema: Option<SchemaRef>,
Copy link
Contributor

Choose a reason for hiding this comment

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

There is also another schema in FileScanConfig. Are they both reflects the file schema, not physical schema? and can we somehow unify them?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

This is the same schema that FileScanConfig passes into ParquetSource

Comment on lines 88 to 92
03)----DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet]]}, projection=[a], file_type=parquet, predicate=b@1 > 2, pruning_predicate=b_null_count@1 != row_count@2 AND b_max@0 > 2, required_guarantees=[]
03)----CoalesceBatchesExec: target_batch_size=8192
04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2
05)--------DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet]]}, projection=[a], file_type=parquet, predicate=b@1 > 2 AND b@1 > 2
Copy link
Contributor

Choose a reason for hiding this comment

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

I've a guess but not proved: CoalesceBatchesExec comes because of RepartitionExec, and RepartitionExec is inserted to satisfy partition count, which is 4. That's required by FilterExec now (which was pushed down at the logical level before), but that FilterExec is pushed down later after EnforceDistribution.

So, this makes me think about the correct order of physical rules. PushdownFilter should probably work before distribution&order satisfiers. But that could also bring some issues, I'm not sure.

@adriangb
Copy link
Contributor Author

adriangb commented Apr 22, 2025

#15812 surfaced another reason why building the predicates from the files schemas is necessary. I think once we merge this we can tackle that.

@adriangb adriangb force-pushed the parquet-filter-pushdown branch from 1af7766 to 3fde445 Compare April 22, 2025 20:53
Copy link
Contributor

@berkaysynnada berkaysynnada left a comment

Choose a reason for hiding this comment

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

There are good things here, but the main change doesn't seem correct to me. Why are we reducing the capabilities of logical optimizations? I think these planning changes will harm some people. How does it block the dynamic filtering approach?

Comment on lines -985 to -1015
// if we can't push it down completely with only the filename-based/path-based
// column names, then we should check if we can do parquet predicate pushdown
let supports_pushdown = self.options.format.supports_filters_pushdown(
&self.file_schema,
&self.table_schema,
&[filter],
)?;

if supports_pushdown == FilePushdownSupport::Supported {
return Ok(TableProviderFilterPushDown::Exact);
}
Copy link
Contributor

Choose a reason for hiding this comment

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

We can justify implementing other TableProviders for Parquet, but still I cannot understand why we need to degrade the capabilities of our ListingTable. Is't it always better pruning/simplifying things at the higher levels as possible?

@adriangb
Copy link
Contributor Author

adriangb commented Apr 24, 2025

I'm not really sure how this degrades anything. The end result is the same, users won't see any difference.

What ListingTable does currently is misguided and wrong since it is not really at the logical level as you say, instead it pierces the logical / physical separation (see how it converts Expr to PhysicalExpr, etc). It even produces bugs (I believe the pushdown of struct fields may currently be broken, or at least the implementation is confusing and the test is completely wrong).

I think there is pushdown that makes sense at the logical level, namely partition pruning. And I left that for TableProvider to continue to do. But the pruning that relies on a PhysicalExpr seems to me like it should be happening at the physical layer not the logical. It kinda gets away with it because it's the last thing that happens at the logical layer I think, but it's still smelly.

We might be able to leave the stuff in TableProvider in place but we'll be dealing with duplication and confusing methods on DataSource, which is already a complex bit of code. When I first tried to implement it this way I ran into cases with duplicate pushdown and other confusing scenarios. Probably it could have been resolved but I felt like why make one of the most complex bits in DataFusion even more complex instead of simplifying it where possible.

@berkaysynnada
Copy link
Contributor

berkaysynnada commented Apr 24, 2025

I'm not really sure how this degrades anything. The end result is the same, users won't see any difference.

Logical planning results are changing. We are also using DF end-to-end, but there could always be people only relying on logical plans of DF.

We might be able to leave the stuff in TableProvider in place but we'll be dealing with duplication and confusing methods on DataSource, which is already a complex bit of code. When I first tried to implement it this way I ran into cases with duplicate pushdown and other confusing scenarios. Probably it could have been resolved but I felt like why make one of the most complex bits in DataFusion even more complex instead of simplifying it where possible.

I'm also challenging to decide to be which side because of that complexity :D

I will take a look to other parts in this PR, and try to find a solution for the points like https://github.com/apache/datafusion/pull/15769/files#r2051612926. Maybe there isn't something wrong at all as you said, there is no harm to double check

Unlike the other PRs in this work, we might be touching some core components here. So, having a few more people review and approval would make us feel more confident.

@adriangb
Copy link
Contributor Author

I do think you make a good point of "can we keep the current thing and add the new one". It's worth a shot, at least to split the PR into two. And if that's too complicated or if we just want to simplify we can evaluate from there.

@alamb alamb mentioned this pull request Apr 29, 2025
26 tasks
alamb
alamb previously approved these changes May 1, 2025
Copy link
Contributor

@alamb alamb left a comment

Choose a reason for hiding this comment

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

Thank you @adriangb -- this change makes sense to me and I think is an improvement

The code that is moved was used to avoid adding a FilterExec when the table provider would be able to do exact filter pushdown

I am 1/2 the way through the review of this PR -- I hope to finish up shortly

Comment on lines -985 to -1015
// if we can't push it down completely with only the filename-based/path-based
// column names, then we should check if we can do parquet predicate pushdown
let supports_pushdown = self.options.format.supports_filters_pushdown(
&self.file_schema,
&self.table_schema,
&[filter],
)?;

if supports_pushdown == FilePushdownSupport::Supported {
return Ok(TableProviderFilterPushDown::Exact);
}
Copy link
Contributor

Choose a reason for hiding this comment

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

I have one question: aren't we expecting/preparing for, people to use ListingTable if they read Parquet files? Are we eventually planning to remove all format-specific handlings? Or this is a case only for filter pushdown?

For what it is worth, we (InfluxData) doesn't use ListingTable to read parquet files, instead we provide our own equivalent and create the DataSourceExec's directly

I would keep supports_filters_pushdown so that TableProviders can do Exact pruning of filters, e.g. using partition columns.

Yes I think that is important too -- I don't think we should be removing any APIs from ListingTable

return Ok(TableProviderFilterPushDown::Exact);
}

// if we can't push it down completely with only the filename-based/path-based
Copy link
Contributor

Choose a reason for hiding this comment

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

This change makes sense to me -- when @itsjunetime originally implemented this code, there was some complexity because there was no way to do filter pushdown in ExecutionPlans so in my mind this approach was a (clever) workaround

The comments even hint that this is a parquet specific special case

I think the new pattern of handling predicates more generally in this PR is cleaner and will support more cases. Since this code is only currently executed

Perhaps @cisaacson has some other thoughts

Comment on lines -112 to -114
/// Check if the specified file format has support for pushing down the provided filters within
/// the given schemas. Added initially to support the Parquet file format's ability to do this.
fn supports_filters_pushdown(
Copy link
Contributor

Choose a reason for hiding this comment

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

yes I agree

Since FileFormat is a pub trait, this is technically a breaking API change, but I do think it was a parquet specific optimization

I recommend we mark this PR as an API change and add a note to the upgrade guide https://github.com/apache/datafusion/blob/main/docs/source/library-user-guide/upgrading.md

I think it should basically say if you implemented FileFormat (which probably no one did) and ListingTable you will have to implement the newly added ExecutionPlan::try_pushdown_filter method into your execution plan directly if you want the filters to be pushed down

Comment on lines -985 to -1015
// if we can't push it down completely with only the filename-based/path-based
// column names, then we should check if we can do parquet predicate pushdown
let supports_pushdown = self.options.format.supports_filters_pushdown(
&self.file_schema,
&self.table_schema,
&[filter],
)?;

if supports_pushdown == FilePushdownSupport::Supported {
return Ok(TableProviderFilterPushDown::Exact);
}
Copy link
Contributor

Choose a reason for hiding this comment

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

We can justify implementing other TableProviders for Parquet, but still I cannot understand why we need to degrade the capabilities of our ListingTable. Is't it always better pruning/simplifying things at the higher levels as possible?

I don't think this degrades the capabilities of the current listing table. I think the only implications are for anyone who used a custom FileFormat and impleented supports_filters_pushdown -- I suspect this is not very common and we can likely avoid consternation by mentioning it in the upgrade guide (see comment below)

base_config,
predicate,
pruning_predicate: parquet.pruning_predicate,
pruning_predicate: None, // for backwards compat since `ParquetExec` is only for backwards compat anyway
Copy link
Contributor

Choose a reason for hiding this comment

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

yeah this is fine too in my opinion. It is almost time to remove ParquetExec anyways -- maybe we should just do it in this release 🤔

Comment on lines 562 to 580
let pruning_predicate_string = self
.pruning_predicate
.as_ref()
.map(|pre| {
let mut guarantees = pre
.literal_guarantees()
.iter()
.map(|item| format!("{}", item))
.collect_vec();
guarantees.sort();
format!(
", pruning_predicate={}, required_guarantees=[{}]",
pre.predicate_expr(),
guarantees.join(", ")
)
})
.unwrap_or_default();
Copy link
Contributor

Choose a reason for hiding this comment

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

I think it is important to keep these in the physical plans -- in particular what I think is important is to be able to check via the explain plan if pruning is happening by looking at the explain plan

@alamb alamb dismissed their stale review May 1, 2025 13:18

clicked wrong button

Copy link
Contributor

@alamb alamb left a comment

Choose a reason for hiding this comment

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

Ok, I went through this PR and TLDR is I think it is an improvement. Thank you very much @adriangb

I left several suggestions on how to improve it / the tests, but I also think we could do that as a follow on PR.

I think this is a really nice step forward -- and while it is taking a long time I am confident it will be worth it in the end

fd: FilterDescription,
config: &datafusion_common::config::ConfigOptions,
) -> datafusion_common::Result<FilterPushdownResult<Arc<dyn FileSource>>> {
let Some(file_schema) = self.file_schema.clone() else {
Copy link
Contributor

Choose a reason for hiding this comment

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

maybe we file a follow on ticket

Comment on lines -84 to +85
02)--TableScan: t_pushdown projection=[a], full_filters=[t_pushdown.b > Int32(2)]
02)--Projection: t_pushdown.a
03)----Filter: t_pushdown.b > Int32(2)
04)------TableScan: t_pushdown projection=[a, b], partial_filters=[t_pushdown.b > Int32(2)]
Copy link
Contributor

Choose a reason for hiding this comment

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

I think it makes sense and is ok that the logical plans show the filter not pushed down

Comment on lines 88 to 92
03)----DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet]]}, projection=[a], file_type=parquet, predicate=b@1 > 2, pruning_predicate=b_null_count@1 != row_count@2 AND b_max@0 > 2, required_guarantees=[]
03)----CoalesceBatchesExec: target_batch_size=8192
04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2
05)--------DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet]]}, projection=[a], file_type=parquet, predicate=b@1 > 2 AND b@1 > 2
Copy link
Contributor

Choose a reason for hiding this comment

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

I agree it is important to remove the coalesce / repartition

Maybe we can make a separate PR to move the filter pushdown code earlier in the physical planning

An alternate could be to update the filter pushdown optimizer pass somehow to remove these -- but I think it would be cleaner / easier to understand if they were never added in the first place

logical_plan TableScan: t projection=[a], full_filters=[t.a != Int32(100)]
logical_plan
01)Filter: t.a != Int32(100)
02)--TableScan: t projection=[a], partial_filters=[t.a != Int32(100)]
Copy link
Contributor

Choose a reason for hiding this comment

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

I recommend we change these tests to show the physical plans (not the logical plans) as that would more accurately show the pushdown happening. Maybe also something we could do as a separate PR

Copy link
Contributor Author

Choose a reason for hiding this comment

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

@alamb do you know why these don't display the phyiscal plan already? Is something parsing them out?

Copy link
Contributor

Choose a reason for hiding this comment

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

There is a statement in 21st line in this file:
set datafusion.explain.logical_plan_only = true;

There is also counter-part of that: set datafusion.explain.physical_plan_only = true

You can set the display type of explain queries with these.

@berkaysynnada
Copy link
Contributor

@adriangb do you have time to address the last suggestions? I understand the mistake here, and I think we should take this in asap

@adriangb
Copy link
Contributor Author

adriangb commented May 3, 2025

@adriangb do you have time to address the last suggestions? I understand the mistake here, and I think we should take this in asap

I am going to try to address the last round of review later today on a flight. In particular:

Is there anything I'm missing? Is this what you meant by the mistake?

@github-actions github-actions bot added the optimizer Optimizer rules label May 4, 2025
@adriangb
Copy link
Contributor Author

adriangb commented May 4, 2025

I updated the order of the pushdown rules in this PR, it worked to get rid of the extra nodes.

I've also added the upgrade guide and the pushdown preview is being shown in the physical plans.

@alamb I think the only point missing is #15769 (comment) which I need a big of guidance on

Copy link
Contributor

@berkaysynnada berkaysynnada left a comment

Choose a reason for hiding this comment

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

Once the conflicts are resolved, I'll merge this: Thanks @adriangb

logical_plan TableScan: t projection=[a], full_filters=[t.a != Int32(100)]
logical_plan
01)Filter: t.a != Int32(100)
02)--TableScan: t projection=[a], partial_filters=[t.a != Int32(100)]
Copy link
Contributor

Choose a reason for hiding this comment

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

There is a statement in 21st line in this file:
set datafusion.explain.logical_plan_only = true;

There is also counter-part of that: set datafusion.explain.physical_plan_only = true

You can set the display type of explain queries with these.

@adriangb adriangb force-pushed the parquet-filter-pushdown branch from e9cb59b to 7924c97 Compare May 5, 2025 08:31
@berkaysynnada
Copy link
Contributor

I think you can disable logical plans. That's what @alamb suggests, if I don't get it wrong

@adriangb
Copy link
Contributor Author

adriangb commented May 5, 2025

I think you can disable logical plans. That's what @alamb suggests, if I don't get it wrong

done!

@berkaysynnada
Copy link
Contributor

I think we should also inform people who might be possibly effected by these logical plan changes

@andygrove

@adriangb adriangb force-pushed the parquet-filter-pushdown branch from 5665a2b to d3057ff Compare May 5, 2025 15:53
@adriangb
Copy link
Contributor Author

adriangb commented May 5, 2025

Okay folks this is rebased and ready for another round of review

Copy link
Contributor

@alamb alamb left a comment

Choose a reason for hiding this comment

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

Thank you @adriangb -- I think it looks great. I'll plan to merge it tomorrow unless anyone else would like a chance to review

use url::Url;

#[tokio::test]
async fn parquet_partition_pruning_filter() -> Result<()> {
Copy link
Contributor

Choose a reason for hiding this comment

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

can you remind me again why this test was removed ?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

It was relying on ListingTable to do the pushdown without the optimizer pass.
I felt that it was better suited for an .slt test that is fully wired up, so I essentially moved it: https://github.com/apache/datafusion/pull/15769/files#r2074193501

Copy link
Contributor

@berkaysynnada berkaysynnada May 6, 2025

Choose a reason for hiding this comment

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

can we also apply the changes similar to the test in datafusion/core/tests/parquet/file_statistics.rs to keep this test for now?
I saw the new slt now 👍🏻

base_config,
predicate,
pruning_predicate: parquet.pruning_predicate,
pruning_predicate: None, // for backwards compat since `ParquetExec` is only for backwards compat anyway
Copy link
Contributor

Choose a reason for hiding this comment

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

fd: FilterDescription,
config: &datafusion_common::config::ConfigOptions,
) -> datafusion_common::Result<FilterPushdownResult<Arc<dyn FileSource>>> {
let Some(file_schema) = self.file_schema.clone() else {
Copy link
Contributor

Choose a reason for hiding this comment

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

Let's file that follow on ticket


statement ok
set datafusion.explain.logical_plan_only = true;
set datafusion.explain.physical_plan_only = true;
Copy link
Contributor

Choose a reason for hiding this comment

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

💯

02)--TableScan: test_filter_with_limit projection=[part_key, value], full_filters=[test_filter_with_limit.value = Int32(2)], fetch=1
physical_plan
01)CoalescePartitionsExec: fetch=1
02)--DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_filter_with_limit/part-0.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_filter_with_limit/part-1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_filter_with_limit/part-2.parquet]]}, projection=[part_key, value], limit=1, file_type=parquet, predicate=value@1 = 2, pruning_predicate=value_null_count@2 != row_count@3 AND value_min@0 <= 2 AND 2 <= value_max@1, required_guarantees=[value in (2)]
Copy link
Contributor

Choose a reason for hiding this comment

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

it is good to see this the predicate pushed down

the `return_field` function. There are numerous examples in the `physical-expr`
crate.

### `FileFormat::supports_filters_pushdown` replaced with `FileSource::try_pushdown_filters`
Copy link
Contributor

Choose a reason for hiding this comment

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

👍

@alamb
Copy link
Contributor

alamb commented May 5, 2025

It looks like there are a few doc errors and doc example errors

Update filed

Comment on lines +234 to +248
# If we have a mix of filters:
# - The partition filters get evaluated during planning
# - The mixed filters end up in a FilterExec
# - The file filters get pushed down into the scan
query TT
EXPLAIN select * from t_pushdown where val != 'd' AND val != 'c' AND part = 'a' AND part != val;
----
logical_plan
01)Filter: t_pushdown.val != Utf8("d") AND t_pushdown.val != Utf8("c") AND t_pushdown.val != t_pushdown.part
02)--TableScan: t_pushdown projection=[val, part], full_filters=[t_pushdown.part = Utf8("a")], partial_filters=[t_pushdown.val != Utf8("d"), t_pushdown.val != Utf8("c"), t_pushdown.val != t_pushdown.part]
physical_plan
01)CoalesceBatchesExec: target_batch_size=8192
02)--FilterExec: val@0 != part@1
03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1
04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=a/file.parquet]]}, projection=[val, part], file_type=parquet, predicate=val@0 != d AND val@0 != c, pruning_predicate=val_null_count@2 != row_count@3 AND (val_min@0 != d OR d != val_max@1) AND val_null_count@2 != row_count@3 AND (val_min@0 != c OR c != val_max@1), required_guarantees=[val not in (c, d)]
Copy link
Contributor Author

Choose a reason for hiding this comment

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

Replaces deleted test

Comment on lines +98 to +101
// The FilterPushdown rule tries to push down filters as far as it can.
// For example, it will push down filtering from a `FilterExec` to
// a `DataSourceExec`, or from a `TopK`'s current state to a `DataSourceExec`.
Arc::new(FilterPushdown::new()),
Copy link
Contributor Author

Choose a reason for hiding this comment

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

@alamb this is actually an issue, moving this does not work alongside #15770:

  • SortExecs filter has to be pushed down after any manipulation of the SortExecs in the plan (otherwise it will be referencing a SortExec that gets removed / replaced and not function)
  • EnforceSorting creates new SortExecs, so FilterPushdown would have to come after that
  • The extra RepartitionExecs get introduced in EnforceDistribution
  • Since EnforceSorting must come after EnforceDistribution there is no correct place to put FilterPushdown

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I guess the only logical solution is to place FilterPushdown at the end and remove any unnecessary repartitions?

Copy link
Contributor

Choose a reason for hiding this comment

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

If this version is okay in this PR, I can take a look at that in #15770. I've some ideas, like making FilterPushdown smarter to remove any unnecessary patterns behind

use url::Url;

#[tokio::test]
async fn parquet_partition_pruning_filter() -> Result<()> {
Copy link
Contributor

@berkaysynnada berkaysynnada May 6, 2025

Choose a reason for hiding this comment

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

can we also apply the changes similar to the test in datafusion/core/tests/parquet/file_statistics.rs to keep this test for now?
I saw the new slt now 👍🏻

Comment on lines +98 to +101
// The FilterPushdown rule tries to push down filters as far as it can.
// For example, it will push down filtering from a `FilterExec` to
// a `DataSourceExec`, or from a `TopK`'s current state to a `DataSourceExec`.
Arc::new(FilterPushdown::new()),
Copy link
Contributor

Choose a reason for hiding this comment

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

If this version is okay in this PR, I can take a look at that in #15770. I've some ideas, like making FilterPushdown smarter to remove any unnecessary patterns behind

}

/// Are all filters marked as [`PredicateSupport::Supported`]?
pub fn is_exact(&self) -> bool {
Copy link
Contributor

Choose a reason for hiding this comment

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

can we find a better name for this? and it's not used, maybe removed until it's needed?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Already removed in 15443d5

@berkaysynnada
Copy link
Contributor

Will be merged after resolving conflicts 🚀

@adriangb adriangb force-pushed the parquet-filter-pushdown branch from 3a108b8 to 34adfe3 Compare May 6, 2025 12:31
@berkaysynnada berkaysynnada merged commit 55ba4ca into apache:main May 6, 2025
29 checks passed
@alamb
Copy link
Contributor

alamb commented May 6, 2025

WOOHOO!

&self,
state: &dyn Session,
conf: FileScanConfig,
filters: Option<&Arc<dyn PhysicalExpr>>,
Copy link
Member

Choose a reason for hiding this comment

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

This is definitely a breaking API change, we depend on the interface intensively 😢

Copy link
Contributor

Choose a reason for hiding this comment

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

I feel ilke we have done this a few times now @xudong963 (broken some APIs you are relying on heavily)

I wonder if there is some way to make a tests / example that illustrates what you are doing? That would have some benefits:

  1. it would illustrate exactly the way the API needs to be changed on upgrade
  2. it would make it clearer when changes to APIs have impacts on you (and other's usecases)

Copy link
Member

Choose a reason for hiding this comment

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

Thank you @alamb, I'll add this to my TODO list.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Agreed. Sorry for the breakage. Do the new APIs get the job done for you, or is there something now missing?

Copy link
Member

Choose a reason for hiding this comment

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

I've found a way to work around the change, not a big deal, thanks for your quick response

@xudong963
Copy link
Member

There is a bug related to the PR: #16563

mach-kernel added a commit to spiceai/datafusion that referenced this pull request Aug 1, 2025
* Fix: fetch is lost in replace_order_preserving_variants method during EnforceDistribution (#15808)

* Speed up `optimize_projection` (#15787)

* save

* fmt

* Support WITHIN GROUP syntax to standardize certain existing aggregate functions  (#13511)

* Add within group variable to aggregate function and arguments

* Support within group and disable null handling for ordered set aggregate functions (#13511)

* Refactored function to match updated signature

* Modify proto to support within group clause

* Modify physical planner and accumulator to support ordered set aggregate function

* Support session management for ordered set aggregate functions

* Align code, tests, and examples with changes to aggregate function logic

* Ensure compatibility with new `within_group` and `order_by` handling.

* Adjust tests and examples to align with the new logic.

* Fix typo in existing comments

* Enhance test

* Add test cases for changed signature

* Update signature in docs

* Fix bug : handle missing within_group when applying children tree node

* Change the signature of approx_percentile_cont for consistency

* Add missing within_group for expr display

* Handle edge case when over and within group clause are used together

* Apply clippy advice: avoids too many arguments

* Add new test cases using descending order

* Apply cargo fmt

* Revert unintended submodule changes

* Apply prettier guidance

* Apply doc guidance by update_function_doc.sh

* Rollback WITHIN GROUP and related logic after converting it into expr

* Make it not to handle redundant logic

* Rollback ordered set aggregate functions from session to save same info in udf itself

* Convert within group to order by when converting sql to expr

* Add function to determine it is ordered-set aggregate function

* Rollback within group from proto

* Utilize within group as order by in functions-aggregate

* Apply clippy

* Convert order by to within group

* Apply cargo fmt

* Remove plain line breaks

* Remove duplicated column arg in schema name

* Refactor boolean functions to just return primitive type

* Make within group necessary in the signature of existing ordered set aggr funcs

* Apply cargo fmt

* Support a single ordering expression in the signature

* Apply cargo fmt

* Add dataframe function test cases to verify descending ordering

* Apply cargo fmt

* Apply code reviews

* Uses order by consistently after done with sql

* Remove redundant comment

* Serve more clear error msg

* Handle error cases in the same code block

* Update error msg in test as corresponding code changed

* fix

---------

Co-authored-by: Jay Zhan <[email protected]>

* docs: add ArkFlow (#15826)

* chore(deps): bump env_logger from 0.11.7 to 0.11.8 (#15823)

Bumps [env_logger](https://github.com/rust-cli/env_logger) from 0.11.7 to 0.11.8.
- [Release notes](https://github.com/rust-cli/env_logger/releases)
- [Changelog](https://github.com/rust-cli/env_logger/blob/main/CHANGELOG.md)
- [Commits](https://github.com/rust-cli/env_logger/compare/v0.11.7...v0.11.8)

---
updated-dependencies:
- dependency-name: env_logger
  dependency-version: 0.11.8
  dependency-type: direct:production
  update-type: version-update:semver-patch
...

Signed-off-by: dependabot[bot] <[email protected]>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>

* Support unparsing `UNION` for distinct results (#15814)

* Add `MemoryPool::memory_limit`  to expose setting memory usage limit (#15828)

* add `memory_limit` to `MemoryPool`, and impl it for the pools in datafusion.

* Update datafusion/execution/src/memory_pool/mod.rs

Co-authored-by: Ruihang Xia <[email protected]>

---------

Co-authored-by: Ruihang Xia <[email protected]>

* Preserve projection for inline scan (#15825)

* Preserve projection for inline scan

* fix

---------

Co-authored-by: Vadim Piven <[email protected]>

* cleanup after emit (#15834)

* chore(deps): bump pyo3 from 0.24.1 to 0.24.2 (#15838)

Bumps [pyo3](https://github.com/pyo3/pyo3) from 0.24.1 to 0.24.2.
- [Release notes](https://github.com/pyo3/pyo3/releases)
- [Changelog](https://github.com/PyO3/pyo3/blob/main/CHANGELOG.md)
- [Commits](https://github.com/pyo3/pyo3/compare/v0.24.1...v0.24.2)

---
updated-dependencies:
- dependency-name: pyo3
  dependency-version: 0.24.2
  dependency-type: direct:production
  update-type: version-update:semver-patch
...

Signed-off-by: dependabot[bot] <[email protected]>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>

* Fix: fetch is missing in `EnforceSorting` optimizer (two places) (#15822)

* Fix: fetch is missing in EnforceSort

* add ut test_parallelize_sort_preserves_fetch

* add ut: test_plan_with_order_preserving_variants_preserves_fetch

* update

* address comments

* Minor: fix potential flaky test in aggregate.slt (#15829)

* Fix `ILIKE` expression support in SQL unparser (#15820)

* Fix ILIKE expression support in SQL unparser (#76)

* update tests

* Make `Diagnostic` easy/convinient to attach by using macro and avoiding `map_err` (#15796)

* First Step

* Final Step?

* Homogenisation

* Feature/benchmark config from env (#15782)

* Read benchmark SessionConfig from env

* Set target partitions from env by default

fix

* Set batch size from env by default

* Fix batch size option for tpch ci

* Log environment variable configuration

* Document benchmarking env variable config

* Add DATAFUSION_* env config to Error: unknown command: help

Orchestrates running benchmarks against DataFusion checkouts

Usage:
./bench.sh data [benchmark] [query]
./bench.sh run [benchmark]
./bench.sh compare <branch1> <branch2>
./bench.sh venv

**********
Examples:
**********
# Create the datasets for all benchmarks in /Users/christian/MA/datafusion/benchmarks/data
./bench.sh data

# Run the 'tpch' benchmark on the datafusion checkout in /source/datafusion
DATAFUSION_DIR=/source/datafusion ./bench.sh run tpch

**********
* Commands
**********
data:         Generates or downloads data needed for benchmarking
run:          Runs the named benchmark
compare:      Compares results from benchmark runs
venv:         Creates new venv (unless already exists) and installs compare's requirements into it

**********
* Benchmarks
**********
all(default): Data/Run/Compare for all benchmarks
tpch:                   TPCH inspired benchmark on Scale Factor (SF) 1 (~1GB), single parquet file per table, hash join
tpch_mem:               TPCH inspired benchmark on Scale Factor (SF) 1 (~1GB), query from memory
tpch10:                 TPCH inspired benchmark on Scale Factor (SF) 10 (~10GB), single parquet file per table, hash join
tpch_mem10:             TPCH inspired benchmark on Scale Factor (SF) 10 (~10GB), query from memory
cancellation:           How long cancelling a query takes
parquet:                Benchmark of parquet reader's filtering speed
sort:                   Benchmark of sorting speed
sort_tpch:              Benchmark of sorting speed for end-to-end sort queries on TPCH dataset
clickbench_1:           ClickBench queries against a single parquet file
clickbench_partitioned: ClickBench queries against a partitioned (100 files) parquet
clickbench_extended:    ClickBench "inspired" queries against a single parquet (DataFusion specific)
external_aggr:          External aggregation benchmark
h2o_small:              h2oai benchmark with small dataset (1e7 rows) for groupby,  default file format is csv
h2o_medium:             h2oai benchmark with medium dataset (1e8 rows) for groupby, default file format is csv
h2o_big:                h2oai benchmark with large dataset (1e9 rows) for groupby,  default file format is csv
h2o_small_join:         h2oai benchmark with small dataset (1e7 rows) for join,  default file format is csv
h2o_medium_join:        h2oai benchmark with medium dataset (1e8 rows) for join, default file format is csv
h2o_big_join:           h2oai benchmark with large dataset (1e9 rows) for join,  default file format is csv
imdb:                   Join Order Benchmark (JOB) using the IMDB dataset converted to parquet

**********
* Supported Configuration (Environment Variables)
**********
DATA_DIR            directory to store datasets
CARGO_COMMAND       command that runs the benchmark binary
DATAFUSION_DIR      directory to use (default /Users/christian/MA/datafusion/benchmarks/..)
RESULTS_NAME        folder where the benchmark files are stored
PREFER_HASH_JOIN    Prefer hash join algorithm (default true)
VENV_PATH           Python venv to use for compare and venv commands (default ./venv, override by <your-venv>/bin/activate)
DATAFUSION_*        Set the given datafusion configuration

* fmt

* predicate pruning: support cast and try_cast for more types (#15764)

* predicate pruning: support dictionaries

* more types

* clippy

* add tests

* add tests

* simplify to dicts

* revert most changes

* just check for strings, more tests

* more tests

* remove unecessary now confusing clause

* Fix: fetch is missing in plan_with_order_breaking_variants method (#15842)

* Fix `CoalescePartitionsExec` proto serialization (#15824)

* add fetch to CoalescePartitionsExecNode

* gen proto code

* Add test

* fix

* fix build

* Fix test build

* remove comments

* Fix build (#15849)

* Fix scalar list comparison when the compared lists have different lengths (#15856)

* chore: More details to `No UDF registered` error (#15843)

* chore(deps): bump clap from 4.5.36 to 4.5.37 (#15853)

Bumps [clap](https://github.com/clap-rs/clap) from 4.5.36 to 4.5.37.
- [Release notes](https://github.com/clap-rs/clap/releases)
- [Changelog](https://github.com/clap-rs/clap/blob/master/CHANGELOG.md)
- [Commits](https://github.com/clap-rs/clap/compare/clap_complete-v4.5.36...clap_complete-v4.5.37)

---
updated-dependencies:
- dependency-name: clap
  dependency-version: 4.5.37
  dependency-type: direct:production
  update-type: version-update:semver-patch
...

Signed-off-by: dependabot[bot] <[email protected]>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>

* Remove usage of `dbg!` (#15858)

* Fix `from_unixtime` function documentation (#15844)

* Fix `from_unixtime` function documentation

* Update scalar_functions.md

* Minor: Interval singleton (#15859)

* interval singleron

* fmt

* impl from

* Make aggr fuzzer query builder more configurable (#15851)

* refactor and make `QueryBuilder` more configurable.

* fix tests.

* fix clippy.

* extract `QueryBuilder` to a dedicated module.

* add `min_group_by_columns`, and fix some bugs.

* chore(deps): bump aws-config from 1.6.1 to 1.6.2 (#15874)

Bumps [aws-config](https://github.com/smithy-lang/smithy-rs) from 1.6.1 to 1.6.2.
- [Release notes](https://github.com/smithy-lang/smithy-rs/releases)
- [Changelog](https://github.com/smithy-lang/smithy-rs/blob/main/CHANGELOG.md)
- [Commits](https://github.com/smithy-lang/smithy-rs/commits)

---
updated-dependencies:
- dependency-name: aws-config
  dependency-version: 1.6.2
  dependency-type: direct:production
  update-type: version-update:semver-patch
...

Signed-off-by: dependabot[bot] <[email protected]>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>

* Add slt tests for `datafusion.execution.parquet.coerce_int96` setting (#15723)

* Add slt tests for datafusion.execution.parquet.coerce_int96 setting

* tweak

* Improve `ListingTable` / `ListingTableOptions` docs (#15767)

* Improve `ListingTable` / `ListingTableOptions` docs

* Update datafusion/core/src/datasource/listing/table.rs

Co-authored-by: Alex Huang <[email protected]>

---------

Co-authored-by: Alex Huang <[email protected]>

* Upgrade-guide: Downgrade "FileScanConfig –> FileScanConfigBuilder" headline (#15883)

I noticed that https://datafusion.apache.org/library-user-guide/upgrading.html#filescanconfig-filescanconfigbuilder had "FileScanConfig –> FileScanConfigBuilder" as a top-level headline. It should probably be under the 47 release

* Migrate Optimizer tests to insta, part2 (#15884)

* migrate tests in `replace_distinct_aggregate.rs`

* migrate tests in `replace_distinct_aggregate.rs`

* migrate tests in `push_down_limit.rs`

* migrate tests in `eliminate_duplicated_expr.rs`

* migrate tests in `eliminate_filter.rs`

* migrate tests in `eliminate_group_by_constant.rs` to insta

* migrate tests in `eliminate_join.rs` to use snapshot assertions

* migrate tests in `eliminate_nested_union.rs` to use snapshot assertions

* migrate tests in `eliminate_outer_join.rs` to use snapshot assertions

* migrate tests in `filter_null_join_keys.rs` to use snapshot assertions

* fix Type inferance

* fix macro to use crate path for OptimizerContext and Optimizer

* clean up

* fix: Avoid mistaken ILike to string equality optimization (#15836)

* fix: Avoid mistaken ILike to string equality optimization

* test: ILIKE without wildcards

* Improve documentation for `FileSource`, `DataSource` and `DataSourceExec` (#15766)

* Improve documentation for FileSource

* more

* Update datafusion/datasource/src/file.rs

Co-authored-by: Adrian Garcia Badaracco <[email protected]>

* Clippy

* fmt

---------

Co-authored-by: Adrian Garcia Badaracco <[email protected]>

* allow min max dictionary (#15827)

* Map file-level column statistics to the table-level (#15865)

* init

* fix clippy

* add test

* chore(deps): bump blake3 from 1.8.1 to 1.8.2 (#15890)

Bumps [blake3](https://github.com/BLAKE3-team/BLAKE3) from 1.8.1 to 1.8.2.
- [Release notes](https://github.com/BLAKE3-team/BLAKE3/releases)
- [Commits](https://github.com/BLAKE3-team/BLAKE3/compare/1.8.1...1.8.2)

---
updated-dependencies:
- dependency-name: blake3
  dependency-version: 1.8.2
  dependency-type: direct:production
  update-type: version-update:semver-patch
...

Signed-off-by: dependabot[bot] <[email protected]>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>

* Respect ignore_nulls in array_agg (#15544)

* Respect ignore_nulls in array_agg

* Reduce code duplication

* Add another test

* Set HashJoin seed (#15783)

* Set HashJoin seed

* fmt

* whitespace grr

* Document hash seed

Co-authored-by: Alex Huang <[email protected]>

---------

Co-authored-by: Alex Huang <[email protected]>

* Add Extension Type / Metadata support for Scalar UDFs (#15646)

* Add in plumbing to pass around metadata for physical expressions

* Adding argument metadata to scalar argument struct

* Since everywhere we use this we immediately clone, go ahead and returned an owned version of the metadata for simplicity

* Cargo fmt

* Benchmarks required args_metadata in tests

* Clippy warnings

* Switching over to passing Field around instead of metadata so we can handle extension types directly

* Switching return_type_from_args to return_field_from_args

* Updates to unit tests for switching to field instead of data_type

* Resolve unit test issues

* Update after rebase on main

* GetFieldFunc should return the field it finds instead of creating a new one

* Get metadata from scalar functions

* Change expr_schema to use to_field primarily instead of individual calls for getting data type, nullability, and schema

* Scalar function arguments should take return field instead of return data type now

* subquery should just get the field from below and not lose potential metadata

* Update comment

* Remove output_field now that we've determined it using return_field_from_args

* Change name to_field to field_from_column to be more consistent with the usage and prevent misconception about if we are doing some conversion

* Minor moving around of the explicit lifetimes in the struct definition

* Change physical expression to require to output a field which requires a lot of unit test updates, especially because the scalar arguments pass around borrowed values

* Change name from output_field to return_field to be more consistent

* Update migration guide for DF48 with user defined functions

* Whitespace

* Docstring correction

* chore: fix clippy::large_enum_variant for DataFusionError (#15861)

* Saner handling of nulls inside arrays (#15149)

* Saner handling of nulls inside arrays

* Fix array_sort for empty record batch

* Fix get_valid_types for FixedSizeLists

* Optimize array_ndims

* Add a test for result type of Concatenating Mixed types

* Fix array_element of empty array

* Handle more FixedSizeLists

* Feat: introduce `ExecutionPlan::partition_statistics` API (#15852)

* save

* save

* save

* functional way

* fix sort

* adding test

* add tests

* save

* update

* add PartitionedStatistics structure

* use Arc

* refine tests

* save

* resolve conflicts

* use PartitionedStatistics

* impl index and len for PartitionedStatistics

* add test for cross join

* fix clippy

* Check the statistics_by_partition with real results

* rebase main and fix cross join test

* resolve conflicts

* Feat: introduce partition statistics API

* address comments

* deprecated statistics API

* rebase main and fix tests

* fix

* Keeping pull request in sync with the base branch (#15894)

* fix: cast inner fsl to list in flatten (#15898)

* support OR operator in binary `evaluate_bounds` (#15716)

* support OR operator in binary `evaluate_bounds`

* fixup tests

* feat: Add option to adjust writer buffer size for query output (#15747)

* Add execution config option to set buffer size

* Document new configuration option (#15656)

* Minor documentation correction (#15656)

* Add default to documentation (#15656)

* Minor doc. fix and correct failing tests (#15656)

* Fix test (#15656)

* Updated with Builder API

---------

Co-authored-by: m09526 <[email protected]>

* infer placeholder datatype for IN lists (#15864)

* infer placeholder datatype for IN lists

* infer placeholder datatype for Expr::Like

* add tests for Expr::SimilarTo

---------

Co-authored-by: Kevin <[email protected].>

* Update known users (#15895)

* fix(avro): Respect projection order in Avro reader (#15840)

Fixed issue in the Avro reader that caused queries to fail when columns
were reordered in the SELECT statement. The reader now correctly:

1. Builds arrays in the order specified in the projection
2. Creates a properly ordered schema matching the projection

Previously when selecting columns in a different order than the original
schema (e.g., `SELECT timestamp, username FROM avro_table`), the reader
would produce error due to type mismatches between the data arrays and
the expected schema.

Fixes #15839

* Fix allow_update_branch (#15904)

* fix: correctly specify the nullability of `map_values` return type (#15901)

Co-authored-by: Andrew Lamb <[email protected]>

* Add `union_tag` scalar function (#14687)

* feat: add union_tag scalar function

* update for new api

* Add test for second field type

---------

Co-authored-by: Andrew Lamb <[email protected]>

* chore(deps): bump tokio from 1.44.1 to 1.44.2 (#15900)

Bumps [tokio](https://github.com/tokio-rs/tokio) from 1.44.1 to 1.44.2.
- [Release notes](https://github.com/tokio-rs/tokio/releases)
- [Commits](https://github.com/tokio-rs/tokio/compare/tokio-1.44.1...tokio-1.44.2)

---
updated-dependencies:
- dependency-name: tokio
  dependency-version: 1.44.2
  dependency-type: direct:production
  update-type: version-update:semver-patch
...

Signed-off-by: dependabot[bot] <[email protected]>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
Co-authored-by: xudong.w <[email protected]>

* chore(deps): bump assert_cmd from 2.0.16 to 2.0.17 (#15909)

Bumps [assert_cmd](https://github.com/assert-rs/assert_cmd) from 2.0.16 to 2.0.17.
- [Changelog](https://github.com/assert-rs/assert_cmd/blob/master/CHANGELOG.md)
- [Commits](https://github.com/assert-rs/assert_cmd/compare/v2.0.16...v2.0.17)

---
updated-dependencies:
- dependency-name: assert_cmd
  dependency-version: 2.0.17
  dependency-type: direct:production
  update-type: version-update:semver-patch
...

Signed-off-by: dependabot[bot] <[email protected]>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>

* Factor out Substrait consumers into separate files (#15794)

* Factor out Substrait consumers into separate files

* Move relations and expressions into their own modules

* Refactor: rename rex to expr

* Refactor: move from_substrait_extended_expr to mod.rs

---------

Co-authored-by: Andrew Lamb <[email protected]>

* Unparse `UNNEST` projection with the table column alias (#15879)

* add table column alias for unnest projection

* fix clippy

* fix columns check

* feat: Add `datafusion-spark` crate (#15168)

* feat: Add datafusion-spark crate

* spark crate setup

* clean up 2 example functions

* cleanup crate

* Spark crate setup

* fix lint issue

* cargo cleanup

* fix collision in sqllogic

* remove redundant test

* test float precision when casting to string

* reorder

* undo

* save

* save

* save

* add spark crate

* remove spark from core

* add comment to import tests

* Fix: reset submodule to main pointer and clean state

* Save

* fix registration

* modify float64 precision for spark

* Update datafusion/spark/src/lib.rs

Co-authored-by: Andrew Lamb <[email protected]>

* clean up code

* code cleanup

---------

Co-authored-by: Andrew Lamb <[email protected]>

* Fix typo in introduction.md (#15910)

- Fix typo in introduction.md
- Remove period from end of bullet point to maintain consistency with other bullet points

* Fix CI in main (#15917)

* Migrate Optimizer tests to insta, part3 (#15893)

* migrate tests in `push_down_filters.rs` to use snapshot assertions

* remove unused format checks

* Revert "remove unused format checks"

This reverts commit dc4f137c7fc8cf642c8dbb158fbbb5526c69e051.

* migrate `assert_eq!` in `push_down_filters.rs` to use snapshot assertions

* migrate `assert_eq!` in `push_down_filters.rs` to use snapshot assertions

---------

Co-authored-by: Dmitrii Blaginin <[email protected]>

* Add `FormatOptions` to Config (#15793)

* Add `FormatOptions` to Config

* Fix `output_with_header`

* Add cli test

* Add `to_string`

* Prettify

* Prettify

* Preserve the initial `NULL` logic

* Cleanup

* Remove `lt` as no longer needed

* Format assert

* Fix sqllogictest

* Fix tests

* Set formatting params for dates / times

* Lowercase `duration_format`

---------

Co-authored-by: Andrew Lamb <[email protected]>

* Minor: cleanup datafusion-spark scalar functions (#15921)

* Fix ClickBench extended queries after update to APPROX_PERCENTILE_CONT (#15929)

* fix: SqlLogicTest on Windows (#15932)

* docs: Label `bloom_filter_on_read` as a reading config (#15933)

* docs: Label �loom_filter_on_read as a reading config

* fix: Update configs.md

* Add extended query for checking improvement for blocked groups optimization (#15936)

* add query to show improvement for 15591.

* document the new added query.

* Character length (#15931)

* chore(deps): bump tokio-util from 0.7.14 to 0.7.15 (#15918)

Bumps [tokio-util](https://github.com/tokio-rs/tokio) from 0.7.14 to 0.7.15.
- [Release notes](https://github.com/tokio-rs/tokio/releases)
- [Commits](https://github.com/tokio-rs/tokio/compare/tokio-util-0.7.14...tokio-util-0.7.15)

---
updated-dependencies:
- dependency-name: tokio-util
  dependency-version: 0.7.15
  dependency-type: direct:production
  update-type: version-update:semver-patch
...

Signed-off-by: dependabot[bot] <[email protected]>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
Co-authored-by: xudong.w <[email protected]>

* Migrate Optimizer tests to insta, part4 (#15937)

* migrate `assert_eq!` in `optimize_projection/mod.rs` to use snapshot assertions

* migrate `assert_optimized_plan_equal!` in `propagate_empty_relations.rs` to use snapshot assertions

* remove all `assert_optimized_plan_eq`

* migrate `assert_optimized_plan_equal!` in `decorrelate_predicate_subquery.rs` to use snapshot assertions

* Add snapshot assertion macro for optimized plan equality checks

---------

Co-authored-by: Dmitrii Blaginin <[email protected]>

* fix query results for predicates referencing partition columns and data columns (#15935)

* fix query results for predicates referencing partition columns and data columns

* fmt

* add e2e test

* newline

* chore(deps): bump substrait from 0.55.0 to 0.55.1 (#15941)

Bumps [substrait](https://github.com/substrait-io/substrait-rs) from 0.55.0 to 0.55.1.
- [Release notes](https://github.com/substrait-io/substrait-rs/releases)
- [Changelog](https://github.com/substrait-io/substrait-rs/blob/main/CHANGELOG.md)
- [Commits](https://github.com/substrait-io/substrait-rs/compare/v0.55.0...v0.55.1)

---
updated-dependencies:
- dependency-name: substrait
  dependency-version: 0.55.1
  dependency-type: direct:production
  update-type: version-update:semver-patch
...

Signed-off-by: dependabot[bot] <[email protected]>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>

* feat: create helpers to set the max_temp_directory_size (#15919)

* feat: create helpers to set the max_temp_directory_size

Signed-off-by: Jérémie Drouet <[email protected]>

* refactor: use helper in cli

Signed-off-by: Jérémie Drouet <[email protected]>

* refactor: update error message

Signed-off-by: Jérémie Drouet <[email protected]>

* refactor: use setter in tests

Signed-off-by: Jérémie Drouet <[email protected]>

---------

Signed-off-by: Jérémie Drouet <[email protected]>
Co-authored-by: Andrew Lamb <[email protected]>

* Fix main CI (#15942)

* Improve sqllogictest error reporting (#15905)

* refactor filter pushdown apis (#15801)

* refactor filter pushdown apis

* remove commented out code

* fix tests

* fail to fix bug

* fix

* add/fix docs

* lint

* add some docstrings, some minimal cleaup

* review suggestions

* add more comments

* fix doc links

* fmt

* add comments

* make test deterministic

* add bench

* fix bench

* register bench

* fix bench

* cargo fmt

---------

Co-authored-by: berkaysynnada <[email protected]>
Co-authored-by: Berkay Şahin <[email protected]>

* fix: fold cast null to substrait typed null (#15854)

* fix: fold cast null to typed null

* test: unit test

* chore: clippy

* fix: only handle ScalarValue::Null instead of all null-ed value

* Add additional tests for filter pushdown apis (#15955)

* Add additional tests for filter pushdown apis

* rename the testing module

* move TestNode to util

* fmt

---------

Co-authored-by: berkaysynnada <[email protected]>

* Improve filter pushdown optimizer rule performance (#15959)

* Improve filter pushdown optimizer rule performance

* fmt

* fix lint

* feat: ORDER BY ALL (#15772)

* feat: ORDER BY ALL

* refactor: orderyby all

* refactor: order_by_to_sort_expr

* refactor: TODO comment

* fix query results for predicates referencing partition columns and data columns (#15935)

* fix query results for predicates referencing partition columns and data columns

* fmt

* add e2e test

* newline

* chore(deps): bump substrait from 0.55.0 to 0.55.1 (#15941)

Bumps [substrait](https://github.com/substrait-io/substrait-rs) from 0.55.0 to 0.55.1.
- [Release notes](https://github.com/substrait-io/substrait-rs/releases)
- [Changelog](https://github.com/substrait-io/substrait-rs/blob/main/CHANGELOG.md)
- [Commits](https://github.com/substrait-io/substrait-rs/compare/v0.55.0...v0.55.1)

---
updated-dependencies:
- dependency-name: substrait
  dependency-version: 0.55.1
  dependency-type: direct:production
  update-type: version-update:semver-patch
...

Signed-off-by: dependabot[bot] <[email protected]>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>

* feat: create helpers to set the max_temp_directory_size (#15919)

* feat: create helpers to set the max_temp_directory_size

Signed-off-by: Jérémie Drouet <[email protected]>

* refactor: use helper in cli

Signed-off-by: Jérémie Drouet <[email protected]>

* refactor: update error message

Signed-off-by: Jérémie Drouet <[email protected]>

* refactor: use setter in tests

Signed-off-by: Jérémie Drouet <[email protected]>

---------

Signed-off-by: Jérémie Drouet <[email protected]>
Co-authored-by: Andrew Lamb <[email protected]>

* Fix main CI (#15942)

* Improve sqllogictest error reporting (#15905)

* refactor filter pushdown apis (#15801)

* refactor filter pushdown apis

* remove commented out code

* fix tests

* fail to fix bug

* fix

* add/fix docs

* lint

* add some docstrings, some minimal cleaup

* review suggestions

* add more comments

* fix doc links

* fmt

* add comments

* make test deterministic

* add bench

* fix bench

* register bench

* fix bench

* cargo fmt

---------

Co-authored-by: berkaysynnada <[email protected]>
Co-authored-by: Berkay Şahin <[email protected]>

---------

Signed-off-by: dependabot[bot] <[email protected]>
Signed-off-by: Jérémie Drouet <[email protected]>
Co-authored-by: silezhou <[email protected]>
Co-authored-by: Adrian Garcia Badaracco <[email protected]>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
Co-authored-by: Jérémie Drouet <[email protected]>
Co-authored-by: Andrew Lamb <[email protected]>
Co-authored-by: xudong.w <[email protected]>
Co-authored-by: Gabriel <[email protected]>
Co-authored-by: berkaysynnada <[email protected]>
Co-authored-by: Berkay Şahin <[email protected]>

* Implement Parquet filter pushdown via new filter pushdown APIs (#15769)

* Implement Parquet filter pushdown via new filter pushdown APIs

* Update filter_pushdown.rs

---------

Co-authored-by: berkaysynnada <[email protected]>

* Reduce rehashing cost for primitive grouping by also reusing hash value (#15962)

* also save hash in hashtable in primitive single group by.

* address cr.

* chore(deps): bump chrono from 0.4.40 to 0.4.41 (#15956)

Bumps [chrono](https://github.com/chronotope/chrono) from 0.4.40 to 0.4.41.
- [Release notes](https://github.com/chronotope/chrono/releases)
- [Changelog](https://github.com/chronotope/chrono/blob/main/CHANGELOG.md)
- [Commits](https://github.com/chronotope/chrono/compare/v0.4.40...v0.4.41)

---
updated-dependencies:
- dependency-name: chrono
  dependency-version: 0.4.41
  dependency-type: direct:production
  update-type: version-update:semver-patch
...

Signed-off-by: dependabot[bot] <[email protected]>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>

* feat: support min/max for struct (#15667)

* feat: support min/max for struct

* groups aggregator

* update based on lamb's suggestion

* refactor: replace `unwrap_or` with `unwrap_or_else` for improved lazy… (#15841)

* refactor: replace `unwrap_or` with `unwrap_or_else` for improved lazy evaluation

* refactor: improve code readability by adjusting formatting and using `unwrap_or_else` for better lazy evaluation

* [FIX] added imports

* [FIX] formatting and restored original config logic

* config restored

* optimized the use of .clone()

* removed the use of clone

* cleanup the clone usecase

* add benchmark code for `Reuse rows in row cursor stream` (#15913)

* add benchmark for SortPreservingMergeExec

* add comments

* add comments

* Cover more test scenarios

* Update-docs_pr.yaml (#15966)

* Segfault in ByteGroupValueBuilder (#15968)

* test to demonstrate segfault in ByteGroupValueBuilder

* check for offset overflow

* clippy

* make can_expr_be_pushed_down_with_schemas public again (#15971)

* re-export can_expr_be_pushed_down_with_schemas to be public (#15974)

* Migrate Optimizer tests to insta, part5 (#15945)

* migrate `assert_optimized_plan_equal` in `extract_equijoin_predicate.rs` to use snapshot assertions

* format

* migrate `assert_optimized_plan_equal` in `single_distinct_to_groupby.rs` to use snapshot assertions

* remove all `assert_optimized_plan_eq_display_indent`

* remove unused test helper functions

* migrate `assert_optimized_plan_equal` in `scalar_subquery_to_join.rs` to use snapshot assertions

* remove unused test helper functions

* Show LogicalType name for `INFORMATION_SCHEMA` (#15965)

* show logica type instead of arrow type for parameteres table

* use debug fmt directly

* chore(deps): bump sha2 from 0.10.8 to 0.10.9 (#15970)

Bumps [sha2](https://github.com/RustCrypto/hashes) from 0.10.8 to 0.10.9.
- [Commits](https://github.com/RustCrypto/hashes/compare/sha2-v0.10.8...sha2-v0.10.9)

---
updated-dependencies:
- dependency-name: sha2
  dependency-version: 0.10.9
  dependency-type: direct:production
  update-type: version-update:semver-patch
...

Signed-off-by: dependabot[bot] <[email protected]>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>

* refactor: remove deprecated `ParquetExec` (#15973)

* refactor: remove deprecated ParquetExec

* fix doc

* fix: remove allow deprecated attribute

---------

Co-authored-by: Andrew Lamb <[email protected]>

* chore(deps): bump insta from 1.42.2 to 1.43.1 (#15988)

Bumps [insta](https://github.com/mitsuhiko/insta) from 1.42.2 to 1.43.1.
- [Release notes](https://github.com/mitsuhiko/insta/releases)
- [Changelog](https://github.com/mitsuhiko/insta/blob/master/CHANGELOG.md)
- [Commits](https://github.com/mitsuhiko/insta/compare/1.42.2...1.43.1)

---
updated-dependencies:
- dependency-name: insta
  dependency-version: 1.43.1
  dependency-type: direct:production
  update-type: version-update:semver-minor
...

Signed-off-by: dependabot[bot] <[email protected]>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>

* [datafusion-spark] Add Spark-compatible hex function (#15947)

* refactor: remove deprecated AvroExec (#15987)

* Substrait: Handle inner map fields in schema renaming (#15869)

* add tests

* fix tests

* fix

* fix

---------

Co-authored-by: Andrew Lamb <[email protected]>

* refactor: remove deprecated CsvExec (#15991)

* Migrate Optimizer tests to insta, part6 (#15984)

* migrate tests in `type_coercion.rs` to use snapshot assertions

* remove `assert_analyzed_plan_eq` and `assert_analyzed_plan_with_config_eq`

* remove unnecessary `pub`

* refactor: replace custom assertion functions with snapshot assertions in EliminateLimit tests

* format

* rename

* rename

* refactor: replace custom assertion function with macro for optimized plan equality in tests

* format macro

* chore(deps): bump nix from 0.29.0 to 0.30.1 (#16002)

Bumps [nix](https://github.com/nix-rust/nix) from 0.29.0 to 0.30.1.
- [Changelog](https://github.com/nix-rust/nix/blob/master/CHANGELOG.md)
- [Commits](https://github.com/nix-rust/nix/compare/v0.29.0...v0.30.1)

---
updated-dependencies:
- dependency-name: nix
  dependency-version: 0.30.1
  dependency-type: direct:production
  update-type: version-update:semver-minor
...

Signed-off-by: dependabot[bot] <[email protected]>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>

* Implement RightSemi join for SortMergeJoin (#15972)

* Enable repartitioning on MemTable. (#15409)

* test(15088): reproducer of missing sort parallelization

* feat(15088): repartitioning for MemorySourceConfig

* test(15088): update test outcome due to fix

* test: update sort spill test to not parallelize sorts (due to scan repartitioning)

* fix: out of bounds

* test: during fuzz testing, we are hitting limits of FDs open due to mem table repartitioned scan

* refactor: imrpove performance

* chore: change variable naming, and update documentation to make clear how the datasource repartitioning is configured and performed

* test: update test snapshots for updated config description

* refactor: update algo used for even splitting, to proper binpack

* refactor: change config name back to original for backwards compatibility

* fix: maintain ordering within partition, when a single partition

* chore: add more doc comments

* Migrate Optimizer tests to insta, part7 (#16010)

* generalize `assert_optimized_plan_eq_snapshot` interface

* fix clippy

* refactor: simplify assertion for optimized plan equality in tests

* migrate tests in `elimiate_cross_join.rs` to use snapshot assertions

* chore(deps): bump sysinfo from 0.34.2 to 0.35.1 (#16027)

Bumps [sysinfo](https://github.com/GuillaumeGomez/sysinfo) from 0.34.2 to 0.35.1.
- [Changelog](https://github.com/GuillaumeGomez/sysinfo/blob/master/CHANGELOG.md)
- [Commits](https://github.com/GuillaumeGomez/sysinfo/commits/v0.35.1)

---
updated-dependencies:
- dependency-name: sysinfo
  dependency-version: 0.35.1
  dependency-type: direct:production
  update-type: version-update:semver-minor
...

Signed-off-by: dependabot[bot] <[email protected]>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>

* Fix: `build_predicate_expression` method doesn't process `false` expr correctly (#15995)

* Fix: build_predicate_expression method doesn't process false correctly

* fix test

* refactor:  move should_enable_page_index from mod.rs to opener.rs (#16026)

* fix: add an "expr_planners" method to SessionState (#15119)

* add expr_planners to SessionState

* minor

* fix ci

* add test

* flatten imports

---------

Co-authored-by: Andrew Lamb <[email protected]>

* Updated extending operators documentation (#15612)

* Updated extending operators documentation

* commented out Rust code to pass doc test

---------

Co-authored-by: Andrew Lamb <[email protected]>

* feat(proto): udf decoding fallback (#15997)

* feat(proto): udf decoding fallback

* add test case for proto udf decode fallback

* chore: Replace MSRV link on main page with Github badge (#16020)

* Replace MSRV link on main page with Github badge

* Add note to upgrade guide for removal of `ParquetExec`, `AvroExec`, `CsvExec`, `JsonExec` (#16034)

* refactor: remove deprecated ArrowExec (#16006)

* refactor: remove deprecated MemoryExec (#16007)

* refactor: remove deprecated JsonExec (#16005)

Co-authored-by: Andrew Lamb <[email protected]>

* chore(deps): bump sqllogictest from 0.28.1 to 0.28.2 (#16037)

Bumps [sqllogictest](https://github.com/risinglightdb/sqllogictest-rs) from 0.28.1 to 0.28.2.
- [Release notes](https://github.com/risinglightdb/sqllogictest-rs/releases)
- [Changelog](https://github.com/risinglightdb/sqllogictest-rs/blob/main/CHANGELOG.md)
- [Commits](https://github.com/risinglightdb/sqllogictest-rs/compare/v0.28.1...v0.28.2)

---
updated-dependencies:
- dependency-name: sqllogictest
  dependency-version: 0.28.2
  dependency-type: direct:production
  update-type: version-update:semver-patch
...

Signed-off-by: dependabot[bot] <[email protected]>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>

* chores: Add lint rule to enforce string formatting style (#16024)

* Add lint rule to enforce string formatting style

* format

* extra

* Update datafusion/ffi/src/tests/async_provider.rs

Co-authored-by: kosiew <[email protected]>

* Update datafusion/functions/src/datetime/to_date.rs

Co-authored-by: kosiew <[email protected]>

---------

Co-authored-by: kosiew <[email protected]>

* Use human-readable byte sizes in EXPLAIN (#16043)

* Docs: Add example of creating a field in `return_field_from_args` (#16039)

* Docs: Add example of creating a field in `return_field_from_args`

* fmt

* Update datafusion/expr/src/udf.rs

Co-authored-by: Oleks V <[email protected]>

* fmt

---------

Co-authored-by: Oleks V <[email protected]>

* Support `MIN` and `MAX` for `DataType::List` (#16025)

* Fix comparisons between lists that contain nulls

* Add support for lists in min/max agg functions

* Add sqllogictests

* Support lists in window frame target type

* fix: overcounting of memory in first/last. (#15924)

When aggregating first/last list over a column of lists, the first/last
accumulators hold the necessary scalar value as is, which points to the
list in the original input buffer.

This results in two issues:

1) We prevent the deallocation of the input arrays which might be
significantly larger than the single value we want to hold.

2) During aggreagtion with groups, many accumulators receive slices of the
same input buffer, resulting in all held values pointing to this buffer.
Then, when calculating the size of all accumulators we count the buffer
multiple times, since each accumulator considers it to be part of its own
allocation.

* Improve docs for Exprs and scalar functions (#16036)

* Improve docs for Exprs and scalar functions

* fix links

* Add h2o window benchmark (#16003)

* h2o-window benchmark

* Review: clarify h2o-window is an extended benchmark

* fix: track coalescer's consumption (#16048)

Signed-off-by: Ruihang Xia <[email protected]>

* Fix Infer prepare statement type tests  (#15743)

* draft commit to rolledback changes on function naming and include prepare clause on the infer types tests

* include data types in plan when it is not included in the prepare statement

* fix: prepare statement error

* Update datafusion/sql/src/statement.rs

Co-authored-by: Andrew Lamb <[email protected]>

* remove infer types from prepare statement

the infer data type changes in statement will be introduced in a new PR

* fix to show correct output message

* remove white space

* Restore the original tests too

---------

Co-authored-by: Andrew Lamb <[email protected]>

* fix: Clarify that it is only the name of the field that is ignored (#16052)

* style: simplify some strings for readability (#15999)

* style: simplify some strings for readability

* fix: formatting in `datafusion/` directory

* refactor: replace long `format!` string

* refactor: replace `format!` with `assert_eq!`

---------

Co-authored-by: Andrew Lamb <[email protected]>

* support simple/cross lateral joins (#16015)

* support simple lateral joins

Signed-off-by: Alex Chi Z <[email protected]>

* fix explain test

Signed-off-by: Alex Chi Z <[email protected]>

* plan scalar agg correctly

Signed-off-by: Alex Chi Z <[email protected]>

* add uncorrelated query tests

Signed-off-by: Alex Chi Z <[email protected]>

* fix clippy + fmt

Signed-off-by: Alex Chi Z <[email protected]>

* make rule matching faster

Signed-off-by: Alex Chi Z <[email protected]>

* revert build_join visibility

Signed-off-by: Alex Chi Z <[email protected]>

* revert find plan outer column changes

Signed-off-by: Alex Chi Z <[email protected]>

* remove clone

* address comment

---------

Signed-off-by: Alex Chi Z <[email protected]>
Co-authored-by: Alex Chi Z <[email protected]>

* Make error msg for oom human readable (#16050)

* chore(deps): bump the arrow-parquet group with 7 updates (#16047)

* chore(deps): bump the arrow-parquet group with 7 updates

Bumps the arrow-parquet group with 7 updates:

| Package | From | To |
| --- | --- | --- |
| [arrow](https://github.com/apache/arrow-rs) | `55.0.0` | `55.1.0` |
| [arrow-buffer](https://github.com/apache/arrow-rs) | `55.0.0` | `55.1.0` |
| [arrow-flight](https://github.com/apache/arrow-rs) | `55.0.0` | `55.1.0` |
| [arrow-ipc](https://github.com/apache/arrow-rs) | `55.0.0` | `55.1.0` |
| [arrow-ord](https://github.com/apache/arrow-rs) | `55.0.0` | `55.1.0` |
| [arrow-schema](https://github.com/apache/arrow-rs) | `55.0.0` | `55.1.0` |
| [parquet](https://github.com/apache/arrow-rs) | `55.0.0` | `55.1.0` |


Updates `arrow` from 55.0.0 to 55.1.0
- [Release notes](https://github.com/apache/arrow-rs/releases)
- [Changelog](https://github.com/apache/arrow-rs/blob/main/CHANGELOG-old.md)
- [Commits](https://github.com/apache/arrow-rs/compare/55.0.0...55.1.0)

Updates `arrow-buffer` from 55.0.0 to 55.1.0
- [Release notes](https://github.com/apache/arrow-rs/releases)
- [Changelog](https://github.com/apache/arrow-rs/blob/main/CHANGELOG-old.md)
- [Commits](https://github.com/apache/arrow-rs/compare/55.0.0...55.1.0)

Updates `arrow-flight` from 55.0.0 to 55.1.0
- [Release notes](https://github.com/apache/arrow-rs/releases)
- [Changelog](https://github.com/apache/arrow-rs/blob/main/CHANGELOG-old.md)
- [Commits](https://github.com/apache/arrow-rs/compare/55.0.0...55.1.0)

Updates `arrow-ipc` from 55.0.0 to 55.1.0
- [Release notes](https://github.com/apache/arrow-rs/releases)
- [Changelog](https://github.com/apache/arrow-rs/blob/main/CHANGELOG-old.md)
- [Commits](https://github.com/apache/arrow-rs/compare/55.0.0...55.1.0)

Updates `arrow-ord` from 55.0.0 to 55.1.0
- [Release notes](https://github.com/apache/arrow-rs/releases)
- [Changelog](https://github.com/apache/arrow-rs/blob/main/CHANGELOG-old.md)
- [Commits](https://github.com/apache/arrow-rs/compare/55.0.0...55.1.0)

Updates `arrow-schema` from 55.0.0 to 55.1.0
- [Release notes](https://github.com/apache/arrow-rs/releases)
- [Changelog](https://github.com/apache/arrow-rs/blob/main/CHANGELOG-old.md)
- [Commits](https://github.com/apache/arrow-rs/compare/55.0.0...55.1.0)

Updates `parquet` from 55.0.0 to 55.1.0
- [Release notes](https://github.com/apache/arrow-rs/releases)
- [Changelog](https://github.com/apache/arrow-rs/blob/main/CHANGELOG-old.md)
- [Commits](https://github.com/apache/arrow-rs/compare/55.0.0...55.1.0)

---
updated-dependencies:
- dependency-name: arrow
  dependency-version: 55.1.0
  dependency-type: direct:production
  update-type: version-update:semver-minor
  dependency-group: arrow-parquet
- dependency-name: arrow-buffer
  dependency-version: 55.1.0
  dependency-type: direct:production
  update-type: version-update:semver-minor
  dependency-group: arrow-parquet
- dependency-name: arrow-flight
  dependency-version: 55.1.0
  dependency-type: direct:production
  update-type: version-update:semver-minor
  dependency-group: arrow-parquet
- dependency-name: arrow-ipc
  dependency-version: 55.1.0
  dependency-type: direct:production
  update-type: version-update:semver-minor
  dependency-group: arrow-parquet
- dependency-name: arrow-ord
  dependency-version: 55.1.0
  dependency-type: direct:production
  update-type: version-update:semver-minor
  dependency-group: arrow-parquet
- dependency-name: arrow-schema
  dependency-version: 55.1.0
  dependency-type: direct:production
  update-type: version-update:semver-minor
  dependency-group: arrow-parquet
- dependency-name: parquet
  dependency-version: 55.1.0
  dependency-type: direct:production
  update-type: version-update:semver-minor
  dependency-group: arrow-parquet
...

Signed-off-by: dependabot[bot] <[email protected]>

* Update sqllogictest results

---------

Signed-off-by: dependabot[bot] <[email protected]>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
Co-authored-by: Andrew Lamb <[email protected]>

* chore(deps): bump petgraph from 0.7.1 to 0.8.1 (#15669)

Bumps [petgraph](https://github.com/petgraph/petgraph) from 0.7.1 to 0.8.1.
- [Release notes](https://github.com/petgraph/petgraph/releases)
- [Changelog](https://github.com/petgraph/petgraph/blob/master/CHANGELOG.md)
- [Commits](https://github.com/petgraph/petgraph/compare/[email protected]@v0.8.1)

---
updated-dependencies:
- dependency-name: petgraph
  dependency-version: 0.8.1
  dependency-type: direct:production
  update-type: version-update:semver-minor
...

Signed-off-by: dependabot[bot] <[email protected]>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>

* [datafusion-spark] Add Spark-compatible `char` expression (#15994)

* Add Spark-compatible char expression

* Add slt test

* [Docs]: Added SQL example for all window functions (#16074)

* update window function

* pretier fix

* Update window_functions.md

* chore(deps): bump substrait from 0.55.1 to 0.56.0 (#16091)

Bumps [substrait](https://github.com/substrait-io/substrait-rs) from 0.55.1 to 0.56.0.
- [Release notes](https://github.com/substrait-io/substrait-rs/releases)
- [Changelog](https://github.com/substrait-io/substrait-rs/blob/main/CHANGELOG.md)
- [Commits](https://github.com/substrait-io/substrait-rs/compare/v0.55.1...v0.56.0)

---
updated-dependencies:
- dependency-name: substrait
  dependency-version: 0.56.0
  dependency-type: direct:production
  update-type: version-update:semver-minor
...

Signed-off-by: dependabot[bot] <[email protected]>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>

* Add test for collect_statistics (#16098)

* Add window function examples in code (#16102)

* Refactor substrait producer into multiple files (#16089)

* Fix temp dir leak in tests (#16094)

`TempDir::into_path` "leaks" the temp dir. This updates the `tempfile`
crate to a version where this method is deprecated and fixes all usages.

* Label Spark functions PRs with spark label (#16095)

* Rename Labeler workflow file name

Sync workflow name and its file name.

* Fix typo in Labeler config

* Label Spark functions PRs with `spark` label

* feat: add slt tests for imdb data (#16067)

* fix: stack overflow for substrait functions with large argument lists that translate to DataFusion binary operators   (#16031)

* Add substrait consumer test causing a stack overflow

* Mitigate stack overflow for substrait binary op with large arg list

When transforming a substrait function call to DataFusion logical plan,
if the substrait function maps to a DataFusion binary operator, but has
more than 2 arguments, it is mapped to a tree of BinaryExpr. This
BinaryExpr tree is not balanced, and its depth is the number of
arguments:

       Op
      /  \
    arg1  Op
         /  \
       arg2  ...
             /  \
           argN  Op

Since many functions manipulating the logical plan are recursive, it
means that N arguments result in an O(N) recursion, leading to stack
overflows for large N (1000 for example).

Transforming these function calls into a balanced tree mitigates the
issue:

             .__ Op __.
            /          \
          Op            Op
         /  \          /  \
       ...  ...      ...  ...
      /  \  /  \    /  \  /  \
    arg1        ...          argN

The recursion depth is now O(log2(N)), meaning that 1000 arguments
results in a depth of ~10, and it would take 2^1000 arguments to reach a
depth of 1000, which is a vastly unreasonable amount of data.

Therefore, it's not possible to use this flaw anymore to trigger stack
overflows in processes running DataFusion.

* arg_list_to_binary_op_tree: avoid cloning Expr

* cargo fmt

* from_scalar_function: improve error handling

* Move test_binary_op_large_argument_list test to scalar_function module

* arg_list_to_binary_op_tree: add more unit tests

Courtesy of @gabotechs

* substrait consumer scalar_function tests: more explicit function name

---------

Co-authored-by: Andrew Lamb <[email protected]>

* chore: Remove SMJ experimental status (#16072)

* chore(CI) Update workspace / CI to Rust 1.87 (#16068)

Co-authored-by: Andrew Lamb <[email protected]>

* minor: Add benchmark query and corresponding documentation for Average Duration (#16105)

* ADD query and documentation

* Prettier

---------

Co-authored-by: Andrew Lamb <[email protected]>

* feat: metadata handling for aggregates and window functions (#15911)

* Move expr_schema to use return_field instead of return_type

* More work on moving to Field from DataType for aggregates

* Update field output name for aggregates

* Improve unit test for aggregate udf with metadata

* Move window functions over to use Field instead of DataType

* Correct nullability flag

* Add import after rebase

* Add unit test for using udaf as window function with metadata processing

* Update documentation for migration guide

* Update naming from data type to field to match the actual parameters passed

* Avoid some allocations

* Update docs to use aggregate example

---------

Co-authored-by: Andrew Lamb <[email protected]>

* doc: fix indent format explain (#16085)

* doc: fix indent format explain

* update

* fix: coerce int96 resolution inside of list, struct, and map types (#16058)

* Add test generated from schema in Comet.

* Checkpoint DFS.

* Checkpoint with working transformation.

* fmt, clippy fixes.

* Remove maximum stack depth.

* More testing.

* Improve tests.

* Improve docs.

* Use a smaller HashSet instead of HashMap with every field in it. More docs.

* Use a smaller HashSet instead of HashMap with every field in it. More docs.

* More docs.

* More docs.

* Fix typo.

* Refactor match with nested if lets to make it more readable.

* Address some PR feedback.

* Rename variables in struct processing to address PR feedback. Do List next.

* Rename variables in list processing to address PR feedback.

* Update docs.

* Simplify list parquet path generation.

* Map support.

* Remove old TODO.

* Reduce redundant docs be referring to docs above.

* Reduce redundant docs be referring to docs above.

* Add parquet file generated from CometFuzzTestSuite ParquetGenerator (similar to schema in file_format tests) to exercise end-to-end support.

* Fix clippy.

* Update documentation for `datafusion.execution.collect_statistics` (#16100)

* Update documentation for `datafusion.execution.collect_statistics` setting

* Update test

* Update datafusion/common/src/config.rs

Co-authored-by: Leonardo Yvens <[email protected]>

* update docs

* Update doc

---------

Co-authored-by: Leonardo Yvens <[email protected]>

* fix: Add coercion rules for Float16 types (#15816)

* handle coercion for Float16 types

* Add some basic slt tests

---------

Co-authored-by: Andrew Lamb <[email protected]>

* Use qualified names on DELETE selections (#16033)

Co-authored-by: Andrew Lamb <[email protected]>

* chore(deps): bump testcontainers from 0.23.3 to 0.24.0 (#15989)

* chore(deps): bump testcontainers from 0.23.3 to 0.24.0

Bumps [testcontainers](https://github.com/testcontainers/testcontainers-rs) from 0.23.3 to 0.24.0.
- [Release notes](https://github.com/testcontainers/testcontainers-rs/releases)
- [Changelog](https://github.com/testcontainers/testcontainers-rs/blob/main/CHANGELOG.md)
- [Commits](https://github.com/testcontainers/testcontainers-rs/compare/0.23.3...0.24.0)

---
updated-dependencies:
- dependency-name: testcontainers
  dependency-version: 0.24.0
  dependency-type: direct:production
  update-type: version-update:semver-minor
...

Signed-off-by: dependabot[bot] <[email protected]>

* Update test_containers_modules too

---------

Signed-off-by: dependabot[bot] <[email protected]>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
Co-authored-by: Andrew Lamb <[email protected]>

* feat: make error handling in indent explain consistent with that in tree (#16097)

* feat: make error handling in indent consistent with that in tree

* update test

* return all plans instead of throwing err

* update test

* Clean up ExternalSorter and use upstream converter (#16109)

* Support `GroupsAccumulator` for Avg duration (#15748)

* Support GroupsAccumulator for avg duration

* update test

---------

Co-authored-by: Andrew Lamb <[email protected]>

* Test Duration in `fuzz` tests (#16111)

* Move PruningStatistics into datafusion::common (#16069)

* Move PruningStatistics into datafusion::common

* fix doc

* remove new code

* fmt

* Revert use file schema in parquet pruning (#16086)

* wip

* comment

* Update datafusion/core/src/datasource/physical_plan/parquet.rs

* remove prints

* better test

* fmt

* Make `SessionContext::register_parquet` obey `collect_statistics` config (#16080)

* fix

* add a test

* fmt

* add to upgrade guide

* fix tests

* fix test

* fix test

* fix ci

* Fix example in upgrade guide (#29)

---------

Co-authored-by: Andrew Lamb <[email protected]>

* fix: describe escaped quoted identifiers (#16082)

* feat: escape quote wrap identifiers in describe

rm: dev files

fmt: final formatting

sed: s/<comment>//

* fix: use ident instead of col + format

* Minor: Add `ScalarFunctionArgs::return_type` method (#16113)

* feat: coerce from fixed size binary to binary view (#16110)

* Improve the DML / DDL Documentation (#16115)

* Update documentation about DDL and DML

* Improve the DML Documentation

* Apply suggestions from code review

Co-authored-by: Oleks V <[email protected]>

* Fix docs

* Fix docs

---------

Co-authored-by: Oleks V <[email protected]>

* Fix `contains` function expression (#16046)

* Optimize performance of `string::ascii` function (#16087)

* Optimize performance of string::ascii function

d

* Add benchmark with with NULL_DENSITY=0

d

---------

Co-authored-by: Tai Le Manh <[email protected]>
Co-authored-by: Andrew Lamb <[email protected]>

* chore: Use materialized data for filter pushdown tests (#16123)

* chore: Use pre created data for filter pushdown tests

* chore: Use pre created data for filter pushdown tests

* chore: Upgrade rand crate and some other minor crates (#16062)

* chore: Upgrade `rand` crate and some other minor crates

---------

Co-authored-by: Andrew Lamb <[email protected]>

* Include data types in logical plans of inferred prepare statements (#16019)

* draft commit to rolledback changes on function naming and include prepare clause on the infer types tests

* include data types in plan when it is not included in the prepare statement

* fix: prepare statement error

* Update datafusion/sql/src/statement.rs

Co-authored-by: Andrew Lamb <[email protected]>

* remove infer types from prepare statement

the infer data type changes in statement will be introduced in a new PR

* fix to show correct output message

* include data types on logical plans of prepare statements without explicit type declaration

* fix using clippy sugestions

* explicitly get the data types using the placeholder id to avoid sorting

* Restore the original tests too

* update set data type routine to be more rust idiomatic

Co-authored-by: Tommy shu <[email protected]>

* update set datatype routine

* fix formatting in sql_integration

---------

Co-authored-by: Andrew Lamb <[email protected]>
Co-authored-by: Tommy shu <[email protected]>

* docs: Fix typos and minor grammatical issues in Architecture docs (#16119)

* minor fixes to arch docs


Co-authored-by: Oleks V <[email protected]>

---------

Co-authored-by: Oleks V <[email protected]>

* add top-memory-consumers option in cli (#16081)

add snapshot tests for memory exhaustion

* fix ci extended test (#16144)

* Fix: handle column name collisions when combining UNION logical inputs & nested Column expressions in maybe_fix_physical_column_name (#16064)

* Fix union schema name coercion

* Address renaming for columns that are not in the top level as well

* Add unit test

* Format

* Use insta tests properly

* Address review - comment + minor simplification change

---------

Co-authored-by: Berkay Şahin <[email protected]>

* adding support for Min/Max over LargeList and FixedSizeList (#16071)

* initial Iteration

* add Sql Logic tests

* tweak comments

* unify data, structure tests

* Deleted by mistake

* Move prepare/parameter handling tests into `params.rs` (#16141)

* Move prepare/parameter handling tests into `params.rs`

* Resolve conflicts

* Add `StateFieldsArgs::return_field` (#16112)

* Support filtering specific sqllogictests identified by line number (#16029)

* Support filtering specific sqllogictests identified by line number

* Add license header

* Try parsing in different dialects

* Add test filtering example to README.md

* Improve Filter doc comment

* Factor out statement_is_skippable into its own function

* Add example about how filters work in the doc comments

* Enrich GroupedHashAggregateStream name to ease debugging Resources exhausted errors (#16152)

* Enrich GroupedHashAggregateStream name to ease debugging Resources exhausted errors

* Use human_display

* clippy

* chore(deps): bump uuid from 1.16.0 to 1.17.0 (#16162)

Bumps [uuid](https://github.com/uuid-rs/uuid) from 1.16.0 to 1.17.0.
- [Release notes](https://github.com/uuid-rs/uuid/releases)
- [Commits](https://github.com/uuid-rs/uuid/compare/v1.16.0...v1.17.0)

---
updated-dependencies:
- dependency-name: uuid
  dependency-version: 1.17.0
  dependency-type: direct:production
  update-type: version-update:semver-minor
...

Signed-off-by: dependabot[bot] <[email protected]>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>

* Minor: Fix links in substrait readme (#16156)

* Remove Filter::having field (#16154)

Both WHERE clause and HAVING clause translate to a Filter plan node.
They differ in how the references and aggregates are handled.
HAVING goes after aggregation and may reference aggregate expressions
and therefore HAVING's filter will be placed after Aggregation plan
node.

Once a plan has been built, however, there is no special additional
semantics to filters created from HAVING. Remove the unnecessary field.

For reference, the field was added along with usage in
a50aeefcbfc84d491495887d57fa8ebc0db57ff2 commit and the usage was later
removed in eb62e2871e49c402ec7b0d25658faa6dc5219969 commit.

* Clarify docs and names in parquet predicate pushdown tests (#16155)

* Clarify docs and names in parquet predicate pushdown tests

* Update datafusion/datasource/src/file_scan_config.rs

Co-authored-by: Adrian Garcia Badaracco <[email protected]>

* clippy

---------

Co-authored-by: Adrian Garcia Badaracco <[email protected]>

* Minor: Fix name() for FilterPushdown physical optimizer rule (#16175)

* Fix name() for FilterPushdown physical optimizer rule

Typo that wasn't caught during review...

* fix

* migrate tests in `pool.rs` to use insta (#16145)

fix according to review

fix to_string error

fix test by stripping backtrace

* refactor(optimizer): add `.with_schema` for defining test tables (#16138)

Added `tables: HashMap<String, Arc<dyn TableSource>>` and `MyContextProvider::with_schema` method for dynamically defining tables for optimizer integration tests.

* [Minor] Speedup TPC-H benchmark run with memtable option (#16159)

* Speedup tpch run with memtable

* Clippy

* Clippy

* Fast path for joins with distinct values in build side (#16153)

* Specialize unique join

* handle splitting

* rename a bit

* fix

* fix

* fix

* fix

* Fix the test, add explanation

* Simplify

* Update datafusion/physical-plan/src/joins/join_hash_map.rs

Co-authored-by: Christian <[email protected]>

* Update datafusion/physical-plan/src/joins/join_hash_map.rs

Co-authored-by: Christian <[email protected]>

* Simplify

* Simplify

* Simplify

---------

Co-authored-by: Christian <[email protected]>

* chore: Reduce repetition in the parameter type inference tests (#16079)

* added test

* added parameterTest

* cargo fmt

* Update sql_integration.rs

* allow needless_lifetimes

* remove needless lifetime

* update some tests

* move to params.rs

* feat: array_length for fixed size list (#16167)

* feat: array_length for fixed size list

* remove list view

* fix: remove trailing whitespace in `Display` for `LogicalPlan::Projection` (#16164)

* chore(deps): bump tokio from 1.45.0 to 1.45.1 (#16190)

Bumps [tokio](https://github.com/tokio-rs/tokio) from 1.45.0 to 1.45.1.
- [Release notes](https://github.com/tokio-rs/tokio/releases)
- [Commits](https://github.com/tokio-rs/tokio/compare/tokio-1.45.0...tokio-1.45.1)

---
updated-dependencies:
- dependency-name: tokio
  dependency-version: 1.45.1
  dependency-type: direct:production
  update-type: version-update:semver-patch
...

Signed-off-by: dependabot[bot] <[email protected]>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>

* Improve `unproject_sort_expr` to handle arbitrary expressions (#16127)

* Add failing test to demonstrate problem

* Improve `unproject_sort_expr` to handle arbitrary expressions (#83)

* Remove redundant return

* chore(deps): bump rustyline from 15.0.0 to 16.0.0 (#16194)

Bumps [rustyline](https://github.com/kkawakam/rustyline) from 15.0.0 to 16.0.0.
- [Release notes](https://github.com/kkawakam/rustyline/releases)
- [Changelog](https://github.com/kkawakam/rustyline/blob/master/History.md)
- [Commits](https://github.com/kkawakam/rustyline/compare/v15.0.0...v16.0.0)

---
updated-dependencies:
- dependency-name: rustyline
  dependency-version: 16.0.0
  dependency-type: direct:production
  update-type: version-update:semver-major
...

Signed-off-by: dependabot[bot] <[email protected]>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>

* feat: ADD sha2 spark function (#16168)

ADD sha2 spark function

* Add macro for creating DataFrame (#16090) (#16104)

* Add macro for creating DataFr…
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

core Core DataFusion crate datasource Changes to the datasource crate documentation Improvements or additions to documentation optimizer Optimizer rules proto Related to proto crate sqllogictest SQL Logic Tests (.slt)

Projects

None yet

Development

Successfully merging this pull request may close these issues.

5 participants