<?xml version="1.0" encoding="utf-8"?>
<feed xmlns="http://www.w3.org/2005/Atom"><title>Apache DataFusion Blog</title><link href="https://datafusion.apache.org/blog/" rel="alternate"/><link href="https://datafusion.apache.org/blog/feeds/all-en.atom.xml" rel="self"/><id>https://datafusion.apache.org/blog/</id><updated>2026-05-07T00:00:00+00:00</updated><entry><title>Apache DataFusion Comet 0.16.0 Release</title><link href="https://datafusion.apache.org/blog/2026/05/07/datafusion-comet-0.16.0" rel="alternate"/><published>2026-05-07T00:00:00+00:00</published><updated>2026-05-07T00:00:00+00:00</updated><author><name>pmc</name></author><id>tag:datafusion.apache.org,2026-05-07:/blog/2026/05/07/datafusion-comet-0.16.0</id><summary type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;p&gt;The Apache DataFusion PMC is pleased to announce version 0.16.0 of the &lt;a href="https://datafusion.apache.org/comet/"&gt;Comet&lt;/a&gt; subproject.&lt;/p&gt;
&lt;p&gt;Comet is an accelerator for Apache Spark that translates Spark physical plans to DataFusion physical plans for
improved performance and efficiency without requiring any code changes.&lt;/p&gt;
&lt;p&gt;This release covers approximately three weeks of development …&lt;/p&gt;</summary><content type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;p&gt;The Apache DataFusion PMC is pleased to announce version 0.16.0 of the &lt;a href="https://datafusion.apache.org/comet/"&gt;Comet&lt;/a&gt; subproject.&lt;/p&gt;
&lt;p&gt;Comet is an accelerator for Apache Spark that translates Spark physical plans to DataFusion physical plans for
improved performance and efficiency without requiring any code changes.&lt;/p&gt;
&lt;p&gt;This release covers approximately three weeks of development work and is the result of merging 115 PRs from 17
contributors. See the &lt;a href="https://github.com/apache/datafusion-comet/blob/main/dev/changelog/0.16.0.md"&gt;change log&lt;/a&gt; for more information.&lt;/p&gt;
&lt;h2 id="expanded-spark-4-support"&gt;Expanded Spark 4 Support&lt;a class="headerlink" href="#expanded-spark-4-support" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;Spark 4 is a major theme of this release. Comet now ships first-class support for both Spark 4.0.2 and
Spark 4.1.1, with dedicated Maven profiles, shim sources, and CI matrices for each.&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;strong&gt;Spark 4.1.1&lt;/strong&gt;: New &lt;code&gt;spark-4.1&lt;/code&gt; Maven profile and shim sources, with Comet's PR test matrix and Spark SQL
  test suites enabled against Spark 4.1.1. The default Maven profile has been updated to Spark 4.1 / Scala 2.13
  to reflect that this is now the primary development target.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Shared 4.x shims&lt;/strong&gt;: Identical pieces of the Spark 4.0 and 4.1 shims have been consolidated into a shared
  &lt;code&gt;spark-4.x&lt;/code&gt; source tree, reducing duplication as more 4.x minor versions land.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Spark 4.0 / JDK 21&lt;/strong&gt;: Added a Spark 4.0 / JDK 21 CI profile to validate Comet on the JDK most users are
  expected to deploy with Spark 4.&lt;/li&gt;
&lt;/ul&gt;
&lt;h3 id="adapting-to-spark-4-behavior-changes"&gt;Adapting to Spark 4 Behavior Changes&lt;a class="headerlink" href="#adapting-to-spark-4-behavior-changes" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;Spark 4 introduced a number of type, planner, and on-disk format changes relative to Spark 3.x. Several
correctness fixes this release bring Comet's behavior in line with these changes:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;strong&gt;&lt;code&gt;Variant&lt;/code&gt; type (new in Spark 4.0)&lt;/strong&gt;: Spark 4.0 added a new &lt;code&gt;Variant&lt;/code&gt; data type for semi-structured
  data. Comet does not yet read the shredded Variant on-disk format natively, and delegates these scans
  to Spark.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;String collation (new in Spark 4.0)&lt;/strong&gt;: Spark 4.0 added collation support for &lt;code&gt;StringType&lt;/code&gt;. Comet's
  native operators do not yet implement non-default collations, so hash join and sort-merge join reject
  collated string join keys, and shuffle, sort, and aggregate fall back to Spark when keys carry a
  non-default collation.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Wider &lt;code&gt;TimestampNTZType&lt;/code&gt; usage&lt;/strong&gt;: Spark 4 uses &lt;code&gt;TimestampNTZType&lt;/code&gt; (timestamp without time zone) in
  more places than 3.x — for example, in expression return types and as the inferred type for some
  literal forms. Comet adds support this cycle for cast to and from &lt;code&gt;timestamp_ntz&lt;/code&gt;, cast from string to
  &lt;code&gt;timestamp_ntz&lt;/code&gt;, and &lt;code&gt;unix_timestamp&lt;/code&gt; over &lt;code&gt;TimestampNTZType&lt;/code&gt; inputs.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;&lt;code&gt;to_json&lt;/code&gt; and &lt;code&gt;array_compact&lt;/code&gt; (Spark 4.0)&lt;/strong&gt;: Spark 4.0 adjusted output formatting and return-type
  metadata for these expressions; Comet now matches the new behavior.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;BloomFilter V2 (new in Spark 4.1)&lt;/strong&gt;: Spark 4.1 introduced a new BloomFilter binary format with
  different bit-scattering. Comet now reads this format so that runtime filters produced by Spark 4.1
  remain usable in native execution.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Spark 4.1.1 analyzer refinements&lt;/strong&gt;: Spark 4.1.1 changed how struct projections handle the case where
  every requested child field is missing from the Parquet file, and how &lt;code&gt;allowDecimalPrecisionLoss&lt;/code&gt;
  flows through the &lt;code&gt;DecimalPrecision&lt;/code&gt; rule. Comet now preserves parent-struct nullness in the first
  case and the stored &lt;code&gt;allowDecimalPrecisionLoss&lt;/code&gt; flag in the second.&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;Most of these behavior differences were caught because &lt;strong&gt;Comet runs the full Apache Spark SQL test suite
against each supported Spark version&lt;/strong&gt; — 3.4.3, 3.5.8, 4.0.2, and 4.1.1 — as part of CI. Running Spark's
own correctness tests through Comet's native execution path is what surfaces semantic shifts like
&lt;code&gt;TimestampNTZType&lt;/code&gt; propagation, ANSI-driven cast and overflow changes, BloomFilter V2 encoding, and the
4.1.1 analyzer rule changes, often before they show up in user workloads. As more Spark 4.x minor releases
land, this same harness is what gives us confidence that Comet keeps up.&lt;/p&gt;
&lt;h3 id="ansi-sql-semantics"&gt;ANSI SQL Semantics&lt;a class="headerlink" href="#ansi-sql-semantics" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;Spark 4 enables ANSI SQL semantics by default. ANSI mode changes how arithmetic overflow, invalid casts,
division by zero, and similar error conditions are handled, and Spark itself now treats this as the standard
configuration rather than an opt-in.&lt;/p&gt;
&lt;p&gt;This is a critical area for any Spark accelerator: an engine that falls back to vanilla Spark whenever ANSI is
enabled effectively does not run on Spark 4 by default. &lt;strong&gt;Comet implements ANSI semantics for the expressions
it supports natively&lt;/strong&gt;, including arithmetic overflow checks, ANSI cast behavior, and &lt;code&gt;try_*&lt;/code&gt; variants.
Queries running with &lt;code&gt;spark.sql.ansi.enabled=true&lt;/code&gt; continue to be accelerated rather than falling back.&lt;/p&gt;
&lt;p&gt;See the &lt;a href="https://datafusion.apache.org/comet/user-guide/latest/compatibility/index.html"&gt;Comet Compatibility Guide&lt;/a&gt; for details on which expressions have full ANSI coverage.&lt;/p&gt;
&lt;h2 id="expanded-adaptive-execution-support"&gt;Expanded Adaptive Execution Support&lt;a class="headerlink" href="#expanded-adaptive-execution-support" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;Modern Spark plans are adaptive: AQE re-plans stages at runtime, Dynamic Partition Pruning (DPP) prunes
fact-table partitions based on broadcast dimension filters, and &lt;code&gt;ReuseExchange&lt;/code&gt; and &lt;code&gt;ReuseSubquery&lt;/code&gt; ensure
that a broadcast or subquery referenced in multiple places executes only once. For star-schema workloads,
these mechanisms are not optional. They are often the difference between a query that reads 1% of the fact
table and one that reads all of it.&lt;/p&gt;
&lt;p&gt;Prior to 0.16.0, Comet's native scans only partially participated in this machinery. &lt;code&gt;CometNativeScanExec&lt;/code&gt;
(the DataFusion-based native Parquet scan) fell back to Spark entirely whenever a DPP filter was present.
&lt;code&gt;CometIcebergNativeScanExec&lt;/code&gt; supported non-AQE DPP as of 0.15.0
(&lt;a href="https://github.com/apache/datafusion-comet/pull/3349"&gt;#3349&lt;/a&gt;), but without broadcast exchange reuse, so
the DPP subquery re-executed the dimension broadcast.&lt;/p&gt;
&lt;p&gt;Comet 0.16.0 closes both gaps and aligns the native Parquet and native Iceberg scans on a single DPP and
subquery-resolution path:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;strong&gt;Non-AQE DPP for native Parquet, with broadcast exchange reuse&lt;/strong&gt;
  (&lt;a href="https://github.com/apache/datafusion-comet/pull/4011"&gt;#4011&lt;/a&gt;,
  &lt;a href="https://github.com/apache/datafusion-comet/pull/4037"&gt;#4037&lt;/a&gt;): A new &lt;code&gt;CometSubqueryBroadcastExec&lt;/code&gt; replaces
  Spark's &lt;code&gt;SubqueryBroadcastExec&lt;/code&gt; in DPP expressions and wraps a &lt;code&gt;CometBroadcastExchangeExec&lt;/code&gt;, so
  &lt;code&gt;ReuseExchangeAndSubquery&lt;/code&gt; matches the join side and the DPP subquery and broadcasts the dimension exactly
  once.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;AQE DPP for native Parquet&lt;/strong&gt; (&lt;a href="https://github.com/apache/datafusion-comet/pull/4112"&gt;#4112&lt;/a&gt;): Under AQE,
  Spark's &lt;code&gt;PlanAdaptiveDynamicPruningFilters&lt;/code&gt; cannot match Comet's broadcast hash join and would otherwise
  rewrite DPP to &lt;code&gt;TrueLiteral&lt;/code&gt;, disabling pruning. 0.16.0 intercepts &lt;code&gt;SubqueryAdaptiveBroadcastExec&lt;/code&gt; before
  Spark's rule runs, and applies Spark's decision tree in a Comet-aware rule that searches both the current
  stage and the root plan for a reusable broadcast. DPP subqueries are registered in AQE's shared
  &lt;code&gt;subqueryCache&lt;/code&gt; so cross-plan DPP (for example, a main query and a scalar subquery referencing the same
  dimension) deduplicates correctly. A narrower tagging-based fallback covers Spark 3.4, which lacks the
  &lt;code&gt;injectQueryStageOptimizerRule&lt;/code&gt; extension point.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;AQE DPP broadcast reuse for native Iceberg&lt;/strong&gt;
  (&lt;a href="https://github.com/apache/datafusion-comet/pull/4215"&gt;#4215&lt;/a&gt;): Lifts &lt;code&gt;runtimeFilters&lt;/code&gt; to a top-level
  constructor field on &lt;code&gt;CometIcebergNativeScanExec&lt;/code&gt; (mirroring &lt;code&gt;BatchScanExec&lt;/code&gt;), so Spark's
  expression-rewrite passes can see and convert the DPP subquery. The same &lt;code&gt;CometSubqueryBroadcastExec&lt;/code&gt;
  machinery from the Parquet path now handles the Iceberg case.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Scalar subquery pushdown and AQE subquery reuse&lt;/strong&gt;
  (&lt;a href="https://github.com/apache/datafusion-comet/pull/4053"&gt;#4053&lt;/a&gt;,
  &lt;a href="https://issues.apache.org/jira/browse/SPARK-43402"&gt;SPARK-43402&lt;/a&gt;): &lt;code&gt;CometNativeScanExec&lt;/code&gt; now participates
  in scalar subquery pushdown into Parquet data filters, and in AQE-time subquery deduplication via a new
  &lt;code&gt;CometReuseSubquery&lt;/code&gt; rule that re-applies Spark's &lt;code&gt;ReuseAdaptiveSubquery&lt;/code&gt; algorithm after Comet's node
  replacements.&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;&lt;strong&gt;Measured impact on TPC-DS:&lt;/strong&gt; 78 queries previously fell back to Spark whenever
DPP filters were planned, running 30–50% natively. With native DPP in 0.16.0, the same queries run 80–97%
natively. Representative examples:&lt;/p&gt;
&lt;table class="table"&gt;
&lt;thead&gt;
&lt;tr&gt;
&lt;th&gt;Query&lt;/th&gt;
&lt;th&gt;Before&lt;/th&gt;
&lt;th&gt;After&lt;/th&gt;
&lt;/tr&gt;
&lt;/thead&gt;
&lt;tbody&gt;
&lt;tr&gt;
&lt;td&gt;q1&lt;/td&gt;
&lt;td&gt;36%&lt;/td&gt;
&lt;td&gt;96%&lt;/td&gt;
&lt;/tr&gt;
&lt;tr&gt;
&lt;td&gt;q4&lt;/td&gt;
&lt;td&gt;31%&lt;/td&gt;
&lt;td&gt;95%&lt;/td&gt;
&lt;/tr&gt;
&lt;tr&gt;
&lt;td&gt;q31&lt;/td&gt;
&lt;td&gt;31%&lt;/td&gt;
&lt;td&gt;95%&lt;/td&gt;
&lt;/tr&gt;
&lt;tr&gt;
&lt;td&gt;q74&lt;/td&gt;
&lt;td&gt;32%&lt;/td&gt;
&lt;td&gt;95%&lt;/td&gt;
&lt;/tr&gt;
&lt;tr&gt;
&lt;td&gt;q92&lt;/td&gt;
&lt;td&gt;36%&lt;/td&gt;
&lt;td&gt;95%&lt;/td&gt;
&lt;/tr&gt;
&lt;/tbody&gt;
&lt;/table&gt;
&lt;p&gt;Several Spark SQL DPP tests that Comet previously skipped are re-enabled to guarantee Spark compatibility
and prevent regressions.&lt;/p&gt;
&lt;h2 id="improved-tpc-ds-benchmark-results"&gt;Improved TPC-DS Benchmark Results&lt;a class="headerlink" href="#improved-tpc-ds-benchmark-results" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;TPC-DS performance increased significantly compared to the 0.15.0 release and Comet is now very close to 2x faster than Spark.&lt;/p&gt;
&lt;p&gt;&lt;img alt="TPC-DS Overall Performance" class="img-fluid" src="/blog/images/comet-0.16.0/tpcds_speedup.png" width="100%"/&gt;&lt;/p&gt;
&lt;p&gt;See the &lt;a href="https://datafusion.apache.org/comet/contributor-guide/benchmarking.html"&gt;Comet Benchmarking Guide&lt;/a&gt; for more details about these benchmark results.&lt;/p&gt;
&lt;h2 id="other-key-features"&gt;Other Key Features&lt;a class="headerlink" href="#other-key-features" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;h3 id="hash-join-improvements"&gt;Hash Join Improvements&lt;a class="headerlink" href="#hash-join-improvements" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;ul&gt;
&lt;li&gt;&lt;strong&gt;&lt;code&gt;BuildRight&lt;/code&gt; + &lt;code&gt;LeftAnti&lt;/code&gt;&lt;/strong&gt; (&lt;a href="https://github.com/apache/datafusion-comet/pull/4073"&gt;#4073&lt;/a&gt;): Regular hash
  joins now support the &lt;code&gt;BuildRight&lt;/code&gt; + &lt;code&gt;LeftAnti&lt;/code&gt; combination, eliminating a common fallback path. Tests
  previously gated on &lt;code&gt;InjectRuntimeFilterSuite&lt;/code&gt; issues have been re-enabled.&lt;/li&gt;
&lt;/ul&gt;
&lt;h3 id="aggregation"&gt;Aggregation&lt;a class="headerlink" href="#aggregation" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;ul&gt;
&lt;li&gt;&lt;strong&gt;&lt;code&gt;PartialMerge&lt;/code&gt; aggregation mode&lt;/strong&gt; (&lt;a href="https://github.com/apache/datafusion-comet/pull/4003"&gt;#4003&lt;/a&gt;): The
  &lt;code&gt;PartialMerge&lt;/code&gt; mode is now executed natively, allowing more multi-stage aggregation plans to remain in
  Comet without falling back to Spark.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;&lt;code&gt;collect_set&lt;/code&gt;&lt;/strong&gt; (&lt;a href="https://github.com/apache/datafusion-comet/pull/3954"&gt;#3954&lt;/a&gt;): Native support for the
  &lt;code&gt;collect_set&lt;/code&gt; aggregate.&lt;/li&gt;
&lt;/ul&gt;
&lt;h3 id="new-expression-support"&gt;New Expression Support&lt;a class="headerlink" href="#new-expression-support" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;This release adds native support for the following Spark expressions:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;strong&gt;Math&lt;/strong&gt;: &lt;code&gt;Pi&lt;/code&gt;, &lt;code&gt;Cbrt&lt;/code&gt;, &lt;code&gt;Acosh&lt;/code&gt;, &lt;code&gt;Asinh&lt;/code&gt;, &lt;code&gt;Atanh&lt;/code&gt;, &lt;code&gt;ToDegrees&lt;/code&gt;, &lt;code&gt;ToRadians&lt;/code&gt;&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Date/time&lt;/strong&gt;: &lt;code&gt;timestamp_seconds&lt;/code&gt;, &lt;code&gt;unix_timestamp&lt;/code&gt; with &lt;code&gt;TimestampNTZType&lt;/code&gt;&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;String / URL&lt;/strong&gt;: &lt;code&gt;url_encode&lt;/code&gt;, &lt;code&gt;url_decode&lt;/code&gt;, &lt;code&gt;try_url_decode&lt;/code&gt;, &lt;code&gt;str_to_map&lt;/code&gt;&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Array / map&lt;/strong&gt;: &lt;code&gt;arrays_zip&lt;/code&gt;, &lt;code&gt;array_position&lt;/code&gt;, &lt;code&gt;array_union&lt;/code&gt;, &lt;code&gt;array_distinct&lt;/code&gt;, &lt;code&gt;arrays_overlap&lt;/code&gt;,
  &lt;code&gt;MapSort&lt;/code&gt; (Spark 4.0)&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Cast&lt;/strong&gt;: string to &lt;code&gt;timestamp_ntz&lt;/code&gt;, cast to and from &lt;code&gt;timestamp_ntz&lt;/code&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;&lt;code&gt;array_insert&lt;/code&gt; and &lt;code&gt;array_compact&lt;/code&gt; have been audited and promoted to &lt;code&gt;Compatible&lt;/code&gt;.&lt;/p&gt;
&lt;h3 id="object-storage"&gt;Object Storage&lt;a class="headerlink" href="#object-storage" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;ul&gt;
&lt;li&gt;&lt;strong&gt;OpenDAL 0.56.0&lt;/strong&gt;: Picks up the latest OpenDAL release, including upstream object-store fixes.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Profile credential chain&lt;/strong&gt;: &lt;code&gt;ProfileCredentialsProvider&lt;/code&gt; is now mapped to the AWS profile credential
  chain, matching the credential resolution behavior users expect.&lt;/li&gt;
&lt;/ul&gt;
&lt;h3 id="native-scan-improvements"&gt;Native Scan Improvements&lt;a class="headerlink" href="#native-scan-improvements" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;ul&gt;
&lt;li&gt;&lt;strong&gt;Parquet field ID matching&lt;/strong&gt;: The &lt;code&gt;native_datafusion&lt;/code&gt; scan now supports field-ID-based column resolution,
  matching Spark's behavior for files written with field IDs.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Schema-mismatch errors&lt;/strong&gt;: &lt;code&gt;native_datafusion&lt;/code&gt; now throws &lt;code&gt;SchemaColumnConvertNotSupportedException&lt;/code&gt; on
  schema mismatch, allowing Spark's standard error handling to engage.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Stricter type validation&lt;/strong&gt;: The &lt;code&gt;native_datafusion&lt;/code&gt; scan now detects incompatible decimal precision/scale
  and string/binary columns read as numeric, and delegates these reads to Spark.&lt;/li&gt;
&lt;/ul&gt;
&lt;h3 id="metrics-and-observability"&gt;Metrics and Observability&lt;a class="headerlink" href="#metrics-and-observability" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;ul&gt;
&lt;li&gt;&lt;strong&gt;Spark UI task output metrics&lt;/strong&gt;: Native execution now reports task output metrics through the standard
  Spark UI path.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Iceberg input metrics&lt;/strong&gt;: Task-level &lt;code&gt;bytesRead&lt;/code&gt; is now reported for the Iceberg native scan, matching
  Comet's native Parquet scan.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Shuffle encode time&lt;/strong&gt;: Shuffle operations now track encode time as a separate metric, making it easier
  to attribute shuffle cost.&lt;/li&gt;
&lt;/ul&gt;
&lt;h2 id="stability-and-correctness"&gt;Stability and Correctness&lt;a class="headerlink" href="#stability-and-correctness" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;ul&gt;
&lt;li&gt;&lt;strong&gt;Substring with negative start index&lt;/strong&gt;: Fixed a Spark-incompatibility in &lt;code&gt;substring&lt;/code&gt; for negative indices.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Strict floating-point comparison&lt;/strong&gt;: &lt;code&gt;RangePartitioning&lt;/code&gt; now honors &lt;code&gt;strictFloatingPoint&lt;/code&gt;, ensuring NaN
  and ±0.0 are partitioned consistently with Spark.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Broadcast / AQE coalescing&lt;/strong&gt;: Broadcast exchanges now bypass AQE partition coalescing, fixing plans that
  could otherwise be coalesced into invalid shapes.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;JNI&lt;/strong&gt;: JNI local frame management has been hardened with explicit error handling.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Shuffle fallback logic&lt;/strong&gt;: Shuffle fallback decisions have been improved, with a new config to gate
  conversion of Spark shuffle to Comet shuffle when the child plan is non-Comet, and a fix to avoid
  redundant columnar shuffle when both parent and child are non-Comet.&lt;/li&gt;
&lt;/ul&gt;
&lt;h2 id="compatibility"&gt;Compatibility&lt;a class="headerlink" href="#compatibility" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;Supported platforms include:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;strong&gt;Spark 3.4.3&lt;/strong&gt; with Java 11/17 and Scala 2.12/2.13&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Spark 3.5.8&lt;/strong&gt; with Java 11/17 and Scala 2.12/2.13&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Spark 4.0.2&lt;/strong&gt; with Java 17 and Scala 2.13&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Spark 4.1.1&lt;/strong&gt; with Java 17 and Scala 2.13&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;See the &lt;a href="https://datafusion.apache.org/comet/user-guide/latest/compatibility/spark-versions.html"&gt;Spark Version Compatibility&lt;/a&gt; page for known limitations specific to each version.&lt;/p&gt;
&lt;p&gt;This release continues to build on &lt;strong&gt;DataFusion 53.1&lt;/strong&gt; and &lt;strong&gt;Arrow 58.1&lt;/strong&gt;.&lt;/p&gt;
&lt;h2 id="get-started-with-comet-0160"&gt;Get Started with Comet 0.16.0&lt;a class="headerlink" href="#get-started-with-comet-0160" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;Ready to try it out? Follow the &lt;a href="https://datafusion.apache.org/comet/user-guide/0.16/installation.html"&gt;Comet 0.16.0 Installation Guide&lt;/a&gt;
to get up and running, then point Comet at your existing Spark workloads — including Spark 4 with ANSI mode
enabled — and see the speedup for yourself.&lt;/p&gt;</content><category term="blog"/></entry><entry><title>Apache DataFusion Comet 0.15.0 Release</title><link href="https://datafusion.apache.org/blog/2026/04/18/datafusion-comet-0.15.0" rel="alternate"/><published>2026-04-18T00:00:00+00:00</published><updated>2026-04-18T00:00:00+00:00</updated><author><name>pmc</name></author><id>tag:datafusion.apache.org,2026-04-18:/blog/2026/04/18/datafusion-comet-0.15.0</id><summary type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;p&gt;The Apache DataFusion PMC is pleased to announce version 0.15.0 of the &lt;a href="https://datafusion.apache.org/comet/"&gt;Comet&lt;/a&gt; subproject.&lt;/p&gt;
&lt;p&gt;Comet is an accelerator for Apache Spark that translates Spark physical plans to DataFusion physical plans for
improved performance and efficiency without requiring any code changes.&lt;/p&gt;
&lt;p&gt;This release covers approximately four weeks of development …&lt;/p&gt;</summary><content type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;p&gt;The Apache DataFusion PMC is pleased to announce version 0.15.0 of the &lt;a href="https://datafusion.apache.org/comet/"&gt;Comet&lt;/a&gt; subproject.&lt;/p&gt;
&lt;p&gt;Comet is an accelerator for Apache Spark that translates Spark physical plans to DataFusion physical plans for
improved performance and efficiency without requiring any code changes.&lt;/p&gt;
&lt;p&gt;This release covers approximately four weeks of development work and is the result of merging 142 PRs from 19
contributors. See the &lt;a href="https://github.com/apache/datafusion-comet/blob/main/dev/changelog/0.15.0.md"&gt;change log&lt;/a&gt; for more information.&lt;/p&gt;
&lt;h2 id="performance"&gt;Performance&lt;a class="headerlink" href="#performance" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;&lt;strong&gt;Comet 0.15.0 provides a 2x speedup for TPC-H @ SF1000 (1TB), resulting in 50% cost savings.&lt;/strong&gt;&lt;/p&gt;
&lt;p&gt;That 2x speedup gives you a choice: finish the same Spark workload in half the time on the cluster you already
have, or match your current Spark performance on roughly half the resources. Either way, the gain translates
directly into lower cloud bills, reduced on-prem capacity, and lower energy usage, with no changes to your
existing Spark SQL, DataFrame, or PySpark code. Comet runs on commodity hardware: no GPUs, FPGAs, or other
specialized accelerators are required, so the savings come from better utilization of the infrastructure you
already run on.&lt;/p&gt;
&lt;p&gt;&lt;img alt="TPC-H Overall Performance" class="img-fluid" src="/blog/images/comet-0.15.0/tpch_allqueries.png" width="100%"/&gt;&lt;/p&gt;
&lt;p&gt;&lt;img alt="TPC-H Query-by-Query Comparison" class="img-fluid" src="/blog/images/comet-0.15.0/tpch_queries_compare.png" width="100%"/&gt;&lt;/p&gt;
&lt;p&gt;See the &lt;a href="https://datafusion.apache.org/comet/contributor-guide/benchmarking.html"&gt;Comet Benchmarking Guide&lt;/a&gt; for
more details.&lt;/p&gt;
&lt;p&gt;Performance was a major theme of this release, with a series of targeted optimizations across the shuffle, scan,
and execution layers.&lt;/p&gt;
&lt;h3 id="reducing-jvmnative-boundary-overhead"&gt;Reducing JVM/Native Boundary Overhead&lt;a class="headerlink" href="#reducing-jvmnative-boundary-overhead" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;Several changes in this release target the cost of crossing between the JVM and native sides, which can dominate
execution time in shuffle- and broadcast-heavy workloads:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;strong&gt;Shuffle read path&lt;/strong&gt;: The native shuffle reader no longer uses FFI on the read side, removing a per-batch cost
  that was particularly visible in shuffle-heavy queries.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Broadcast exchanges&lt;/strong&gt;: Batches are now coalesced before broadcasting, reducing the number of small batches
  crossing the JVM/native boundary.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;FFI-safe operators&lt;/strong&gt;: More operators are marked as FFI-safe, avoiding unnecessary deep copies when crossing
  the JVM/native boundary.&lt;/li&gt;
&lt;/ul&gt;
&lt;h3 id="expanded-native-execution-coverage"&gt;Expanded Native Execution Coverage&lt;a class="headerlink" href="#expanded-native-execution-coverage" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;ul&gt;
&lt;li&gt;&lt;strong&gt;Columnar-to-row (C2R)&lt;/strong&gt;: Native C2R conversion is now exercised for a broader set of query shapes.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;&lt;code&gt;auto&lt;/code&gt; scan mode&lt;/strong&gt;: The &lt;code&gt;auto&lt;/code&gt; scan mode now enables the &lt;code&gt;native_datafusion&lt;/code&gt; scan where supported, giving
  users the benefits of the native Parquet reader without having to explicitly opt in. This is part of the
  ongoing effort to make &lt;code&gt;native_datafusion&lt;/code&gt; the default Parquet path once the deprecation of
  &lt;code&gt;native_iceberg_compat&lt;/code&gt; completes.&lt;/li&gt;
&lt;/ul&gt;
&lt;h3 id="memory-management"&gt;Memory Management&lt;a class="headerlink" href="#memory-management" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;ul&gt;
&lt;li&gt;&lt;strong&gt;Shared memory pools&lt;/strong&gt;: Unified memory pools are now shared across native execution contexts within a Spark
  task, improving memory accounting and reducing OOMs.&lt;/li&gt;
&lt;/ul&gt;
&lt;h3 id="object-storage-io"&gt;Object Storage I/O&lt;a class="headerlink" href="#object-storage-io" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;ul&gt;
&lt;li&gt;&lt;strong&gt;Object store caching&lt;/strong&gt;: Object stores and bucket region lookups are cached, dramatically reducing DNS query
  volume on workloads that open many files.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;&lt;code&gt;get_ranges&lt;/code&gt; performance&lt;/strong&gt;: Picked up an upstream &lt;code&gt;opendal&lt;/code&gt; fix that restores fast range reads from object
  storage.&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;Together, these changes reduce CPU and memory overhead for shuffle-heavy, broadcast-heavy, and
object-storage-bound workloads.&lt;/p&gt;
&lt;h2 id="native-iceberg-reader-enabled-by-default"&gt;Native Iceberg Reader Enabled by Default&lt;a class="headerlink" href="#native-iceberg-reader-enabled-by-default" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;This release marks a major milestone for Iceberg users: &lt;strong&gt;Comet's fully-native Iceberg reader is now enabled by
default&lt;/strong&gt;. Workloads that read Iceberg tables will automatically benefit from native Rust-based scans built on
iceberg-rust, with no additional configuration required.&lt;/p&gt;
&lt;p&gt;To support this change, the release bundles a broad set of Iceberg-focused improvements:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;strong&gt;Dynamic Partition Pruning (DPP)&lt;/strong&gt;: The native Iceberg reader supports DPP, allowing partition filters
  derived at runtime to prune Iceberg file scans and substantially reduce I/O for star-schema-style queries.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Correct classloader handling&lt;/strong&gt;: Iceberg classes are now loaded via the thread context classloader, resolving
  class-loading issues in environments where the executor classloader differs from the application classloader.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Continuous Iceberg CI&lt;/strong&gt;: Iceberg Spark integration tests now run on every PR and push to &lt;code&gt;main&lt;/code&gt;, providing
  continuous validation of the native Iceberg code path. Test diffs for Spark 3.4 were updated to keep the matrix
  green across supported Spark versions.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;iceberg-rust upgrade&lt;/strong&gt;: Comet picks up the latest iceberg-rust, pulling in fixes for Parquet reader edge cases
  discovered in earlier testing.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Refreshed documentation&lt;/strong&gt;: The Iceberg user guide has been rewritten to reflect current capabilities, and the
  contributor guide now documents how to run the Iceberg Spark test suites locally.&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;Users who need to fall back to the previous behavior can still opt out, but we encourage the community to exercise
the native reader and report any issues.&lt;/p&gt;
&lt;h3 id="sort-merge-join-performance"&gt;Sort-Merge Join Performance&lt;a class="headerlink" href="#sort-merge-join-performance" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;Comet relies heavily on sort-merge join (SMJ) because DataFusion's hash joins do not yet support spilling to
disk. For larger-than-memory joins, SMJ is the only viable path, making its performance critical for real-world
workloads at scale.&lt;/p&gt;
&lt;p&gt;DataFusion 53 includes several SMJ improvements that Comet 0.15.0 benefits from directly:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;strong&gt;Zero-copy slicing&lt;/strong&gt; instead of the take kernel (&lt;a href="https://github.com/apache/datafusion/pull/20463"&gt;datafusion#20463&lt;/a&gt;)&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Streaming output&lt;/strong&gt; instead of waiting for all input before emitting (&lt;a href="https://github.com/apache/datafusion/pull/20482"&gt;datafusion#20482&lt;/a&gt;)&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Cached row counts&lt;/strong&gt; to avoid O(n) recounting (&lt;a href="https://github.com/apache/datafusion/pull/20478"&gt;datafusion#20478&lt;/a&gt;)&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;Additional SMJ work is landing in upstream DataFusion and will arrive in a future Comet release:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;Specialized semi/anti join stream (&lt;a href="https://github.com/apache/datafusion/pull/20806"&gt;datafusion#20806&lt;/a&gt;)&lt;/li&gt;
&lt;li&gt;Batch deferred filtering with 20–50x improvements for near-unique LEFT and FULL joins (&lt;a href="https://github.com/apache/datafusion/pull/21184"&gt;datafusion#21184&lt;/a&gt;)&lt;/li&gt;
&lt;li&gt;DynComparator for ~5% TPC-H improvement (&lt;a href="https://github.com/apache/datafusion/pull/21484"&gt;datafusion#21484&lt;/a&gt;)&lt;/li&gt;
&lt;li&gt;Vec-based filter state replacing HashMap (&lt;a href="https://github.com/apache/datafusion/pull/21517"&gt;datafusion#21517&lt;/a&gt;)&lt;/li&gt;
&lt;li&gt;Full outer join correctness fix for NULL filter results (&lt;a href="https://github.com/apache/datafusion/pull/21660"&gt;datafusion#21660&lt;/a&gt;)&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;With these performance improvements, the next release of Comet will enable SMJ with filters by default.&lt;/p&gt;
&lt;h2 id="other-key-features"&gt;Other Key Features&lt;a class="headerlink" href="#other-key-features" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;h3 id="new-expressions-and-function-support"&gt;New Expressions and Function Support&lt;a class="headerlink" href="#new-expressions-and-function-support" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;This release adds support for the following:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;strong&gt;Date/time functions&lt;/strong&gt;: &lt;code&gt;days&lt;/code&gt;, &lt;code&gt;hours&lt;/code&gt;, &lt;code&gt;date_from_unix_date&lt;/code&gt;&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;String/JSON functions&lt;/strong&gt;: native &lt;code&gt;get_json_object&lt;/code&gt; with improved performance over the fallback path&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Hash/math functions&lt;/strong&gt;: &lt;code&gt;bin&lt;/code&gt;&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Array functions&lt;/strong&gt;: &lt;code&gt;sort_array&lt;/code&gt;&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Window functions&lt;/strong&gt;: &lt;code&gt;LEAD&lt;/code&gt; and &lt;code&gt;LAG&lt;/code&gt; with &lt;code&gt;IGNORE NULLS&lt;/code&gt;&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Aggregates&lt;/strong&gt;: SQL &lt;code&gt;FILTER (WHERE ...)&lt;/code&gt; clauses now execute natively; &lt;code&gt;Corr&lt;/code&gt; aggregate enabled&lt;/li&gt;
&lt;/ul&gt;
&lt;h3 id="expanded-metrics-and-observability"&gt;Expanded Metrics and Observability&lt;a class="headerlink" href="#expanded-metrics-and-observability" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;Comet metrics can now be exposed through Spark's external monitoring system, making it easier to integrate Comet
execution statistics with existing observability dashboards. Native DataFusion scans also now report accurate
&lt;code&gt;filesScanned&lt;/code&gt; and &lt;code&gt;bytesScanned&lt;/code&gt; input metrics, matching Spark's native Parquet scan reporting.&lt;/p&gt;
&lt;h2 id="stability-and-correctness"&gt;Stability and Correctness&lt;a class="headerlink" href="#stability-and-correctness" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;A significant portion of this release is dedicated to stability and Spark compatibility. Highlights include:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;strong&gt;Cast string to timestamp&lt;/strong&gt;: Multiple fixes for UTC timestamps, timezone handling, special formats
  (&lt;code&gt;epoch&lt;/code&gt;, &lt;code&gt;now&lt;/code&gt;, etc.), and compatibility with Spark's semantics.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Cast decimal to string&lt;/strong&gt;: Added legacy mode handling to match Spark's output formatting.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;String to decimal&lt;/strong&gt;: Support for full-width characters, null characters, and negative scale.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Decimal arithmetic&lt;/strong&gt;: Fixes for decimal division and additional test coverage for ANSI overflow handling,
  including scalar decimal overflow.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Array expressions&lt;/strong&gt;: Corrected &lt;code&gt;GetArrayItem&lt;/code&gt; null handling for dynamic indices; &lt;code&gt;array_append&lt;/code&gt; return type
  fixed and marked &lt;code&gt;Compatible&lt;/code&gt;; audited &lt;code&gt;array_insert&lt;/code&gt; for correctness; &lt;code&gt;array_compact&lt;/code&gt; marked &lt;code&gt;Compatible&lt;/code&gt;;
  array-to-array cast enabled.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;DateTrunc/TimestampTrunc&lt;/strong&gt;: Fixed native crashes when the input is a literal.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Ambiguous local times&lt;/strong&gt;: Correct handling of ambiguous and non-existent local times across DST transitions.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Case-insensitive Parquet fields&lt;/strong&gt;: &lt;code&gt;native_datafusion&lt;/code&gt; now correctly detects duplicate/ambiguous fields in
  case-insensitive mode and falls back where appropriate.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Shuffle planning&lt;/strong&gt;: Shuffle fallback decisions are now "sticky" across planning passes, and Comet columnar
  shuffle is skipped for stages containing DPP scans to avoid mismatched partitioning.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Error propagation&lt;/strong&gt;: Native error messages are now propagated through &lt;code&gt;SparkException&lt;/code&gt; even when the
  &lt;code&gt;errorClass&lt;/code&gt; is empty, and file-not-found errors flow through the standard Spark error JSON path.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Trigonometric compatibility&lt;/strong&gt;: &lt;code&gt;tan&lt;/code&gt; and &lt;code&gt;atan2&lt;/code&gt; are now Spark-compatible.&lt;/li&gt;
&lt;/ul&gt;
&lt;h2 id="dependency-upgrades"&gt;Dependency Upgrades&lt;a class="headerlink" href="#dependency-upgrades" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;This release upgrades to &lt;strong&gt;DataFusion 53.1&lt;/strong&gt; and &lt;strong&gt;Arrow 58.1&lt;/strong&gt;, and picks up the latest &lt;code&gt;iceberg-rust&lt;/code&gt; release
with additional reader fixes. The &lt;code&gt;jni&lt;/code&gt; crate was upgraded to 0.22.4.&lt;/p&gt;
&lt;h2 id="deprecations-and-removals"&gt;Deprecations and Removals&lt;a class="headerlink" href="#deprecations-and-removals" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;The &lt;code&gt;SupportsComet&lt;/code&gt; interface has been removed, along with the Java-based Iceberg integration path (which is
fully superseded by the native Iceberg reader). See &lt;a href="https://github.com/apache/datafusion-comet/issues/2921"&gt;comet#2921&lt;/a&gt;
for background on the decision to standardize on the native iceberg-rust integration. The &lt;code&gt;native_iceberg_compat&lt;/code&gt;
scan remains deprecated and is expected to be removed in a future release in favor of &lt;code&gt;native_datafusion&lt;/code&gt;.&lt;/p&gt;
&lt;h2 id="compatibility"&gt;Compatibility&lt;a class="headerlink" href="#compatibility" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;Supported platforms include Spark 3.4.3, 3.5.4–3.5.8, and Spark 4.0.x with various JDK and Scala combinations.&lt;/p&gt;
&lt;p&gt;The community encourages users to test Comet with existing Spark and Iceberg workloads and welcomes contributions
to ongoing development.&lt;/p&gt;
&lt;h2 id="get-started-with-comet-0150"&gt;Get Started with Comet 0.15.0&lt;a class="headerlink" href="#get-started-with-comet-0150" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;Ready to try it out? Follow the &lt;a href="https://datafusion.apache.org/comet/user-guide/0.15/installation.html"&gt;Comet 0.15.0 Installation Guide&lt;/a&gt;
to get up and running, then point Comet at your existing Spark workloads and see the speedup for yourself.&lt;/p&gt;</content><category term="blog"/></entry><entry><title>Apache DataFusion 53.0.0 Released</title><link href="https://datafusion.apache.org/blog/2026/04/02/datafusion-53.0.0" rel="alternate"/><published>2026-04-02T00:00:00+00:00</published><updated>2026-04-02T00:00:00+00:00</updated><author><name>pmc</name></author><id>tag:datafusion.apache.org,2026-04-02:/blog/2026/04/02/datafusion-53.0.0</id><summary type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;p&gt;We are proud to announce the release of &lt;a href="https://crates.io/crates/datafusion/53.0.0"&gt;DataFusion 53.0.0&lt;/a&gt;. This post highlights
some of the major improvements since &lt;a href="https://datafusion.apache.org/blog/2026/01/12/datafusion-52.0.0/"&gt;DataFusion 52.0.0&lt;/a&gt;. The complete list of
changes is available in the &lt;a href="https://github.com/apache/datafusion/blob/branch-53/dev/changelog/53.0.0.md"&gt;changelog&lt;/a&gt;. Thanks to the &lt;a href="https://github.com/apache/datafusion/blob/branch-53/dev/changelog/53.0.0.md#credits"&gt;114 contributors&lt;/a&gt; for
making this release possible.&lt;/p&gt;
&lt;h2 id="performance-improvements"&gt;Performance Improvements 🚀&lt;a class="headerlink" href="#performance-improvements" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;&lt;img alt="Performance over time" class="img-fluid" src="/blog/images/datafusion-53.0.0/performance_over_time_clickbench.png" width="100%"/&gt;&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Figure 1&lt;/strong&gt;: Average …&lt;/p&gt;</summary><content type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;p&gt;We are proud to announce the release of &lt;a href="https://crates.io/crates/datafusion/53.0.0"&gt;DataFusion 53.0.0&lt;/a&gt;. This post highlights
some of the major improvements since &lt;a href="https://datafusion.apache.org/blog/2026/01/12/datafusion-52.0.0/"&gt;DataFusion 52.0.0&lt;/a&gt;. The complete list of
changes is available in the &lt;a href="https://github.com/apache/datafusion/blob/branch-53/dev/changelog/53.0.0.md"&gt;changelog&lt;/a&gt;. Thanks to the &lt;a href="https://github.com/apache/datafusion/blob/branch-53/dev/changelog/53.0.0.md#credits"&gt;114 contributors&lt;/a&gt; for
making this release possible.&lt;/p&gt;
&lt;h2 id="performance-improvements"&gt;Performance Improvements 🚀&lt;a class="headerlink" href="#performance-improvements" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;&lt;img alt="Performance over time" class="img-fluid" src="/blog/images/datafusion-53.0.0/performance_over_time_clickbench.png" width="100%"/&gt;&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Figure 1&lt;/strong&gt;: Average and median normalized execution times for DataFusion 53.0.0 on ClickBench queries, compared to previous releases.
Query times are normalized using the ClickBench definition. See the
&lt;a href="https://alamb.github.io/datafusion-benchmarking/"&gt;DataFusion Benchmarking Page&lt;/a&gt;
for more details.&lt;/p&gt;
&lt;p&gt;DataFusion 53 continues the project-wide focus on performance. This release
reduces planning overhead, skips more unnecessary I/O, and pushes more work
into earlier and cheaper stages of execution.&lt;/p&gt;
&lt;h3 id="limit-aware-parquet-row-group-pruning"&gt;&lt;code&gt;LIMIT&lt;/code&gt;-Aware Parquet Row Group Pruning&lt;a class="headerlink" href="#limit-aware-parquet-row-group-pruning" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;DataFusion 53 includes a new optimization that makes Parquet pruning aware of
&lt;code&gt;LIMIT&lt;/code&gt;. This optimization is described in full in &lt;a href="https://datafusion.apache.org/blog/2026/03/20/limit-pruning/"&gt;limit pruning blog post&lt;/a&gt;. If
DataFusion can prove that an entire row group matches the predicate, and those
fully matching row groups contain enough rows to satisfy the &lt;code&gt;LIMIT&lt;/code&gt;, partially
matching row groups are skipped entirely.&lt;/p&gt;
&lt;figure&gt;
&lt;img alt="Pruning pipeline with limit pruning highlighted" class="img-fluid" src="/blog/images/limit-pruning/pruning-pipeline.svg" width="80%"/&gt;
&lt;figcaption&gt;&lt;b&gt;Figure 2&lt;/b&gt;: Limit pruning is inserted between row group and page index pruning.&lt;/figcaption&gt;
&lt;/figure&gt;
&lt;p&gt;Thanks to &lt;a href="https://github.com/xudong963"&gt;@xudong963&lt;/a&gt; for implementing this feature. Related PRs: &lt;a href="https://github.com/apache/datafusion/pull/18868"&gt;#18868&lt;/a&gt;&lt;/p&gt;
&lt;h3 id="improved-filter-pushdown"&gt;Improved Filter Pushdown&lt;a class="headerlink" href="#improved-filter-pushdown" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;DataFusion 53 pushes filters down through more join types and through &lt;code&gt;UnionExec&lt;/code&gt;,
and expands support for pushing down &lt;a href="https://datafusion.apache.org/blog/2025/09/10/dynamic-filters"&gt;dynamic filters&lt;/a&gt;. More
pushdown means fewer rows flow into joins, repartitions, and later operators,
which reduces CPU, memory, and I/O.&lt;/p&gt;
&lt;p&gt;For example:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-sql"&gt;SELECT *
FROM (
    SELECT *
    FROM t1
    LEFT ANTI JOIN t2 ON t1.k = t2.k
) a
JOIN t1 b ON a.k = b.k
WHERE b.v = 1;
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;Now DataFusion can often transform the physical plan so filters and
&lt;a href="https://datafusion.apache.org/blog/2025/09/10/dynamic-filters"&gt;dynamic filters&lt;/a&gt; are pushed deeper into the plan, even through subqueries and
nested joins. In this example, the filter on &lt;code&gt;b.v&lt;/code&gt; helps produce dynamic filters
that can be pushed into both sides of the nested anti join.&lt;/p&gt;
&lt;figure&gt;
&lt;img alt="Before and after diagram of dynamic filter pushdown through a subquery with nested joins" class="img-fluid" src="/blog/images/datafusion-53.0.0/join-filter-pushdown.svg" width="100%"/&gt;
&lt;figcaption&gt;&lt;b&gt;Figure 3&lt;/b&gt;: DataFusion 53 pushes dynamic filters through subqueries and into both sides of nested joins.&lt;/figcaption&gt;
&lt;/figure&gt;
&lt;p&gt;Thanks to &lt;a href="https://github.com/nuno-faria"&gt;@nuno-faria&lt;/a&gt;, &lt;a href="https://github.com/haohuaijin"&gt;@haohuaijin&lt;/a&gt;, and &lt;a href="https://github.com/jackkleeman"&gt;@jackkleeman&lt;/a&gt; for
driving this work. Related PRs: &lt;a href="https://github.com/apache/datafusion/pull/19918"&gt;#19918&lt;/a&gt;, &lt;a href="https://github.com/apache/datafusion/pull/20145"&gt;#20145&lt;/a&gt;, &lt;a href="https://github.com/apache/datafusion/pull/20192"&gt;#20192&lt;/a&gt;&lt;/p&gt;
&lt;h3 id="faster-query-planning"&gt;Faster Query Planning&lt;a class="headerlink" href="#faster-query-planning" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;DataFusion 53 improves query planning performance by making immutable pieces of
execution plans cheaper to clone. This helps applications that need extremely
low latency, plan many or complex queries, or use prepared statements or
parameterized queries. In some benchmarks, overall execution time drops from
roughly 4-5 ms to about 100 us.&lt;/p&gt;
&lt;p&gt;Thanks to &lt;a href="https://github.com/askalt"&gt;@askalt&lt;/a&gt; for leading this work. Related PRs: &lt;a href="https://github.com/apache/datafusion/pull/19792"&gt;#19792&lt;/a&gt;, &lt;a href="https://github.com/apache/datafusion/pull/19893"&gt;#19893&lt;/a&gt;&lt;/p&gt;
&lt;h3 id="faster-functions"&gt;Faster Functions&lt;a class="headerlink" href="#faster-functions" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;DataFusion includes &lt;a href="https://datafusion.apache.org/user-guide/sql/scalar_functions.html"&gt;235 built-in functions&lt;/a&gt;. Improving the performance of these
functions benefits a wide range of workloads. This release improves the performance of 42 of those
functions, such as &lt;a href="https://github.com/apache/datafusion/pull/20295"&gt;strpos&lt;/a&gt;, &lt;a href="https://github.com/apache/datafusion/pull/20344"&gt;replace&lt;/a&gt;, &lt;a href="https://github.com/apache/datafusion/pull/20317"&gt;concat&lt;/a&gt;, &lt;a href="https://github.com/apache/datafusion/pull/20305"&gt;translate&lt;/a&gt;, &lt;a href="https://github.com/apache/datafusion/pull/20374"&gt;array_has&lt;/a&gt;,
&lt;a href="https://github.com/apache/datafusion/pull/20504"&gt;array_agg&lt;/a&gt;, &lt;a href="https://github.com/apache/datafusion/pull/19980"&gt;left&lt;/a&gt;, &lt;a href="https://github.com/apache/datafusion/pull/20069"&gt;right&lt;/a&gt;, and &lt;a href="https://github.com/apache/datafusion/pull/20097"&gt;case_when&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;Thanks to the contributors who drove this work, especially &lt;a href="https://github.com/neilconway"&gt;@neilconway&lt;/a&gt;,
&lt;a href="https://github.com/theirix"&gt;@theirix&lt;/a&gt;, &lt;a href="https://github.com/lyne7-sc"&gt;@lyne7-sc&lt;/a&gt;, &lt;a href="https://github.com/kumarUjjawal"&gt;@kumarUjjawal&lt;/a&gt;, &lt;a href="https://github.com/pepijnve"&gt;@pepijnve&lt;/a&gt;, &lt;a href="https://github.com/zhangxffff"&gt;@zhangxffff&lt;/a&gt;, and
&lt;a href="https://github.com/UBarney"&gt;@UBarney&lt;/a&gt;.&lt;/p&gt;
&lt;h3 id="nested-field-pushdown"&gt;Nested Field Pushdown&lt;a class="headerlink" href="#nested-field-pushdown" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;DataFusion 53 pushes expressions such as &lt;code&gt;get_field&lt;/code&gt; down the plan and into data
sources. This is especially important for nested data such as structs in
Parquet files. Instead of reading an entire struct column and then extracting
the field of interest, DataFusion 53 pushes the field extraction into the scan.&lt;/p&gt;
&lt;p&gt;For example, the following query reads a struct column &lt;code&gt;s&lt;/code&gt; and extracts the
&lt;code&gt;label&lt;/code&gt; field for rows where the &lt;code&gt;value&lt;/code&gt; field is greater than 150:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-sql"&gt;SELECT id, s['label']
FROM t
WHERE s['value'] &amp;gt; 150;
&lt;/code&gt;&lt;/pre&gt;
&lt;figure&gt;
&lt;img alt="Before and after diagram of field access pushdown into a data source" class="img-fluid" src="/blog/images/datafusion-53.0.0/field-access-pushdown.svg" width="80%"/&gt;
&lt;figcaption&gt;&lt;b&gt;Figure 4&lt;/b&gt;: DataFusion 53 pushes field-access expressions closer to the scan.&lt;/figcaption&gt;
&lt;/figure&gt;
&lt;p&gt;Special thanks to &lt;a href="https://github.com/adriangb"&gt;@adriangb&lt;/a&gt; for designing and implementing this optimizer
work. Related PRs: &lt;a href="https://github.com/apache/datafusion/pull/20065"&gt;#20065&lt;/a&gt;, &lt;a href="https://github.com/apache/datafusion/pull/20117"&gt;#20117&lt;/a&gt;, &lt;a href="https://github.com/apache/datafusion/pull/20239"&gt;#20239&lt;/a&gt;&lt;/p&gt;
&lt;h2 id="new-features"&gt;New Features ✨&lt;a class="headerlink" href="#new-features" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;ul&gt;
&lt;li&gt;
&lt;p&gt;&lt;strong&gt;JSON Array File Support&lt;/strong&gt;: DataFusion 53 can now read JSON arrays such as
  &lt;code&gt;[{...}, {...}]&lt;/code&gt; directly as multiple rows, including streaming inputs from
  object stores. Thanks to &lt;a href="https://github.com/zhuqi-lucas"&gt;@zhuqi-lucas&lt;/a&gt; for implementing this feature.
  Related PRs: &lt;a href="https://github.com/apache/datafusion/pull/19924"&gt;#19924&lt;/a&gt;&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;
&lt;p&gt;&lt;strong&gt;Support for &lt;code&gt;:&lt;/code&gt; operator&lt;/strong&gt;: DataFusion can plan queries such as
  &lt;code&gt;SELECT payload:'user_id' FROM events;&lt;/code&gt;, enabling better &lt;a href="https://parquet.apache.org/blog/2026/02/27/variant-type-in-apache-parquet-for-semi-structured-data/"&gt;Parquet Variant&lt;/a&gt;
  support via &lt;a href="https://github.com/datafusion-contrib/datafusion-variant"&gt;datafusion-variant&lt;/a&gt;. Thanks to &lt;a href="https://github.com/Samyak2"&gt;@Samyak2&lt;/a&gt;. Related PRs:
  &lt;a href="https://github.com/apache/datafusion/pull/20717"&gt;#20717&lt;/a&gt;&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;
&lt;p&gt;&lt;strong&gt;New SQL&lt;/strong&gt;: DataFusion supports additional set-comparison subqueries, null-aware
  anti join, and deletion predicates. Thanks to &lt;a href="https://github.com/waynexia"&gt;@waynexia&lt;/a&gt;, &lt;a href="https://github.com/viirya"&gt;@viirya&lt;/a&gt;, and
  &lt;a href="https://github.com/askalt"&gt;@askalt&lt;/a&gt; for key contributions in this area. Related PRs: &lt;a href="https://github.com/apache/datafusion/pull/19109"&gt;#19109&lt;/a&gt;,
  &lt;a href="https://github.com/apache/datafusion/pull/19635"&gt;#19635&lt;/a&gt;, &lt;a href="https://github.com/apache/datafusion/pull/20137"&gt;#20137&lt;/a&gt;&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;
&lt;p&gt;&lt;strong&gt;Spark-Compatible Functions&lt;/strong&gt;: This release includes almost 20 new or improved
  Spark-compatible functions and behaviors in the &lt;a href="https://docs.rs/datafusion-spark/latest/datafusion_spark/index.html"&gt;datafusion-spark crate&lt;/a&gt;.
  It includes functions such as &lt;a href="https://github.com/apache/datafusion/pull/19699"&gt;collect_list&lt;/a&gt;, &lt;a href="https://github.com/apache/datafusion/pull/19845"&gt;date_diff&lt;/a&gt;,
  &lt;a href="https://github.com/apache/datafusion/pull/19880"&gt;from_utc_timestamp&lt;/a&gt;, &lt;a href="https://github.com/apache/datafusion/pull/20412"&gt;json_tuple&lt;/a&gt;, &lt;a href="https://github.com/apache/datafusion/pull/20440"&gt;arrays_zip&lt;/a&gt;, &lt;a href="https://github.com/apache/datafusion/pull/20479"&gt;bin&lt;/a&gt;, and &lt;a href="https://github.com/apache/datafusion/pull/20685"&gt;array_contains&lt;/a&gt;.
  Thanks to the contributors who drove this work, especially &lt;a href="https://github.com/cht42"&gt;@cht42&lt;/a&gt;,
  &lt;a href="https://github.com/CuteChuanChuan"&gt;@CuteChuanChuan&lt;/a&gt;, &lt;a href="https://github.com/SubhamSinghal"&gt;@SubhamSinghal&lt;/a&gt;, &lt;a href="https://github.com/kazantsev-maksim"&gt;@kazantsev-maksim&lt;/a&gt;, &lt;a href="https://github.com/unknowntpo"&gt;@unknowntpo&lt;/a&gt;,
  &lt;a href="https://github.com/aryan-212"&gt;@aryan-212&lt;/a&gt;, &lt;a href="https://github.com/hsiang-c"&gt;@hsiang-c&lt;/a&gt;, and &lt;a href="https://github.com/davidlghellin"&gt;@davidlghellin&lt;/a&gt;.&lt;/p&gt;
&lt;/li&gt;
&lt;/ul&gt;
&lt;h2 id="stability-and-release-engineering"&gt;Stability and Release Engineering 🦺&lt;a class="headerlink" href="#stability-and-release-engineering" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;The community spent significant time this release cycle stabilizing the release
branch and improving the release process. While such improvements are not as
headline-friendly as new features, they are highly important for real
deployments. We are discussing ways to improve the process on &lt;a href="https://github.com/apache/datafusion/issues/21034"&gt;#21034&lt;/a&gt; and would
welcome suggestions and contributions to help with release engineering work in
the future.&lt;/p&gt;
&lt;p&gt;Thanks to &lt;a href="https://github.com/comphead"&gt;@comphead&lt;/a&gt; for running this release, and to &lt;a href="https://github.com/jonathanc-n"&gt;@jonathanc-n&lt;/a&gt;, &lt;a href="https://github.com/alamb"&gt;@alamb&lt;/a&gt;,
&lt;a href="https://github.com/xanderbailey"&gt;@xanderbailey&lt;/a&gt;, &lt;a href="https://github.com/haohuaijin"&gt;@haohuaijin&lt;/a&gt;, &lt;a href="https://github.com/friendlymatthew"&gt;@friendlymatthew&lt;/a&gt;, &lt;a href="https://github.com/fwojciec"&gt;@fwojciec&lt;/a&gt;,
&lt;a href="https://github.com/Kontinuation"&gt;@Kontinuation&lt;/a&gt;, &lt;a href="https://github.com/nathanb9"&gt;@nathanb9&lt;/a&gt;, and many others who helped stabilize the release
branch.&lt;/p&gt;
&lt;h2 id="upgrade-notes"&gt;Upgrade Notes&lt;a class="headerlink" href="#upgrade-notes" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;DataFusion 53 includes some breaking changes, including updates to the SQL
parser, optimizer behavior, and some physical-plan APIs. Please see the &lt;a href="https://datafusion.apache.org/library-user-guide/upgrading/index.html"&gt;upgrade
guide&lt;/a&gt; and &lt;a href="https://github.com/apache/datafusion/blob/branch-53/dev/changelog/53.0.0.md"&gt;changelog&lt;/a&gt; for the full details before upgrading.&lt;/p&gt;
&lt;h2 id="known-issues"&gt;Known Issues&lt;a class="headerlink" href="#known-issues" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;A small number of issues were discovered after the 53.0.0 release,
and we expect to publish DataFusion 53.1.0 soon. See the &lt;a href="https://github.com/apache/datafusion/issues/21079"&gt;53.1.0 release tracking
issue&lt;/a&gt; for the latest status.&lt;/p&gt;
&lt;h2 id="thank-you"&gt;Thank You&lt;a class="headerlink" href="#thank-you" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;Thank you to everyone in the DataFusion community who contributed code, reviews,
testing, bug reports, documentation, and release engineering work for 53.0.0.
This release contains direct contributions from 114 different people, and we are
grateful for the time and effort that everyone put in to make it happen.&lt;/p&gt;</content><category term="blog"/></entry><entry><title>Writing Custom Table Providers in Apache DataFusion</title><link href="https://datafusion.apache.org/blog/2026/03/31/writing-table-providers" rel="alternate"/><published>2026-03-31T00:00:00+00:00</published><updated>2026-03-31T00:00:00+00:00</updated><author><name>Tim Saucer (rerun.io)</name></author><id>tag:datafusion.apache.org,2026-03-31:/blog/2026/03/31/writing-table-providers</id><summary type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;p&gt;One of DataFusion's greatest strengths is its extensibility. If your data lives
in a custom format, behind an API, or in a system that DataFusion does not
natively support, you can teach DataFusion to read it by implementing a
&lt;strong&gt;custom table provider&lt;/strong&gt;. This post walks through the three layers you …&lt;/p&gt;</summary><content type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;p&gt;One of DataFusion's greatest strengths is its extensibility. If your data lives
in a custom format, behind an API, or in a system that DataFusion does not
natively support, you can teach DataFusion to read it by implementing a
&lt;strong&gt;custom table provider&lt;/strong&gt;. This post walks through the three layers you need to
understand to design a table provider and where planning and execution work should happen.&lt;/p&gt;
&lt;h2 id="the-three-layers"&gt;The Three Layers&lt;a class="headerlink" href="#the-three-layers" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;hr/&gt;
&lt;p&gt;When DataFusion executes a query against a table, three abstractions collaborate
to produce results:&lt;/p&gt;
&lt;ol&gt;
&lt;li&gt;&lt;strong&gt;&lt;a href="https://docs.rs/datafusion/latest/datafusion/catalog/trait.TableProvider.html"&gt;TableProvider&lt;/a&gt;&lt;/strong&gt; -- Describes the table (schema, capabilities) and
   produces an execution plan when queried. This is part of the &lt;strong&gt;Logical Plan&lt;/strong&gt;.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;&lt;a href="https://docs.rs/datafusion/latest/datafusion/physical_plan/trait.ExecutionPlan.html"&gt;ExecutionPlan&lt;/a&gt;&lt;/strong&gt; -- Describes &lt;em&gt;how&lt;/em&gt; to compute the result: partitioning,
   ordering, and child plan relationships. This is part of the &lt;strong&gt;Physical Plan&lt;/strong&gt;.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;&lt;a href="https://docs.rs/datafusion/latest/datafusion/execution/type.SendableRecordBatchStream.html"&gt;SendableRecordBatchStream&lt;/a&gt;&lt;/strong&gt; -- The async stream that &lt;em&gt;actually does the
   work&lt;/em&gt;, yielding &lt;code&gt;RecordBatch&lt;/code&gt;es one at a time.&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;Think of these as a funnel: &lt;code&gt;TableProvider::scan()&lt;/code&gt; is called once during
planning to create an &lt;code&gt;ExecutionPlan&lt;/code&gt;, then &lt;code&gt;ExecutionPlan::execute()&lt;/code&gt; is called
once per partition to create a stream, and those streams are where rows are
actually produced during execution.&lt;/p&gt;
&lt;h2 id="background-logical-and-physical-planning"&gt;Background: Logical and Physical Planning&lt;a class="headerlink" href="#background-logical-and-physical-planning" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;hr/&gt;
&lt;p&gt;Before diving into the three layers, it helps to understand how DataFusion
processes a query. There are several phases between a SQL string (or DataFrame
call) and streaming results:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-text"&gt;SQL / DataFrame API
  → Logical Plan          (abstract: what to compute)
  → Logical Optimization  (rewrite rules that preserve semantics)
  → Physical Plan         (concrete: how to compute it)
  → Physical Optimization (hardware- and data-aware rewrites)
  → Execution             (streaming RecordBatches)
&lt;/code&gt;&lt;/pre&gt;
&lt;h3 id="logical-planning"&gt;Logical Planning&lt;a class="headerlink" href="#logical-planning" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;A &lt;strong&gt;logical plan&lt;/strong&gt; describes &lt;em&gt;what&lt;/em&gt; the query computes without specifying &lt;em&gt;how&lt;/em&gt;.
It is a tree of relational operators -- &lt;code&gt;TableScan&lt;/code&gt;, &lt;code&gt;Filter&lt;/code&gt;, &lt;code&gt;Projection&lt;/code&gt;,
&lt;code&gt;Aggregate&lt;/code&gt;, &lt;code&gt;Join&lt;/code&gt;, &lt;code&gt;Sort&lt;/code&gt;, &lt;code&gt;Limit&lt;/code&gt;, and so on. The logical optimizer rewrites
this tree to reduce work while preserving the query's meaning. Some logical
optimizations include:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;strong&gt;Predicate pushdown&lt;/strong&gt; -- moves filters as close to the data source as
  possible, so fewer rows flow through the rest of the plan.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Projection pruning&lt;/strong&gt; -- eliminates columns that are never referenced
  downstream, reducing memory and I/O.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Expression simplification&lt;/strong&gt; -- rewrites expressions like &lt;code&gt;1 = 1&lt;/code&gt; or
  &lt;code&gt;x AND true&lt;/code&gt; into simpler forms.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Subquery decorrelation&lt;/strong&gt; -- converts correlated &lt;code&gt;IN&lt;/code&gt; / &lt;code&gt;EXISTS&lt;/code&gt; subqueries
  into more efficient semi-joins.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Limit pushdown&lt;/strong&gt; -- pushes &lt;code&gt;LIMIT&lt;/code&gt; earlier in the plan so operators
  produce less data.&lt;/li&gt;
&lt;/ul&gt;
&lt;h3 id="physical-planning"&gt;Physical Planning&lt;a class="headerlink" href="#physical-planning" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;The &lt;strong&gt;physical planner&lt;/strong&gt; converts the optimized logical plan into an
&lt;code&gt;ExecutionPlan&lt;/code&gt; tree -- the concrete plan that will actually run. This is where
decisions like "use a hash join vs. a sort-merge join" or "how many partitions
to scan" are made. The physical optimizer then refines this tree further with rewrites such as:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;strong&gt;Distribution enforcement&lt;/strong&gt; -- inserts &lt;code&gt;RepartitionExec&lt;/code&gt; nodes so that data
  is partitioned correctly for joins and aggregations.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Sort enforcement&lt;/strong&gt; -- inserts &lt;code&gt;SortExec&lt;/code&gt; nodes where ordering is required,
  and removes them where the data is already sorted.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Join selection&lt;/strong&gt; -- picks the most efficient join strategy based on
  statistics and table sizes.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Aggregate optimization&lt;/strong&gt; -- combines partial and final aggregation stages,
  and can use exact statistics to skip scanning entirely.&lt;/li&gt;
&lt;/ul&gt;
&lt;h3 id="why-this-matters-for-table-providers"&gt;Why This Matters for Table Providers&lt;a class="headerlink" href="#why-this-matters-for-table-providers" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;Your &lt;code&gt;TableProvider&lt;/code&gt; sits at the boundary between logical and physical planning.
During logical optimization, DataFusion determines which filters and projections
&lt;em&gt;could&lt;/em&gt; be pushed down to the source. When &lt;code&gt;scan()&lt;/code&gt; is called during physical
planning, those hints are passed to you. By implementing capabilities like
&lt;code&gt;supports_filters_pushdown&lt;/code&gt;, you influence what the optimizer can do -- and the
metadata you declare in your &lt;code&gt;ExecutionPlan&lt;/code&gt; (partitioning, ordering) directly
affects which physical optimizations apply.&lt;/p&gt;
&lt;h2 id="choosing-the-right-starting-point"&gt;Choosing the Right Starting Point&lt;a class="headerlink" href="#choosing-the-right-starting-point" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;hr/&gt;
&lt;p&gt;Not every custom data source requires implementing all three layers from
scratch. DataFusion provides building blocks that let you plug in at whatever
level makes sense:&lt;/p&gt;
&lt;table class="table"&gt;
&lt;thead&gt;
&lt;tr&gt;
&lt;th&gt;If your data is...&lt;/th&gt;
&lt;th&gt;Start with&lt;/th&gt;
&lt;th&gt;You implement&lt;/th&gt;
&lt;/tr&gt;
&lt;/thead&gt;
&lt;tbody&gt;
&lt;tr&gt;
&lt;td&gt;Already in &lt;code&gt;RecordBatch&lt;/code&gt;es in memory&lt;/td&gt;
&lt;td&gt;&lt;a href="https://docs.rs/datafusion/latest/datafusion/datasource/memory/struct.MemTable.html"&gt;MemTable&lt;/a&gt;&lt;/td&gt;
&lt;td&gt;Nothing -- just construct it&lt;/td&gt;
&lt;/tr&gt;
&lt;tr&gt;
&lt;td&gt;An async stream of batches&lt;/td&gt;
&lt;td&gt;&lt;a href="https://docs.rs/datafusion/latest/datafusion/datasource/stream/struct.StreamTable.html"&gt;StreamTable&lt;/a&gt;&lt;/td&gt;
&lt;td&gt;A stream factory&lt;/td&gt;
&lt;/tr&gt;
&lt;tr&gt;
&lt;td&gt;A logical transformation of other tables&lt;/td&gt;
&lt;td&gt;&lt;a href="https://docs.rs/datafusion/latest/datafusion/datasource/view/struct.ViewTable.html"&gt;ViewTable&lt;/a&gt; wrapping a logical plan&lt;/td&gt;
&lt;td&gt;The logical plan&lt;/td&gt;
&lt;/tr&gt;
&lt;tr&gt;
&lt;td&gt;A variant of an existing file format&lt;/td&gt;
&lt;td&gt;&lt;a href="https://docs.rs/datafusion/latest/datafusion/datasource/listing/struct.ListingTable.html"&gt;ListingTable&lt;/a&gt; with a custom &lt;a href="https://docs.rs/datafusion/latest/datafusion/datasource/file_format/trait.FileFormat.html"&gt;FileFormat&lt;/a&gt; wrapping an existing one&lt;/td&gt;
&lt;td&gt;A thin &lt;code&gt;FileFormat&lt;/code&gt; wrapper&lt;/td&gt;
&lt;/tr&gt;
&lt;tr&gt;
&lt;td&gt;Files in a custom format on disk or object storage&lt;/td&gt;
&lt;td&gt;&lt;a href="https://docs.rs/datafusion/latest/datafusion/datasource/listing/struct.ListingTable.html"&gt;ListingTable&lt;/a&gt; with a custom &lt;a href="https://docs.rs/datafusion/latest/datafusion/datasource/file_format/trait.FileFormat.html"&gt;FileFormat&lt;/a&gt;, &lt;a href="https://docs.rs/datafusion-datasource/latest/datafusion_datasource/file/trait.FileSource.html"&gt;FileSource&lt;/a&gt;, and &lt;a href="https://docs.rs/datafusion-datasource/latest/datafusion_datasource/file_stream/trait.FileOpener.html"&gt;FileOpener&lt;/a&gt;&lt;/td&gt;
&lt;td&gt;The format, source, and opener&lt;/td&gt;
&lt;/tr&gt;
&lt;tr&gt;
&lt;td&gt;A custom source needing full control&lt;/td&gt;
&lt;td&gt;&lt;code&gt;TableProvider&lt;/code&gt; + &lt;code&gt;ExecutionPlan&lt;/code&gt; + stream&lt;/td&gt;
&lt;td&gt;All three layers&lt;/td&gt;
&lt;/tr&gt;
&lt;/tbody&gt;
&lt;/table&gt;
&lt;p&gt;If your data is file-based, &lt;code&gt;ListingTable&lt;/code&gt; handles file discovery, partition
column inference, and plan construction -- you only need to implement
&lt;code&gt;FileFormat&lt;/code&gt;, &lt;code&gt;FileSource&lt;/code&gt;, and &lt;code&gt;FileOpener&lt;/code&gt; to describe how to read your
files. See the &lt;a href="https://github.com/apache/datafusion/blob/main/datafusion-examples/examples/custom_data_source/custom_file_format.rs"&gt;custom_file_format example&lt;/a&gt; for a minimal wrapping approach,
or &lt;a href="https://docs.rs/datafusion/latest/datafusion/datasource/file_format/parquet/struct.ParquetSource.html"&gt;ParquetSource&lt;/a&gt; and &lt;a href="https://docs.rs/datafusion/latest/datafusion/datasource/file_format/parquet/struct.ParquetOpener.html"&gt;ParquetOpener&lt;/a&gt; for a full custom implementation to
use as a reference.&lt;/p&gt;
&lt;p&gt;The rest of this post focuses on the full &lt;code&gt;TableProvider&lt;/code&gt; + &lt;code&gt;ExecutionPlan&lt;/code&gt; +
stream path, which gives you complete control and applies to any data source.&lt;/p&gt;
&lt;h2 id="layer-1-tableprovider"&gt;Layer 1: TableProvider&lt;a class="headerlink" href="#layer-1-tableprovider" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;hr/&gt;
&lt;p&gt;A &lt;a href="https://docs.rs/datafusion/latest/datafusion/catalog/trait.TableProvider.html"&gt;TableProvider&lt;/a&gt; represents a queryable data source. For a minimal read-only
table, you need four methods:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-rust"&gt;impl TableProvider for MyTable {
    fn as_any(&amp;amp;self) -&amp;gt; &amp;amp;dyn Any { self }

    fn schema(&amp;amp;self) -&amp;gt; SchemaRef {
        Arc::clone(&amp;amp;self.schema)
    }

    fn table_type(&amp;amp;self) -&amp;gt; TableType {
        TableType::Base
    }

    async fn scan(
        &amp;amp;self,
        state: &amp;amp;dyn Session,
        projection: Option&amp;lt;&amp;amp;Vec&amp;lt;usize&amp;gt;&amp;gt;,
        filters: &amp;amp;[Expr],
        limit: Option&amp;lt;usize&amp;gt;,
    ) -&amp;gt; Result&amp;lt;Arc&amp;lt;dyn ExecutionPlan&amp;gt;&amp;gt; {
        // Build and return an ExecutionPlan -- don't do any execution work here -- keep lightweight!
        Ok(Arc::new(MyExecPlan::new(
            Arc::clone(&amp;amp;self.schema),
            projection,
            limit,
        )))
    }
}
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;The &lt;code&gt;scan&lt;/code&gt; method is the heart of &lt;code&gt;TableProvider&lt;/code&gt;. It receives three pushdown
hints from the optimizer, each reducing the amount of data your source needs
to produce:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;strong&gt;&lt;code&gt;projection&lt;/code&gt;&lt;/strong&gt; -- Which columns are needed. This reduces the &lt;strong&gt;width&lt;/strong&gt; of
  the output. If your source supports it, read only these columns rather than
  the full schema.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;&lt;code&gt;filters&lt;/code&gt;&lt;/strong&gt; -- Predicates the engine would like you to apply during the
  scan. This reduces the &lt;strong&gt;number of rows&lt;/strong&gt; by skipping data that does not
  match. Implement &lt;code&gt;supports_filters_pushdown&lt;/code&gt; to advertise which filters you
  can handle.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;&lt;code&gt;limit&lt;/code&gt;&lt;/strong&gt; -- A row count cap. This also reduces the &lt;strong&gt;number of rows&lt;/strong&gt; --
  if you can stop reading early once you have produced enough rows, this avoids
  unnecessary work.&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;You can also use the &lt;a href="https://docs.rs/datafusion/latest/datafusion/catalog/trait.TableProvider.html#method.scan_with_args"&gt;scan_with_args()&lt;/a&gt;
variant that provides additional pushdown information for other advanced use cases.&lt;/p&gt;
&lt;h3 id="keep-scan-lightweight"&gt;Keep &lt;code&gt;scan()&lt;/code&gt; Lightweight&lt;a class="headerlink" href="#keep-scan-lightweight" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;This is a critical point: &lt;strong&gt;&lt;code&gt;scan()&lt;/code&gt; runs during planning, not execution.&lt;/strong&gt; It
should return quickly. Best practice is to avoid performing I/O, network
calls, or heavy computation here. The &lt;code&gt;scan&lt;/code&gt; method's job is to &lt;em&gt;describe&lt;/em&gt; how
the data will be produced, not to produce it. All the real work belongs in the
stream (Layer 3).&lt;/p&gt;
&lt;p&gt;A common pitfall is to fetch data or open connections in &lt;code&gt;scan()&lt;/code&gt;. This blocks
the planning thread and can cause timeouts or deadlocks, especially if the query
involves multiple tables or subqueries that all need to be planned before
execution begins.&lt;/p&gt;
&lt;h3 id="existing-implementations-to-learn-from"&gt;Existing Implementations to Learn From&lt;a class="headerlink" href="#existing-implementations-to-learn-from" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;DataFusion ships several &lt;code&gt;TableProvider&lt;/code&gt; implementations that are excellent
references:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;strong&gt;&lt;a href="https://docs.rs/datafusion/latest/datafusion/datasource/memory/struct.MemTable.html"&gt;MemTable&lt;/a&gt;&lt;/strong&gt; -- Holds data in memory as &lt;code&gt;Vec&amp;lt;RecordBatch&amp;gt;&lt;/code&gt;. The simplest
  possible provider; great for tests and small datasets.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;&lt;a href="https://docs.rs/datafusion/latest/datafusion/datasource/stream/struct.StreamTable.html"&gt;StreamTable&lt;/a&gt;&lt;/strong&gt; -- Wraps a user-provided stream factory. Useful when your
  data arrives as a continuous stream (e.g., from Kafka or a socket).&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;&lt;a href="https://docs.rs/datafusion/latest/datafusion/datasource/listing/struct.ListingTable.html"&gt;ListingTable&lt;/a&gt;&lt;/strong&gt; -- The file-based data source behind DataFusion's
  built-in Parquet, CSV, and JSON support. Demonstrates sophisticated filter
  and projection pushdown, file pruning, and schema inference.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;&lt;a href="https://docs.rs/datafusion/latest/datafusion/datasource/view/struct.ViewTable.html"&gt;ViewTable&lt;/a&gt;&lt;/strong&gt; -- Wraps a logical plan, representing a SQL view. Useful
  if your provider is best expressed as a transformation of other tables.&lt;/li&gt;
&lt;/ul&gt;
&lt;h2 id="layer-2-executionplan"&gt;Layer 2: ExecutionPlan&lt;a class="headerlink" href="#layer-2-executionplan" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;hr/&gt;
&lt;p&gt;An &lt;a href="https://docs.rs/datafusion/latest/datafusion/physical_plan/trait.ExecutionPlan.html"&gt;ExecutionPlan&lt;/a&gt; is a node in the physical query plan tree. Your table
provider's &lt;code&gt;scan()&lt;/code&gt; method returns one. The required methods are:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-rust"&gt;impl ExecutionPlan for MyExecPlan {
    fn name(&amp;amp;self) -&amp;gt; &amp;amp;str { "MyExecPlan" }

    fn as_any(&amp;amp;self) -&amp;gt; &amp;amp;dyn Any { self }

    fn properties(&amp;amp;self) -&amp;gt; &amp;amp;PlanProperties {
        &amp;amp;self.properties
    }

    fn children(&amp;amp;self) -&amp;gt; Vec&amp;lt;&amp;amp;Arc&amp;lt;dyn ExecutionPlan&amp;gt;&amp;gt; {
        vec![]  // Leaf node -- no children
    }

    fn with_new_children(
        self: Arc&amp;lt;Self&amp;gt;,
        children: Vec&amp;lt;Arc&amp;lt;dyn ExecutionPlan&amp;gt;&amp;gt;,
    ) -&amp;gt; Result&amp;lt;Arc&amp;lt;dyn ExecutionPlan&amp;gt;&amp;gt; {
        assert!(children.is_empty());
        Ok(self)
    }

    fn execute(
        &amp;amp;self,
        partition: usize,
        context: Arc&amp;lt;TaskContext&amp;gt;,
    ) -&amp;gt; Result&amp;lt;SendableRecordBatchStream&amp;gt; {
        // This is where you build and return your stream
        // ...
    }
}
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;The key properties to set correctly in &lt;a href="https://docs.rs/datafusion/latest/datafusion/physical_plan/struct.PlanProperties.html"&gt;PlanProperties&lt;/a&gt; are &lt;strong&gt;output
partitioning&lt;/strong&gt; and &lt;strong&gt;output ordering&lt;/strong&gt;.&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Output partitioning&lt;/strong&gt; tells the engine how many partitions your data has,
which determines parallelism. If your source naturally partitions data (e.g.,
by file or by shard), expose that here.&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Output ordering&lt;/strong&gt; declares whether your data is naturally sorted. This
enables the optimizer to avoid inserting a &lt;code&gt;SortExec&lt;/code&gt; when a query requires
ordered data. Getting this right can be a significant performance win.&lt;/p&gt;
&lt;h3 id="partitioning-strategies"&gt;Partitioning Strategies&lt;a class="headerlink" href="#partitioning-strategies" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;Since &lt;code&gt;execute()&lt;/code&gt; is called once per partition, partitioning directly controls
the parallelism of your table scan. Each partition produces an independent
stream that DataFusion schedules as a &lt;strong&gt;task&lt;/strong&gt; on the tokio runtime. It is
important to distinguish tasks from threads: tasks are lightweight units of
async work that are multiplexed onto a thread pool. You can have many more
tasks (partitions) than physical threads -- the runtime will interleave them
efficiently as they await I/O or yield.&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Start simple: match your data's natural layout.&lt;/strong&gt; If you have 4 files, expose
4 partitions. If your source has 8 shards, expose 8 partitions. DataFusion will
insert a &lt;code&gt;RepartitionExec&lt;/code&gt; above your scan when downstream operators need a
different distribution. You can also implement the
&lt;a href="https://docs.rs/datafusion/latest/datafusion/physical_plan/trait.ExecutionPlan.html#method.repartitioned"&gt;&lt;code&gt;repartitioned&lt;/code&gt;&lt;/a&gt;
method on your &lt;code&gt;ExecutionPlan&lt;/code&gt; to let DataFusion request a different partition
count directly from your source, avoiding the extra operator entirely.&lt;/p&gt;
&lt;p&gt;Consider how your data source naturally divides its data:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;strong&gt;By file or object:&lt;/strong&gt; If you are reading from S3, each file can be a
  partition. DataFusion will read them in parallel.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;By shard or region:&lt;/strong&gt; If your source is a sharded database, each shard
  maps naturally to a partition.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;By key range:&lt;/strong&gt; If your data is keyed (e.g., by timestamp or customer ID),
  you can split it into ranges.&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;&lt;strong&gt;Advanced: aligning with &lt;code&gt;target_partitions&lt;/code&gt;.&lt;/strong&gt; Once you have something
working, you can tune further. Having &lt;em&gt;too many&lt;/em&gt; partitions is not free: each
partition adds scheduling overhead, and downstream operators may need to
repartition the data anyway. The session configuration exposes a
&lt;strong&gt;target partition count&lt;/strong&gt; that reflects how many partitions the optimizer
expects to work with:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-rust"&gt;async fn scan(
    &amp;amp;self,
    state: &amp;amp;dyn Session,
    projection: Option&amp;lt;&amp;amp;Vec&amp;lt;usize&amp;gt;&amp;gt;,
    filters: &amp;amp;[Expr],
    limit: Option&amp;lt;usize&amp;gt;,
) -&amp;gt; Result&amp;lt;Arc&amp;lt;dyn ExecutionPlan&amp;gt;&amp;gt; {
    let target_partitions = state.config().target_partitions();
    // Optionally coalesce or split partitions to match target_partitions.
    // ...
}
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;If your source produces data in exactly &lt;code&gt;target_partitions&lt;/code&gt; partitions, the
optimizer is less likely to insert a &lt;code&gt;RepartitionExec&lt;/code&gt; above your scan.
For small datasets, &lt;code&gt;target_partitions&lt;/code&gt; may be set to 1, which avoids any
repartitioning overhead entirely.&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Advanced: declaring hash partitioning.&lt;/strong&gt; If your source stores data
pre-partitioned by a specific key (e.g., &lt;code&gt;customer_id&lt;/code&gt;), you can declare this
in your output partitioning. For a query like:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-sql"&gt;SELECT customer_id, SUM(amount)
FROM my_table
GROUP BY customer_id;
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;If you declare your output partitioning as &lt;code&gt;Hash([customer_id], N)&lt;/code&gt;, the
optimizer recognizes that the data is already distributed correctly for the
aggregation and eliminates the &lt;code&gt;RepartitionExec&lt;/code&gt; that would otherwise appear
in the plan. You can verify this with &lt;code&gt;EXPLAIN&lt;/code&gt; (more on this below).&lt;/p&gt;
&lt;p&gt;Conversely, if you report &lt;code&gt;UnknownPartitioning&lt;/code&gt;, DataFusion must assume the
worst case and will always insert repartitioning operators as needed.&lt;/p&gt;
&lt;h3 id="keep-execute-lightweight-too"&gt;Keep &lt;code&gt;execute()&lt;/code&gt; Lightweight Too&lt;a class="headerlink" href="#keep-execute-lightweight-too" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;Like &lt;code&gt;scan()&lt;/code&gt;, the &lt;code&gt;execute()&lt;/code&gt; method should construct and return a stream
without doing heavy work. The actual data production happens when the stream
is polled. Do not block on async operations here -- build the stream and let
the runtime drive it.&lt;/p&gt;
&lt;h3 id="existing-implementations-to-learn-from_1"&gt;Existing Implementations to Learn From&lt;a class="headerlink" href="#existing-implementations-to-learn-from_1" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;ul&gt;
&lt;li&gt;&lt;strong&gt;&lt;a href="https://docs.rs/datafusion/latest/datafusion/datasource/stream/struct.StreamingTableExec.html"&gt;StreamingTableExec&lt;/a&gt;&lt;/strong&gt; -- Executes a streaming table scan. It takes a
  stream factory (a closure that produces streams) and handles partitioning.
  Good reference for wrapping external streams.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;&lt;a href="https://docs.rs/datafusion/latest/datafusion/datasource/struct.DataSourceExec.html"&gt;DataSourceExec&lt;/a&gt;&lt;/strong&gt; -- The execution plan behind DataFusion's built-in file
  scanning (Parquet, CSV, JSON). It demonstrates sophisticated partitioning,
  filter pushdown, and projection pushdown.&lt;/li&gt;
&lt;/ul&gt;
&lt;h2 id="layer-3-sendablerecordbatchstream"&gt;Layer 3: SendableRecordBatchStream&lt;a class="headerlink" href="#layer-3-sendablerecordbatchstream" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;hr/&gt;
&lt;p&gt;&lt;a href="https://docs.rs/datafusion/latest/datafusion/execution/type.SendableRecordBatchStream.html"&gt;SendableRecordBatchStream&lt;/a&gt; is where the real work happens. It is defined as:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-rust"&gt;type SendableRecordBatchStream =
    Pin&amp;lt;Box&amp;lt;dyn RecordBatchStream&amp;lt;Item = Result&amp;lt;RecordBatch&amp;gt;&amp;gt; + Send&amp;gt;&amp;gt;;
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;This is an async stream of &lt;code&gt;RecordBatch&lt;/code&gt;es that can be sent across threads. When
the DataFusion runtime polls this stream, your code runs: reading files, calling
APIs, transforming data, etc.&lt;/p&gt;
&lt;h3 id="using-recordbatchstreamadapter"&gt;Using RecordBatchStreamAdapter&lt;a class="headerlink" href="#using-recordbatchstreamadapter" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;The easiest way to create a &lt;code&gt;SendableRecordBatchStream&lt;/code&gt; is with
&lt;a href="https://docs.rs/datafusion/latest/datafusion/physical_plan/stream/struct.RecordBatchStreamAdapter.html"&gt;RecordBatchStreamAdapter&lt;/a&gt;. It bridges any &lt;code&gt;futures::Stream&amp;lt;Item =
Result&amp;lt;RecordBatch&amp;gt;&amp;gt;&lt;/code&gt; into the &lt;code&gt;SendableRecordBatchStream&lt;/code&gt; type:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-rust"&gt;use datafusion::physical_plan::stream::RecordBatchStreamAdapter;

fn execute(
    &amp;amp;self,
    partition: usize,
    context: Arc&amp;lt;TaskContext&amp;gt;,
) -&amp;gt; Result&amp;lt;SendableRecordBatchStream&amp;gt; {
    let schema = self.schema();
    let config = self.config.clone();

    let stream = futures::stream::once(async move {
        // ALL the heavy work happens here, inside the stream:
        // - Open connections
        // - Read data from external sources
        // - Transform and batch the results
        let batches = fetch_data_from_source(&amp;amp;config).await?;
        Ok(batches)
    })
    .flat_map(|result| match result {
        Ok(batch) =&amp;gt; futures::stream::iter(vec![Ok(batch)]),
        Err(e) =&amp;gt; futures::stream::iter(vec![Err(e)]),
    });

    Ok(Box::pin(RecordBatchStreamAdapter::new(schema, stream)))
}
&lt;/code&gt;&lt;/pre&gt;
&lt;h3 id="blocking-work-use-a-separate-thread-pool"&gt;Blocking Work: Use a Separate Thread Pool&lt;a class="headerlink" href="#blocking-work-use-a-separate-thread-pool" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;If your stream performs &lt;strong&gt;blocking&lt;/strong&gt; work -- such as blocking I/O, or CPU work
that runs for hundreds of milliseconds without yielding -- you must avoid
blocking the tokio async runtime. Short CPU work (e.g., parsing a batch in a
few milliseconds) is fine to do inline as long as your code yields back to the
runtime frequently. But for long-running synchronous work that cannot yield,
offload to a dedicated thread pool and send results back through a channel:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-rust"&gt;fn execute(
    &amp;amp;self,
    partition: usize,
    context: Arc&amp;lt;TaskContext&amp;gt;,
) -&amp;gt; Result&amp;lt;SendableRecordBatchStream&amp;gt; {
    let schema = self.schema();
    let config = self.config.clone();

    let (tx, rx) = tokio::sync::mpsc::channel(2);

    // Spawn blocking work on a dedicated thread pool
    tokio::task::spawn_blocking(move || {
        let batches = generate_data(&amp;amp;config);
        for batch in batches {
            if tx.blocking_send(Ok(batch)).is_err() {
                break; // Receiver dropped, query was cancelled
            }
        }
    });

    let stream = tokio_stream::wrappers::ReceiverStream::new(rx);
    Ok(Box::pin(RecordBatchStreamAdapter::new(schema, stream)))
}
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;This pattern keeps the async runtime responsive while long-running synchronous
work runs on its own threads. For a working example that shows how to configure
separate thread pools for I/O and CPU work, see the
&lt;a href="https://github.com/apache/datafusion/blob/main/datafusion-examples/examples/query_planning/thread_pools.rs"&gt;thread_pools example&lt;/a&gt;
in the DataFusion repository.&lt;/p&gt;
&lt;h2 id="where-should-the-work-happen"&gt;Where Should the Work Happen?&lt;a class="headerlink" href="#where-should-the-work-happen" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;hr/&gt;
&lt;p&gt;This table summarizes what belongs at each layer:&lt;/p&gt;
&lt;table class="table"&gt;
&lt;thead&gt;
&lt;tr&gt;
&lt;th&gt;Layer&lt;/th&gt;
&lt;th&gt;Runs During&lt;/th&gt;
&lt;th&gt;Should Do&lt;/th&gt;
&lt;th&gt;Should NOT Do&lt;/th&gt;
&lt;/tr&gt;
&lt;/thead&gt;
&lt;tbody&gt;
&lt;tr&gt;
&lt;td&gt;&lt;code&gt;TableProvider::scan()&lt;/code&gt;&lt;/td&gt;
&lt;td&gt;Planning&lt;/td&gt;
&lt;td&gt;Build an &lt;code&gt;ExecutionPlan&lt;/code&gt; with metadata&lt;/td&gt;
&lt;td&gt;I/O, network calls, heavy computation&lt;/td&gt;
&lt;/tr&gt;
&lt;tr&gt;
&lt;td&gt;&lt;code&gt;ExecutionPlan::execute()&lt;/code&gt;&lt;/td&gt;
&lt;td&gt;Execution (once per partition)&lt;/td&gt;
&lt;td&gt;Construct a stream, set up channels&lt;/td&gt;
&lt;td&gt;Block on async work, read data&lt;/td&gt;
&lt;/tr&gt;
&lt;tr&gt;
&lt;td&gt;&lt;code&gt;RecordBatchStream&lt;/code&gt; (polling)&lt;/td&gt;
&lt;td&gt;Execution&lt;/td&gt;
&lt;td&gt;All I/O, computation, data production&lt;/td&gt;
&lt;td&gt;--&lt;/td&gt;
&lt;/tr&gt;
&lt;/tbody&gt;
&lt;/table&gt;
&lt;p&gt;The guiding principle: &lt;strong&gt;push work as late as possible.&lt;/strong&gt; Planning should be
fast so the optimizer can do its job. Execution setup should be fast so all
partitions can start promptly. The stream is where you spend time producing
data.&lt;/p&gt;
&lt;h3 id="why-this-matters"&gt;Why This Matters&lt;a class="headerlink" href="#why-this-matters" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;When &lt;code&gt;scan()&lt;/code&gt; does heavy work, several problems arise:&lt;/p&gt;
&lt;ol&gt;
&lt;li&gt;&lt;strong&gt;Planning becomes slow.&lt;/strong&gt; If a query touches 10 tables and each &lt;code&gt;scan()&lt;/code&gt;
   takes 500ms, planning alone takes 5 seconds before any data flows.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Execution is single-threaded.&lt;/strong&gt; &lt;code&gt;scan()&lt;/code&gt; runs on a single thread during
   planning, so any work done there cannot benefit from the parallel execution
   that DataFusion provides across partitions.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;The optimizer cannot help.&lt;/strong&gt; The optimizer runs between planning and
   execution. If you have already fetched data during planning, optimizations
   like predicate pushdown or partition pruning cannot reduce the work.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Resource management breaks down.&lt;/strong&gt; DataFusion manages concurrency and
   memory during execution. Work done during planning bypasses these controls.&lt;/li&gt;
&lt;/ol&gt;
&lt;h2 id="filter-pushdown-doing-less-work"&gt;Filter Pushdown: Doing Less Work&lt;a class="headerlink" href="#filter-pushdown-doing-less-work" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;hr/&gt;
&lt;p&gt;One of the most impactful optimizations you can add to a custom table provider
is &lt;strong&gt;filter pushdown&lt;/strong&gt; -- letting the source skip data that the query does not
need, rather than reading everything and filtering it afterward.&lt;/p&gt;
&lt;h3 id="how-filter-pushdown-works"&gt;How Filter Pushdown Works&lt;a class="headerlink" href="#how-filter-pushdown-works" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;When DataFusion plans a query with a &lt;code&gt;WHERE&lt;/code&gt; clause, it passes the filter
predicates to your &lt;code&gt;scan()&lt;/code&gt; method as the &lt;code&gt;filters&lt;/code&gt; parameter. By default,
DataFusion assumes your provider cannot handle any filters and inserts a
&lt;code&gt;FilterExec&lt;/code&gt; node above your scan to apply them. But if your source &lt;em&gt;can&lt;/em&gt;
evaluate some predicates during scanning -- for example, by skipping files,
partitions, or row groups that cannot match -- you can eliminate a huge amount
of unnecessary I/O.&lt;/p&gt;
&lt;p&gt;To opt in, implement &lt;code&gt;supports_filters_pushdown&lt;/code&gt;:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-rust"&gt;fn supports_filters_pushdown(
    &amp;amp;self,
    filters: &amp;amp;[&amp;amp;Expr],
) -&amp;gt; Result&amp;lt;Vec&amp;lt;TableProviderFilterPushDown&amp;gt;&amp;gt; {
    Ok(filters.iter().map(|f| {
        match f {
            // We can fully evaluate equality filters on
            // the partition column at the source
            Expr::BinaryExpr(BinaryExpr {
                left, op: Operator::Eq, right
            }) if is_partition_column(left) || is_partition_column(right) =&amp;gt; {
                TableProviderFilterPushDown::Exact
            }
            // All other filters: let DataFusion handle them
            _ =&amp;gt; TableProviderFilterPushDown::Unsupported,
        }
    }).collect())
}
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;The three possible responses for each filter are:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;strong&gt;&lt;code&gt;Exact&lt;/code&gt;&lt;/strong&gt; -- Your source guarantees that no output rows will have a false
  value for this predicate. Because the filter is fully evaluated at the source,
  DataFusion will &lt;strong&gt;not&lt;/strong&gt; add a &lt;code&gt;FilterExec&lt;/code&gt; for it.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;&lt;code&gt;Inexact&lt;/code&gt;&lt;/strong&gt; -- Your source has the ability to reduce the data produced, but
  the output may still include rows that do not satisfy the predicate. For
  example, you might skip entire files based on metadata statistics but not
  filter individual rows within a file. DataFusion will still add a &lt;code&gt;FilterExec&lt;/code&gt;
  above your scan to remove any remaining rows that slipped through.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;&lt;code&gt;Unsupported&lt;/code&gt;&lt;/strong&gt; -- Your source ignores this filter entirely. DataFusion
  handles it.&lt;/li&gt;
&lt;/ul&gt;
&lt;h3 id="why-filter-pushdown-matters"&gt;Why Filter Pushdown Matters&lt;a class="headerlink" href="#why-filter-pushdown-matters" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;Consider a table with 1 billion rows partitioned by &lt;code&gt;region&lt;/code&gt;, and a query:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-sql"&gt;SELECT * FROM events WHERE region = 'us-east-1' AND event_type = 'click';
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;&lt;strong&gt;Without filter pushdown:&lt;/strong&gt; Your table provider reads all 1 billion rows
across all regions. DataFusion then applies both filters, discarding the vast
majority of the data.&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;With filter pushdown on &lt;code&gt;region&lt;/code&gt;:&lt;/strong&gt; Your &lt;code&gt;scan()&lt;/code&gt; method sees the
&lt;code&gt;region = 'us-east-1'&lt;/code&gt; filter and constructs an execution plan that only reads
the &lt;code&gt;us-east-1&lt;/code&gt; partition. If that partition holds 100 million rows, you have
just eliminated 90% of the I/O. DataFusion still applies the &lt;code&gt;event_type&lt;/code&gt;
filter via &lt;code&gt;FilterExec&lt;/code&gt; if you reported it as &lt;code&gt;Unsupported&lt;/code&gt;.&lt;/p&gt;
&lt;h3 id="only-push-down-filters-when-the-data-source-can-do-better"&gt;Only Push Down Filters When the Data Source Can Do Better&lt;a class="headerlink" href="#only-push-down-filters-when-the-data-source-can-do-better" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;DataFusion already pushes filters as close to the data source as possible, typically placing them directly above the scan. &lt;code&gt;FilterExec&lt;/code&gt; is also highly optimized, with vectorized evaluation and type-specialized kernels for fast predicate evaluation.&lt;/p&gt;
&lt;p&gt;Because of this, you should only implement filter pushdown when your data source
can do strictly better -- for example, by avoiding I/O entirely through
skipping files or partitions based on metadata. If your data source cannot
eliminate I/O in this way, it is usually better to let DataFusion handle the
filter, as its in-memory execution is already highly efficient.&lt;/p&gt;
&lt;h3 id="using-explain-to-debug-your-table-provider"&gt;Using EXPLAIN to Debug Your Table Provider&lt;a class="headerlink" href="#using-explain-to-debug-your-table-provider" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;The &lt;code&gt;EXPLAIN&lt;/code&gt; statement is your best tool for understanding what DataFusion is
actually doing with your table provider. It shows the physical plan that
DataFusion will execute, including any operators it inserted:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-sql"&gt;EXPLAIN SELECT * FROM events WHERE region = 'us-east-1' AND event_type = 'click';
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;If you are using DataFrames, call &lt;code&gt;.explain(false, false)&lt;/code&gt; for the logical plan
or &lt;code&gt;.explain(false, true)&lt;/code&gt; for the physical plan. You can also print the plans
in verbose mode with &lt;code&gt;.explain(true, true)&lt;/code&gt;.&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Before filter pushdown&lt;/strong&gt;, the plan might look like:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-text"&gt;FilterExec: region@0 = us-east-1 AND event_type@1 = click
  MyExecPlan: partitions=50
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;Here DataFusion is reading all 50 partitions and filtering everything
afterward. The &lt;code&gt;FilterExec&lt;/code&gt; above your scan is doing all the predicate work.&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;After implementing pushdown for &lt;code&gt;region&lt;/code&gt;&lt;/strong&gt; (reported as &lt;code&gt;Exact&lt;/code&gt;):&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-text"&gt;FilterExec: event_type@1 = click
  MyExecPlan: partitions=5, filter=[region = us-east-1]
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;Now your exec reads only the 5 partitions for &lt;code&gt;us-east-1&lt;/code&gt;, and the remaining
&lt;code&gt;FilterExec&lt;/code&gt; only handles the &lt;code&gt;event_type&lt;/code&gt; predicate. The &lt;code&gt;region&lt;/code&gt; filter has
been fully absorbed by your scan.&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;After implementing pushdown for both filters&lt;/strong&gt; (both &lt;code&gt;Exact&lt;/code&gt;):&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-text"&gt;MyExecPlan: partitions=5, filter=[region = us-east-1 AND event_type = click]
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;No &lt;code&gt;FilterExec&lt;/code&gt; at all -- your source handles everything.&lt;/p&gt;
&lt;p&gt;Similarly, &lt;code&gt;EXPLAIN&lt;/code&gt; will reveal whether DataFusion is inserting unnecessary
&lt;code&gt;SortExec&lt;/code&gt; or &lt;code&gt;RepartitionExec&lt;/code&gt; nodes that you could eliminate by declaring
better output properties. Whenever your queries seem slower than expected,
&lt;code&gt;EXPLAIN&lt;/code&gt; is the first place to look.&lt;/p&gt;
&lt;h3 id="a-complete-filter-pushdown-example"&gt;A Complete Filter Pushdown Example&lt;a class="headerlink" href="#a-complete-filter-pushdown-example" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;To make filter pushdown concrete, here is an illustrative example. Imagine a
table provider that reads from a set of date-partitioned directories on disk
(e.g., &lt;code&gt;data/2026-03-01/&lt;/code&gt;, &lt;code&gt;data/2026-03-02/&lt;/code&gt;, ...). Each directory contains
one or more Parquet files for that date. By pushing down a filter on the &lt;code&gt;date&lt;/code&gt;
column, the provider can skip entire directories -- avoiding the I/O of listing
and reading files that cannot possibly match the query.&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-rust"&gt;/// A table provider backed by date-partitioned directories.
/// Each date directory contains data files; by filtering on the
/// `date` column we can skip entire directories of I/O.
struct DatePartitionedTable {
    schema: SchemaRef,
    /// Maps date strings ("2026-03-01") to directory paths
    partitions: HashMap&amp;lt;String, String&amp;gt;,
}

#[async_trait::async_trait]
impl TableProvider for DatePartitionedTable {
    fn as_any(&amp;amp;self) -&amp;gt; &amp;amp;dyn Any { self }
    fn schema(&amp;amp;self) -&amp;gt; SchemaRef { Arc::clone(&amp;amp;self.schema) }
    fn table_type(&amp;amp;self) -&amp;gt; TableType { TableType::Base }

    fn supports_filters_pushdown(
        &amp;amp;self,
        filters: &amp;amp;[&amp;amp;Expr],
    ) -&amp;gt; Result&amp;lt;Vec&amp;lt;TableProviderFilterPushDown&amp;gt;&amp;gt; {
        Ok(filters.iter().map(|f| {
            if Self::is_date_equality_filter(f) {
                // We can fully evaluate this: we will only read
                // directories matching the date, so no rows with
                // a different date will appear in the output.
                TableProviderFilterPushDown::Exact
            } else {
                TableProviderFilterPushDown::Unsupported
            }
        }).collect())
    }

    async fn scan(
        &amp;amp;self,
        _state: &amp;amp;dyn Session,
        projection: Option&amp;lt;&amp;amp;Vec&amp;lt;usize&amp;gt;&amp;gt;,
        filters: &amp;amp;[Expr],
        limit: Option&amp;lt;usize&amp;gt;,
    ) -&amp;gt; Result&amp;lt;Arc&amp;lt;dyn ExecutionPlan&amp;gt;&amp;gt; {
        // Determine which date partitions to read by inspecting
        // the pushed-down filters. This is the key optimization:
        // we decide *during planning* which directories to scan,
        // so that execution never touches irrelevant data.
        let dates_to_read: Vec&amp;lt;String&amp;gt; = self
            .extract_date_values(filters)
            .unwrap_or_else(||
                self.partitions.keys().cloned().collect()
            );

        let dirs: Vec&amp;lt;String&amp;gt; = dates_to_read
            .iter()
            .filter_map(|d| self.partitions.get(d).cloned())
            .collect();
        let num_dirs = dirs.len();

        Ok(Arc::new(DatePartitionedExec {
            schema: Arc::clone(&amp;amp;self.schema),
            directories: dirs,
            properties: PlanProperties::new(
                EquivalenceProperties::new(
                    Arc::clone(&amp;amp;self.schema),
                ),
                // One partition per date directory -- these
                // will be read in parallel.
                Partitioning::UnknownPartitioning(num_dirs),
                EmissionType::Incremental,
                Boundedness::Bounded,
            ),
        }))
    }
}

impl DatePartitionedTable {
    /// Check if a filter is an equality comparison on the `date` column.
    fn is_date_equality_filter(expr: &amp;amp;Expr) -&amp;gt; bool {
        // In practice, match on BinaryExpr { left, op: Eq, right }
        // and check if either side references the "date" column.
        // Simplified here for clarity.
        todo!("match on date equality expressions")
    }

    /// Extract date literal values from pushed-down equality filters.
    fn extract_date_values(&amp;amp;self, filters: &amp;amp;[Expr]) -&amp;gt; Option&amp;lt;Vec&amp;lt;String&amp;gt;&amp;gt; {
        // Parse filters like `date = '2026-03-01'` and return
        // the literal date strings. Returns None if no date
        // filters are present (meaning: read all partitions).
        todo!("extract date literals from filter expressions")
    }
}
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;The key insight is that the filter pushdown decision (&lt;code&gt;supports_filters_pushdown&lt;/code&gt;)
and the partition pruning (&lt;code&gt;scan()&lt;/code&gt;) work together: the first tells DataFusion
that a &lt;code&gt;FilterExec&lt;/code&gt; is unnecessary for the &lt;code&gt;date&lt;/code&gt; predicate, and the second
ensures that only the relevant directories are scanned. The actual file reading
happens later, in the stream produced by &lt;code&gt;execute()&lt;/code&gt;.&lt;/p&gt;
&lt;h2 id="putting-it-all-together"&gt;Putting It All Together&lt;a class="headerlink" href="#putting-it-all-together" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;hr/&gt;
&lt;p&gt;Here is a minimal but complete example of a custom table provider that generates
data lazily during streaming:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-rust"&gt;use std::any::Any;
use std::sync::Arc;

use arrow::array::Int64Array;
use arrow::datatypes::{DataType, Field, Schema, SchemaRef};
use arrow::record_batch::RecordBatch;
use datafusion::catalog::TableProvider;
use datafusion::common::Result;
use datafusion::datasource::TableType;
use datafusion::catalog::Session;
use datafusion::execution::SendableRecordBatchStream;
use datafusion::logical_expr::Expr;
use datafusion::physical_expr::EquivalenceProperties;
use datafusion::physical_plan::execution_plan::{Boundedness, EmissionType};
use datafusion::physical_plan::stream::RecordBatchStreamAdapter;
use datafusion::physical_plan::{
    ExecutionPlan, Partitioning, PlanProperties,
};
use futures::stream;

/// A table provider that generates sequential numbers on demand.
struct CountingTable {
    schema: SchemaRef,
    num_partitions: usize,
    rows_per_partition: usize,
}

impl CountingTable {
    fn new(num_partitions: usize, rows_per_partition: usize) -&amp;gt; Self {
        let schema = Arc::new(Schema::new(vec![
            Field::new("partition", DataType::Int64, false),
            Field::new("value", DataType::Int64, false),
        ]));
        Self { schema, num_partitions, rows_per_partition }
    }
}

#[async_trait::async_trait]
impl TableProvider for CountingTable {
    fn as_any(&amp;amp;self) -&amp;gt; &amp;amp;dyn Any { self }
    fn schema(&amp;amp;self) -&amp;gt; SchemaRef { Arc::clone(&amp;amp;self.schema) }
    fn table_type(&amp;amp;self) -&amp;gt; TableType { TableType::Base }

    async fn scan(
        &amp;amp;self,
        _state: &amp;amp;dyn Session,
        projection: Option&amp;lt;&amp;amp;Vec&amp;lt;usize&amp;gt;&amp;gt;,
        _filters: &amp;amp;[Expr],
        limit: Option&amp;lt;usize&amp;gt;,
    ) -&amp;gt; Result&amp;lt;Arc&amp;lt;dyn ExecutionPlan&amp;gt;&amp;gt; {
        // Light work only: build the plan with metadata
        Ok(Arc::new(CountingExec {
            schema: Arc::clone(&amp;amp;self.schema),
            num_partitions: self.num_partitions,
            rows_per_partition: limit
                .unwrap_or(self.rows_per_partition)
                .min(self.rows_per_partition),
            properties: PlanProperties::new(
                EquivalenceProperties::new(Arc::clone(&amp;amp;self.schema)),
                Partitioning::UnknownPartitioning(self.num_partitions),
                EmissionType::Incremental,
                Boundedness::Bounded,
            ),
        }))
    }
}

struct CountingExec {
    schema: SchemaRef,
    num_partitions: usize,
    rows_per_partition: usize,
    properties: PlanProperties,
}

impl ExecutionPlan for CountingExec {
    fn name(&amp;amp;self) -&amp;gt; &amp;amp;str { "CountingExec" }
    fn as_any(&amp;amp;self) -&amp;gt; &amp;amp;dyn Any { self }
    fn properties(&amp;amp;self) -&amp;gt; &amp;amp;PlanProperties { &amp;amp;self.properties }
    fn children(&amp;amp;self) -&amp;gt; Vec&amp;lt;&amp;amp;Arc&amp;lt;dyn ExecutionPlan&amp;gt;&amp;gt; { vec![] }

    fn with_new_children(
        self: Arc&amp;lt;Self&amp;gt;,
        _children: Vec&amp;lt;Arc&amp;lt;dyn ExecutionPlan&amp;gt;&amp;gt;,
    ) -&amp;gt; Result&amp;lt;Arc&amp;lt;dyn ExecutionPlan&amp;gt;&amp;gt; {
        Ok(self)
    }

    fn execute(
        &amp;amp;self,
        partition: usize,
        _context: Arc&amp;lt;TaskContext&amp;gt;,
    ) -&amp;gt; Result&amp;lt;SendableRecordBatchStream&amp;gt; {
        let schema = Arc::clone(&amp;amp;self.schema);
        let rows = self.rows_per_partition;

        // The heavy work (data generation) happens inside the stream,
        // not here in execute().
        let batch_stream = stream::once(async move {
            let partitions = Int64Array::from(
                vec![partition as i64; rows],
            );
            let values = Int64Array::from(
                (0..rows as i64).collect::&amp;lt;Vec&amp;lt;_&amp;gt;&amp;gt;(),
            );
            let batch = RecordBatch::try_new(
                Arc::clone(&amp;amp;schema),
                vec![Arc::new(partitions), Arc::new(values)],
            )?;
            Ok(batch)
        });

        Ok(Box::pin(RecordBatchStreamAdapter::new(
            Arc::clone(&amp;amp;self.schema),
            batch_stream,
        )))
    }
}
&lt;/code&gt;&lt;/pre&gt;
&lt;h2 id="acknowledgements"&gt;Acknowledgements&lt;a class="headerlink" href="#acknowledgements" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;I would like to thank &lt;a href="https://rerun.io"&gt;Rerun.io&lt;/a&gt; for sponsoring the development of this work. Rerun is
building a data visualization system for Physical AI and makes heavy use of DataFusion
table providers for working with data analytics.&lt;/p&gt;
&lt;p&gt;I would also like to thank the reviewers of this post for their helpful feedback and
suggestions: &lt;a href="https://github.com/2010YOUY01"&gt;@2010YOUY01&lt;/a&gt;, &lt;a href="https://github.com/adriangb"&gt;@adriangb&lt;/a&gt;, &lt;a href="https://github.com/alamb"&gt;@alamb&lt;/a&gt;, &lt;a href="https://github.com/kevinjqliu"&gt;@kevinjqliu&lt;/a&gt;, &lt;a href="https://github.com/Omega359"&gt;@Omega359&lt;/a&gt;,
&lt;a href="https://github.com/pgwhalen"&gt;@pgwhalen&lt;/a&gt;, and &lt;a href="https://github.com/stuhood"&gt;@stuhood&lt;/a&gt;.&lt;/p&gt;
&lt;h2 id="get-involved"&gt;Get Involved&lt;a class="headerlink" href="#get-involved" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;DataFusion is not a project built or driven by a single person, company, or
foundation. Our community of users and contributors works together to build a
shared technology that none of us could have built alone.&lt;/p&gt;
&lt;p&gt;If you are interested in joining us, we would love to have you. You can try out
DataFusion on some of your own data and projects and let us know how it goes,
contribute suggestions, documentation, bug reports, or a PR with documentation,
tests, or code. A list of open issues suitable for beginners is &lt;a href="https://github.com/apache/datafusion/issues?q=is%3Aissue+is%3Aopen+label%3A%22good+first+issue%22"&gt;here&lt;/a&gt;, and you
can find out how to reach us on the &lt;a href="https://datafusion.apache.org/contributor-guide/communication.html"&gt;communication doc&lt;/a&gt;.&lt;/p&gt;
&lt;h2 id="further-reading"&gt;Further Reading&lt;a class="headerlink" href="#further-reading" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;hr/&gt;
&lt;ul&gt;
&lt;li&gt;&lt;a href="https://docs.rs/datafusion/latest/datafusion/catalog/trait.TableProvider.html"&gt;&lt;code&gt;TableProvider&lt;/code&gt; API docs&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;&lt;a href="https://docs.rs/datafusion/latest/datafusion/physical_plan/trait.ExecutionPlan.html"&gt;&lt;code&gt;ExecutionPlan&lt;/code&gt; API docs&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;&lt;a href="https://docs.rs/datafusion/latest/datafusion/execution/type.SendableRecordBatchStream.html"&gt;&lt;code&gt;SendableRecordBatchStream&lt;/code&gt; API docs&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;&lt;a href="https://github.com/apache/datafusion/tree/main/datafusion-examples/examples"&gt;DataFusion examples directory&lt;/a&gt; --
  contains working examples including custom table providers&lt;/li&gt;
&lt;/ul&gt;
&lt;hr/&gt;
&lt;p&gt;&lt;em&gt;Note: Portions of this blog post were written with the assistance of an AI agent.&lt;/em&gt;&lt;/p&gt;</content><category term="blog"/></entry><entry><title>Turning LIMIT into an I/O Optimization: Inside DataFusion’s Multi-Layer Pruning Stack</title><link href="https://datafusion.apache.org/blog/2026/03/20/limit-pruning" rel="alternate"/><published>2026-03-20T00:00:00+00:00</published><updated>2026-03-20T00:00:00+00:00</updated><author><name>xudong</name></author><id>tag:datafusion.apache.org,2026-03-20:/blog/2026/03/20/limit-pruning</id><summary type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;style&gt;
figure {
  margin: 20px 0;
}

figure img {
  display: block;
  max-width: 80%;
  margin: auto;
}

figcaption {
  font-style: italic;
  color: #555;
  font-size: 0.9em;
  max-width: 80%;
  margin: auto;
  text-align: center;
}
&lt;/style&gt;
&lt;p&gt;&lt;em&gt;Xudong Wang, &lt;a href="https://www.massive.com/"&gt;Massive&lt;/a&gt;&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;Reading data efficiently means touching as little data as possible. The fastest I/O is the I/O you never make …&lt;/p&gt;</summary><content type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;style&gt;
figure {
  margin: 20px 0;
}

figure img {
  display: block;
  max-width: 80%;
  margin: auto;
}

figcaption {
  font-style: italic;
  color: #555;
  font-size: 0.9em;
  max-width: 80%;
  margin: auto;
  text-align: center;
}
&lt;/style&gt;
&lt;p&gt;&lt;em&gt;Xudong Wang, &lt;a href="https://www.massive.com/"&gt;Massive&lt;/a&gt;&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;Reading data efficiently means touching as little data as possible. The fastest I/O is the I/O you never make. This sounds obvious, but making it happen in practice requires careful engineering at every layer of the query engine. &lt;a href="https://datafusion.apache.org/"&gt;Apache DataFusion&lt;/a&gt; achieves this through a multi-layer &lt;strong&gt;pruning pipeline&lt;/strong&gt; — a series of stages that progressively narrow down the data before decoding a single row.&lt;/p&gt;
&lt;p&gt;In this post, we describe a new optimization called &lt;strong&gt;limit pruning&lt;/strong&gt; that makes this pipeline aware of SQL &lt;code&gt;LIMIT&lt;/code&gt; clauses. By identifying row groups where &lt;em&gt;every&lt;/em&gt; row is guaranteed to match the predicate, DataFusion can satisfy a &lt;code&gt;LIMIT&lt;/code&gt; query without ever touching partially matching row groups — eliminating wasted I/O entirely.&lt;/p&gt;
&lt;p&gt;For example, given a query like:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-sql"&gt;SELECT * FROM tracking_data
WHERE species LIKE 'Alpine%' AND s &amp;gt;= 50
LIMIT 3
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;If the pruning pipeline already knows that certain row groups fully satisfy the &lt;code&gt;WHERE&lt;/code&gt; clause, those groups alone may contain enough rows to fill the &lt;code&gt;LIMIT&lt;/code&gt; — making it unnecessary to scan anything else.&lt;/p&gt;
&lt;p&gt;This work was inspired by the "Pruning for LIMIT Queries" section of Snowflake's paper &lt;a href="https://arxiv.org/pdf/2504.11540"&gt;&lt;em&gt;Pruning in Snowflake: Working Smarter, Not Harder&lt;/em&gt;&lt;/a&gt;.&lt;/p&gt;
&lt;h2 id="datafusions-pruning-pipeline"&gt;DataFusion's Pruning Pipeline&lt;a class="headerlink" href="#datafusions-pruning-pipeline" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;Before diving into limit pruning, let's understand the full pruning pipeline. DataFusion scans Parquet data through a series of increasingly fine-grained filters, each one eliminating data so the next stage processes less:&lt;/p&gt;
&lt;figure&gt;
&lt;img alt="Three phases of DataFusion's pruning pipeline" src="/blog/images/limit-pruning/pruning-phases.svg" width="80%"/&gt;
&lt;figcaption&gt;Figure 1: The three phases of DataFusion's pruning pipeline — from directories down to individual rows.&lt;/figcaption&gt;
&lt;/figure&gt;
&lt;h3 id="phase-1-high-level-discovery"&gt;Phase 1: High-Level Discovery&lt;a class="headerlink" href="#phase-1-high-level-discovery" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;ul&gt;
&lt;li&gt;&lt;strong&gt;Partition Pruning&lt;/strong&gt;: The &lt;a href="https://docs.rs/datafusion/latest/datafusion/datasource/listing/struct.ListingTable.html"&gt;ListingTable&lt;/a&gt; component evaluates filters that depend only on partition columns — things like &lt;code&gt;year&lt;/code&gt;, &lt;code&gt;month&lt;/code&gt;, or &lt;code&gt;region&lt;/code&gt; encoded in directory paths (e.g., &lt;code&gt;s3://data/year=2024/month=01/&lt;/code&gt;). Irrelevant directories are eliminated before we even open a file.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;File Stats Pruning&lt;/strong&gt;: The &lt;a href="https://docs.rs/datafusion/latest/datafusion/physical_optimizer/pruning/struct.FilePruner.html"&gt;FilePruner&lt;/a&gt; checks file-level min/max and null-count statistics. If these statistics prove that a file cannot satisfy the predicate, we drop it entirely — no need to read row group metadata.&lt;/li&gt;
&lt;/ul&gt;
&lt;h3 id="phase-2-row-group-statistics"&gt;Phase 2: Row Group Statistics&lt;a class="headerlink" href="#phase-2-row-group-statistics" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;For each surviving file, DataFusion reads row group metadata and potentially &lt;a href="https://parquet.apache.org/docs/file-format/bloomfilter/"&gt;bloom filters&lt;/a&gt; and classifies each row group into one of three states (the example data shown in the figures below, such as "Snow Vole" and "Alpine Ibex", is adapted from the &lt;a href="https://arxiv.org/pdf/2504.11540"&gt;Snowflake pruning paper&lt;/a&gt;):&lt;/p&gt;
&lt;figure&gt;
&lt;img alt="Row group classification: not matching, partially matching, fully matching" src="/blog/images/limit-pruning/row-group-states.svg" width="80%"/&gt;
&lt;figcaption&gt;Figure 2: Row groups are classified into three states based on their statistics.&lt;/figcaption&gt;
&lt;/figure&gt;
&lt;ul&gt;
&lt;li&gt;&lt;strong&gt;Not Matching (Skipped)&lt;/strong&gt;: Statistics prove no rows can match. The row group is ignored completely.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Partially Matching&lt;/strong&gt;: Statistics cannot rule out matching rows, but also cannot guarantee them. These groups might be scanned and verified row by row later.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Fully Matching&lt;/strong&gt;: Statistics prove that &lt;em&gt;every single row&lt;/em&gt; in the group satisfies the predicate. This state is key to making limit pruning possible.&lt;/li&gt;
&lt;/ul&gt;
&lt;h3 id="phase-3-granular-pruning"&gt;Phase 3: Granular Pruning&lt;a class="headerlink" href="#phase-3-granular-pruning" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;The final phase goes even deeper:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;strong&gt;&lt;a href="https://parquet.apache.org/docs/file-format/pageindex/"&gt;Page Index Pruning&lt;/a&gt;&lt;/strong&gt;: Parquet pages have their own min/max statistics. DataFusion uses these to skip individual data pages within a surviving row group.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;&lt;a href="https://arrow.apache.org/blog/2025/12/11/parquet-late-materialization-deep-dive/"&gt;Late Materialization&lt;/a&gt; (Row Filtering)&lt;/strong&gt;: Instead of decoding all columns at once, DataFusion decodes the cheapest, most selective columns first. It filters rows using those columns, then only decodes the remaining columns for surviving rows.&lt;/li&gt;
&lt;/ul&gt;
&lt;h2 id="the-problem-limit-was-ignored"&gt;The Problem: LIMIT Was Ignored&lt;a class="headerlink" href="#the-problem-limit-was-ignored" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;Before limit pruning, all of these stages worked well — but the pruning pipeline had &lt;strong&gt;no awareness of &lt;code&gt;LIMIT&lt;/code&gt;&lt;/strong&gt;. Consider a query like:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-sql"&gt;SELECT * FROM tracking_data
WHERE species LIKE 'Alpine%' AND s &amp;gt;= 50
LIMIT 3
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;Even when fully matched row groups alone contain enough rows to satisfy the &lt;code&gt;LIMIT&lt;/code&gt;, DataFusion would still decode partially matching groups and filter out rows that did not match, wasting resources decoding rows just to immediately discard them.&lt;/p&gt;
&lt;figure&gt;
&lt;img alt="Traditional pruning decodes partially matching groups with no LIMIT awareness" src="/blog/images/limit-pruning/wasted-io.svg" width="80%"/&gt;
&lt;figcaption&gt;Figure 3: Without limit awareness, partially matching groups are scanned and filtered even when fully matched groups already have enough rows. The left section shows 5 fully matched rows (enough to satisfy &lt;code&gt;LIMIT 5&lt;/code&gt;), while the right section with the dashed red border represents a partially matching group that is still decoded — wasting CPU and I/O on rows that may not match at all.&lt;/figcaption&gt;
&lt;/figure&gt;
&lt;p&gt;If five fully matched rows in a fully matched group already satisfy &lt;code&gt;LIMIT 5&lt;/code&gt;, why bother decoding groups where we're not even sure any rows qualify?&lt;/p&gt;
&lt;h2 id="the-solution-limit-aware-pruning"&gt;The Solution: Limit-Aware Pruning&lt;a class="headerlink" href="#the-solution-limit-aware-pruning" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;The solution adds a new step in the pruning pipeline — right after row group pruning and before page index pruning:&lt;/p&gt;
&lt;figure&gt;
&lt;img alt="Pruning pipeline with limit pruning highlighted" src="/blog/images/limit-pruning/pruning-pipeline.svg" width="80%"/&gt;
&lt;figcaption&gt;Figure 4: Limit pruning is inserted between row group and page index pruning.&lt;/figcaption&gt;
&lt;/figure&gt;
&lt;p&gt;The idea is simple: &lt;strong&gt;if fully matched row groups already contain enough rows to satisfy the &lt;code&gt;LIMIT&lt;/code&gt;, rewrite the access plan to scan only those groups and skip everything else.&lt;/strong&gt;&lt;/p&gt;
&lt;p&gt;This optimization is applied only when the query is a pure limit query with no &lt;code&gt;ORDER BY&lt;/code&gt;, because reordering which groups we scan could change the output ordering of the results. In the implementation, this check is expressed as:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-rust"&gt;// Prune by limit if limit is set and order is not sensitive
if let (Some(limit), false) = (limit, preserve_order) {
    row_groups.prune_by_limit(limit, rg_metadata, &amp;amp;file_metrics);
}
&lt;/code&gt;&lt;/pre&gt;
&lt;h2 id="mechanism-detecting-fully-matched-row-groups"&gt;Mechanism: Detecting Fully Matched Row Groups&lt;a class="headerlink" href="#mechanism-detecting-fully-matched-row-groups" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;The core insight is &lt;strong&gt;predicate negation&lt;/strong&gt;. To determine if every row in a row group satisfies the predicate, we:&lt;/p&gt;
&lt;ol&gt;
&lt;li&gt;Negate the original predicate&lt;/li&gt;
&lt;li&gt;Simplify the negated expression&lt;/li&gt;
&lt;li&gt;Evaluate the negation against the row group's statistics&lt;/li&gt;
&lt;li&gt;If the negation is &lt;em&gt;pruned&lt;/em&gt; (proven impossible), then the original predicate holds for every row&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;Since DataFusion already had expression simplification (step 2) and statistics-based pruning (step 3), implementing this was relatively straightforward — the key addition was composing these existing capabilities with predicate negation.&lt;/p&gt;
&lt;figure&gt;
&lt;img alt="Fully matched detection via predicate negation" src="/blog/images/limit-pruning/fully-matched-detection.svg" width="80%"/&gt;
&lt;figcaption&gt;Figure 5: If the negated predicate is impossible according to row group stats, all rows must match the original predicate.&lt;/figcaption&gt;
&lt;/figure&gt;
&lt;p&gt;In DataFusion's codebase, this logic lives in &lt;code&gt;identify_fully_matched_row_groups&lt;/code&gt; (&lt;a href="https://github.com/apache/datafusion/blob/main/datafusion/datasource-parquet/src/row_group_filter.rs"&gt;row_group_filter.rs&lt;/a&gt;):&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-rust"&gt;fn identify_fully_matched_row_groups(
    &amp;amp;mut self,
    candidate_row_group_indices: &amp;amp;[usize],
    arrow_schema: &amp;amp;Schema,
    parquet_schema: &amp;amp;SchemaDescriptor,
    groups: &amp;amp;[RowGroupMetaData],
    predicate: &amp;amp;PruningPredicate,
    metrics: &amp;amp;ParquetFileMetrics,
) {
    // Create the inverted predicate: NOT(original)
    let inverted_expr = Arc::new(NotExpr::new(
        Arc::clone(predicate.orig_expr()),
    ));

    // Simplify: e.g., NOT(c1 = 0) → c1 != 0
    let simplifier = PhysicalExprSimplifier::new(arrow_schema);
    let Ok(inverted_expr) = simplifier.simplify(inverted_expr) else {
        return;
    };

    let Ok(inverted_predicate) = PruningPredicate::try_new(
        inverted_expr,
        Arc::clone(predicate.schema()),
    ) else {
        return;
    };

    // Evaluate inverted predicate against row group stats
    let Ok(inverted_values) =
        inverted_predicate.prune(&amp;amp;inverted_pruning_stats)
    else {
        return;
    };

    for (i, &amp;amp;original_idx) in
        candidate_row_group_indices.iter().enumerate()
    {
        // If negation is pruned (false), all rows match original
        if !inverted_values[i] {
            self.is_fully_matched[original_idx] = true;
        }
    }
}
&lt;/code&gt;&lt;/pre&gt;
&lt;h2 id="mechanism-rewriting-the-access-plan"&gt;Mechanism: Rewriting the Access Plan&lt;a class="headerlink" href="#mechanism-rewriting-the-access-plan" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;Once we know which row groups are fully matched, the limit pruning algorithm is straightforward:&lt;/p&gt;
&lt;figure&gt;
&lt;img alt="Limit pruning access plan rewrite algorithm" src="/blog/images/limit-pruning/limit-rewrite-algorithm.svg" width="80%"/&gt;
&lt;figcaption&gt;Figure 6: The algorithm iterates fully matched groups, accumulating row counts until the limit is satisfied.&lt;/figcaption&gt;
&lt;/figure&gt;
&lt;p&gt;The implementation in &lt;code&gt;prune_by_limit&lt;/code&gt; (&lt;a href="https://github.com/apache/datafusion/blob/main/datafusion/datasource-parquet/src/row_group_filter.rs"&gt;row_group_filter.rs&lt;/a&gt;):&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-rust"&gt;pub fn prune_by_limit(
    &amp;amp;mut self,
    limit: usize,
    rg_metadata: &amp;amp;[RowGroupMetaData],
    metrics: &amp;amp;ParquetFileMetrics,
) {
    let mut fully_matched_indexes: Vec&amp;lt;usize&amp;gt; = Vec::new();
    let mut fully_matched_rows: usize = 0;

    for &amp;amp;idx in self.access_plan.row_group_indexes().iter() {
        if self.is_fully_matched[idx] {
            fully_matched_indexes.push(idx);
            fully_matched_rows += rg_metadata[idx].num_rows() as usize;
            if fully_matched_rows &amp;gt;= limit {
                break;
            }
        }
    }

    // Rewrite the plan if we have enough rows
    if fully_matched_rows &amp;gt;= limit {
        let mut new_plan = ParquetAccessPlan::new_none(rg_metadata.len());
        for &amp;amp;idx in &amp;amp;fully_matched_indexes {
            new_plan.scan(idx);
        }
        self.access_plan = new_plan;
    }
}
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;Key properties of this algorithm:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;It preserves the original row group ordering&lt;/li&gt;
&lt;li&gt;If fully matched groups don't have enough rows, the plan is unchanged — no harm done&lt;/li&gt;
&lt;li&gt;The cost is minimal: a single pass over the row group list&lt;/li&gt;
&lt;/ul&gt;
&lt;h2 id="case-study-alpine-wildlife-query"&gt;Case Study: Alpine Wildlife Query&lt;a class="headerlink" href="#case-study-alpine-wildlife-query" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;Let's walk through a concrete example adapted from the &lt;a href="https://arxiv.org/pdf/2504.11540"&gt;Snowflake pruning paper&lt;/a&gt;. Given a wildlife tracking dataset with four row groups:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-sql"&gt;SELECT * FROM tracking_data
WHERE species LIKE 'Alpine%' AND s &amp;gt;= 50
LIMIT 3
&lt;/code&gt;&lt;/pre&gt;
&lt;table class="table"&gt;
&lt;thead&gt;
&lt;tr&gt;
&lt;th&gt;Row Group&lt;/th&gt;
&lt;th&gt;Species Range&lt;/th&gt;
&lt;th&gt;S Range&lt;/th&gt;
&lt;th&gt;State&lt;/th&gt;
&lt;/tr&gt;
&lt;/thead&gt;
&lt;tbody&gt;
&lt;tr&gt;
&lt;td&gt;RG1&lt;/td&gt;
&lt;td&gt;Snow Vole, Brown Bear, Gray Wolf&lt;/td&gt;
&lt;td&gt;7–133&lt;/td&gt;
&lt;td&gt;&lt;strong&gt;Not Matching&lt;/strong&gt; (no 'Alpine%')&lt;/td&gt;
&lt;/tr&gt;
&lt;tr&gt;
&lt;td&gt;RG2&lt;/td&gt;
&lt;td&gt;Lynx, Red Fox, Alpine Bat&lt;/td&gt;
&lt;td&gt;6–71&lt;/td&gt;
&lt;td&gt;&lt;strong&gt;Partially Matching&lt;/strong&gt;&lt;/td&gt;
&lt;/tr&gt;
&lt;tr&gt;
&lt;td&gt;RG3&lt;/td&gt;
&lt;td&gt;Alpine Ibex, Alpine Goat, Alpine Sheep&lt;/td&gt;
&lt;td&gt;76–101&lt;/td&gt;
&lt;td&gt;&lt;strong&gt;Fully Matching&lt;/strong&gt;&lt;/td&gt;
&lt;/tr&gt;
&lt;tr&gt;
&lt;td&gt;RG4&lt;/td&gt;
&lt;td&gt;Mixed species&lt;/td&gt;
&lt;td&gt;Mixed&lt;/td&gt;
&lt;td&gt;&lt;strong&gt;Partially Matching&lt;/strong&gt;&lt;/td&gt;
&lt;/tr&gt;
&lt;/tbody&gt;
&lt;/table&gt;
&lt;figure&gt;
&lt;img alt="Before and after limit pruning comparison" src="/blog/images/limit-pruning/before-after.svg" width="80%"/&gt;
&lt;figcaption&gt;Figure 7: Before limit pruning, RG2 is scanned for zero hits. After limit pruning, only RG3 is scanned.&lt;/figcaption&gt;
&lt;/figure&gt;
&lt;p&gt;&lt;strong&gt;Before limit pruning&lt;/strong&gt;: DataFusion scans RG2 (0 hits — wasted I/O), then RG3 (3 hits, early return). RG2 was decoded entirely for nothing.&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;With limit pruning&lt;/strong&gt;: The system detects that RG3 has 3 fully matched rows, which satisfies &lt;code&gt;LIMIT 3&lt;/code&gt;. It rewrites the access plan to scan only RG3, skipping RG2 and RG4 entirely. One row group scanned. Zero waste.&lt;/p&gt;
&lt;h2 id="observing-limit-pruning-via-metrics"&gt;Observing Limit Pruning via Metrics&lt;a class="headerlink" href="#observing-limit-pruning-via-metrics" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;DataFusion exposes limit pruning activity through query metrics. When running a query with &lt;code&gt;EXPLAIN ANALYZE&lt;/code&gt;, you will see entries like:&lt;/p&gt;
&lt;pre&gt;&lt;code&gt;row_groups_pruned_statistics=4 total → 3 matched -&amp;gt; 1 fully matched
limit_pruned_row_groups=3 total → 1 matched
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;This tells us:
- 4 row groups were evaluated, 3 survived statistics pruning, 1 was identified as fully matching
- Of the 3 row groups that entered limit pruning, only 1 survived — 2 were pruned by the limit optimization&lt;/p&gt;
&lt;h2 id="future-directions"&gt;Future Directions&lt;a class="headerlink" href="#future-directions" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;There are two natural extensions of this work:&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;&lt;a href="https://github.com/apache/datafusion/issues/19193"&gt;Page-Level Limit Pruning&lt;/a&gt;&lt;/strong&gt;: Today, "fully matched" detection operates at the row group level. If we extend this to use page index statistics, we could stop decoding pages &lt;em&gt;within&lt;/em&gt; a row group once the limit is met. This would pay dividends for wide row groups where only a few pages hold matching data.&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;&lt;a href="https://github.com/apache/datafusion/issues/19028"&gt;Row Filter Hints&lt;/a&gt;&lt;/strong&gt;: Even when a row group is fully matched, the current row filter still evaluates predicates row by row. If we pass the fully matched groups info into the row filter builder, we can skip predicate evaluation entirely for guaranteed groups — saving CPU cycles on predicate evaluation.&lt;/p&gt;
&lt;h2 id="summary"&gt;Summary&lt;a class="headerlink" href="#summary" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;DataFusion's pruning pipeline trims redundant I/O from the partition level all the way down to individual rows. Limit pruning adds a new step that creates an early exit when fully matched row groups already satisfy the &lt;code&gt;LIMIT&lt;/code&gt;. The result is fewer row groups scanned, less data decoded, and faster queries.&lt;/p&gt;
&lt;p&gt;The key insights are:
1. &lt;strong&gt;Predicate negation&lt;/strong&gt; can identify row groups where &lt;em&gt;all&lt;/em&gt; rows match — not just "some might match"
2. &lt;strong&gt;Row count accumulation&lt;/strong&gt; across fully matched groups enables early termination&lt;/p&gt;
&lt;h2 id="about-datafusion"&gt;About DataFusion&lt;a class="headerlink" href="#about-datafusion" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;&lt;a href="https://datafusion.apache.org/"&gt;Apache DataFusion&lt;/a&gt; is an extensible query engine, written in &lt;a href="https://www.rust-lang.org/"&gt;Rust&lt;/a&gt;, that uses &lt;a href="https://arrow.apache.org"&gt;Apache Arrow&lt;/a&gt; as its in-memory format. DataFusion is used by developers to create new, fast, data-centric systems such as databases, dataframe libraries, and machine learning and streaming applications.&lt;/p&gt;
&lt;p&gt;DataFusion's core thesis is that, as a community, together we can build much more advanced technology than any of us as individuals or companies could build alone.&lt;/p&gt;
&lt;h2 id="how-to-get-involved"&gt;How to Get Involved&lt;a class="headerlink" href="#how-to-get-involved" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;If you are interested in contributing, we would love to have you. You can try out DataFusion on some of your own data and projects and let us know how it goes, contribute suggestions, documentation, bug reports, or a PR with documentation, tests, or code. A list of open issues suitable for beginners is &lt;a href="https://github.com/apache/datafusion/issues?q=is%3Aissue+is%3Aopen+label%3A%22good+first+issue%22"&gt;here&lt;/a&gt;, and you can find out how to reach us on the &lt;a href="https://datafusion.apache.org/contributor-guide/communication.html"&gt;communication doc&lt;/a&gt;.&lt;/p&gt;</content><category term="blog"/></entry><entry><title>Apache DataFusion Comet 0.14.0 Release</title><link href="https://datafusion.apache.org/blog/2026/03/18/datafusion-comet-0.14.0" rel="alternate"/><published>2026-03-18T00:00:00+00:00</published><updated>2026-03-18T00:00:00+00:00</updated><author><name>pmc</name></author><id>tag:datafusion.apache.org,2026-03-18:/blog/2026/03/18/datafusion-comet-0.14.0</id><summary type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;p&gt;The Apache DataFusion PMC is pleased to announce version 0.14.0 of the &lt;a href="https://datafusion.apache.org/comet/"&gt;Comet&lt;/a&gt; subproject.&lt;/p&gt;
&lt;p&gt;Comet is an accelerator for Apache Spark that translates Spark physical plans to DataFusion physical plans for
improved performance and efficiency without requiring any code changes.&lt;/p&gt;
&lt;p&gt;This release covers approximately eight weeks of development …&lt;/p&gt;</summary><content type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;p&gt;The Apache DataFusion PMC is pleased to announce version 0.14.0 of the &lt;a href="https://datafusion.apache.org/comet/"&gt;Comet&lt;/a&gt; subproject.&lt;/p&gt;
&lt;p&gt;Comet is an accelerator for Apache Spark that translates Spark physical plans to DataFusion physical plans for
improved performance and efficiency without requiring any code changes.&lt;/p&gt;
&lt;p&gt;This release covers approximately eight weeks of development work and is the result of merging 189 PRs from 21
contributors. See the &lt;a href="https://github.com/apache/datafusion-comet/blob/main/dev/changelog/0.14.0.md"&gt;change log&lt;/a&gt; for more information.&lt;/p&gt;
&lt;h2 id="key-features"&gt;Key Features&lt;a class="headerlink" href="#key-features" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;h3 id="native-iceberg-improvements"&gt;Native Iceberg Improvements&lt;a class="headerlink" href="#native-iceberg-improvements" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;Comet's fully-native Iceberg integration received several enhancements:&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Per-Partition Plan Serialization&lt;/strong&gt;: &lt;code&gt;CometExecRDD&lt;/code&gt; now supports per-partition plan data, reducing serialization
overhead for native Iceberg scans and enabling dynamic partition pruning (DPP).&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Vended Credentials&lt;/strong&gt;: Native Iceberg scans now support passing vended credentials from the catalog, improving
integration with cloud storage services.&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Upstream Reader Performance Improvements&lt;/strong&gt;: The Comet team contributed a number of
&lt;a href="https://iceberg.apache.org/blog/apache-iceberg-rust-0.9.0-release/#reader-performance-improvements"&gt;reader performance improvements&lt;/a&gt;
to iceberg-rust 0.9.0, which Comet now uses. These improvements benefit all iceberg-rust users.&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Performance Optimizations&lt;/strong&gt;:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;Single-pass &lt;code&gt;FileScanTask&lt;/code&gt; validation for reduced planning overhead&lt;/li&gt;
&lt;li&gt;Configurable data file concurrency via &lt;code&gt;spark.comet.scan.icebergNative.dataFileConcurrencyLimit&lt;/code&gt;&lt;/li&gt;
&lt;li&gt;Channel-based executor thread parking instead of &lt;code&gt;yield_now()&lt;/code&gt; for reduced CPU overhead&lt;/li&gt;
&lt;li&gt;Reuse of &lt;code&gt;CometConf&lt;/code&gt; and native utility instances in batch decoding&lt;/li&gt;
&lt;/ul&gt;
&lt;h3 id="native-columnar-to-row-conversion"&gt;Native Columnar-to-Row Conversion&lt;a class="headerlink" href="#native-columnar-to-row-conversion" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;Comet now uses a native columnar-to-row (C2R) conversion by default. This
feature replaces Comet's JVM-based columnar-to-row transition with a native Rust implementation, reducing JVM memory overhead
when data flows from Comet's native execution back to Spark operators that require row-based input.&lt;/p&gt;
&lt;h3 id="new-expressions"&gt;New Expressions&lt;a class="headerlink" href="#new-expressions" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;This release adds support for the following expressions:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;Date/time functions: &lt;code&gt;make_date&lt;/code&gt;, &lt;code&gt;next_day&lt;/code&gt;&lt;/li&gt;
&lt;li&gt;String functions: &lt;code&gt;right&lt;/code&gt;, &lt;code&gt;string_split&lt;/code&gt;, &lt;code&gt;luhn_check&lt;/code&gt;&lt;/li&gt;
&lt;li&gt;Math functions: &lt;code&gt;crc32&lt;/code&gt;&lt;/li&gt;
&lt;li&gt;Map functions: &lt;code&gt;map_contains_key&lt;/code&gt;, &lt;code&gt;map_from_entries&lt;/code&gt;&lt;/li&gt;
&lt;li&gt;Conversion functions: &lt;code&gt;to_csv&lt;/code&gt;&lt;/li&gt;
&lt;li&gt;Cast support: date to timestamp, numeric to timestamp, integer to binary, boolean to decimal, date to numeric&lt;/li&gt;
&lt;/ul&gt;
&lt;h3 id="ansi-mode-error-messages"&gt;ANSI Mode Error Messages&lt;a class="headerlink" href="#ansi-mode-error-messages" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;ANSI SQL mode now produces proper error messages matching Spark's expected output, improving compatibility for
workloads that rely on strict SQL error handling.&lt;/p&gt;
&lt;h3 id="datafusion-configuration-passthrough"&gt;DataFusion Configuration Passthrough&lt;a class="headerlink" href="#datafusion-configuration-passthrough" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;DataFusion session-level configurations can now be set directly from Spark using the &lt;code&gt;spark.comet.datafusion.*&lt;/code&gt;
prefix. This enables tuning DataFusion internals such as batch sizes and memory limits without modifying Comet code.&lt;/p&gt;
&lt;h2 id="performance-improvements"&gt;Performance Improvements&lt;a class="headerlink" href="#performance-improvements" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;This release includes extensive performance optimizations:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;strong&gt;Sum aggregation&lt;/strong&gt;: Specialized implementations for each eval mode eliminate per-row mode checks&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Contains expression&lt;/strong&gt;: SIMD-based scalar pattern search for faster string matching&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Batch coalescing&lt;/strong&gt;: Reduced IPC schema overhead in &lt;code&gt;BufBatchWriter&lt;/code&gt; by coalescing small batches&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Tokio runtime&lt;/strong&gt;: Worker threads now initialize from &lt;code&gt;spark.executor.cores&lt;/code&gt; for better resource utilization&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Decimal expressions&lt;/strong&gt;: Optimized decimal arithmetic operations&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Row-to-columnar transition&lt;/strong&gt;: Improved performance for JVM shuffle data conversion&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Aligned pointer reads&lt;/strong&gt;: Optimized &lt;code&gt;SparkUnsafeRow&lt;/code&gt; field accessors using aligned memory reads&lt;/li&gt;
&lt;/ul&gt;
&lt;h2 id="deprecations-and-removals"&gt;Deprecations and Removals&lt;a class="headerlink" href="#deprecations-and-removals" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;The deprecated &lt;code&gt;native_comet&lt;/code&gt; scan mode has been removed. Use &lt;code&gt;native_datafusion&lt;/code&gt; instead. Note 
that the &lt;code&gt;native_iceberg_compat&lt;/code&gt; scan is now deprecated and will be removed from a future release. &lt;/p&gt;
&lt;h2 id="compatibility"&gt;Compatibility&lt;a class="headerlink" href="#compatibility" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;This release upgrades to DataFusion 52.3, Arrow 57.3, and iceberg-rust 0.9.0. Published binaries now target
x86-64-v3 and neoverse-n1 CPU architectures for improved performance on modern hardware.&lt;/p&gt;
&lt;p&gt;Supported platforms include Spark 3.4.3, 3.5.4-3.5.8, and Spark 4.0.x with various JDK and Scala combinations.&lt;/p&gt;
&lt;p&gt;The community encourages users to test Comet with existing Spark workloads and welcomes contributions to ongoing development.&lt;/p&gt;</content><category term="blog"/></entry><entry><title>Optimizing SQL CASE Expression Evaluation</title><link href="https://datafusion.apache.org/blog/2026/02/02/datafusion_case" rel="alternate"/><published>2026-02-02T00:00:00+00:00</published><updated>2026-02-02T00:00:00+00:00</updated><author><name>Pepijn Van Eeckhoudt</name></author><id>tag:datafusion.apache.org,2026-02-02:/blog/2026/02/02/datafusion_case</id><summary type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;style&gt;
figure {
  margin: 20px 0;
}

figure img {
  display: block;
  max-width: 80%;
  margin: auto;
}

figcaption {
  font-style: italic;
  color: #555;
  font-size: 0.9em;
  max-width: 80%;
  margin: auto;
  text-align: center;
}
&lt;/style&gt;
&lt;p&gt;SQL's &lt;code&gt;CASE&lt;/code&gt; expression is one of the few explicit conditional evaluation constructs the language provides.
It allows you to control which expression from a …&lt;/p&gt;</summary><content type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;style&gt;
figure {
  margin: 20px 0;
}

figure img {
  display: block;
  max-width: 80%;
  margin: auto;
}

figcaption {
  font-style: italic;
  color: #555;
  font-size: 0.9em;
  max-width: 80%;
  margin: auto;
  text-align: center;
}
&lt;/style&gt;
&lt;p&gt;SQL's &lt;code&gt;CASE&lt;/code&gt; expression is one of the few explicit conditional evaluation constructs the language provides.
It allows you to control which expression from a set of expressions is evaluated for each row based on arbitrary boolean expressions.
Its deceptively simple syntax hides significant implementation complexity.
Over the past few &lt;a href="https://datafusion.apache.org/"&gt;Apache DataFusion&lt;/a&gt; releases, a series of improvements to &lt;code&gt;CASE&lt;/code&gt; expression evaluator have been merged that reduce both CPU time and memory allocations.
This post provides an overview of the original implementation, its performance bottlenecks, and the steps taken to address them.&lt;/p&gt;
&lt;h2 id="background-case-expression-evaluation"&gt;Background: CASE Expression Evaluation&lt;a class="headerlink" href="#background-case-expression-evaluation" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;SQL supports two forms of CASE expressions:&lt;/p&gt;
&lt;ol&gt;
&lt;li&gt;&lt;strong&gt;Simple&lt;/strong&gt;: &lt;code&gt;CASE expr WHEN value1 THEN result1 WHEN value2 THEN result2 ... END&lt;/code&gt;&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Searched&lt;/strong&gt;: &lt;code&gt;CASE WHEN condition1 THEN result1 WHEN condition2 THEN result2 ... END&lt;/code&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;The simple form evaluates an expression once for each input row and then tests that value against the expressions (typically constants) in each &lt;code&gt;WHEN&lt;/code&gt; clause using equality comparisons.&lt;/p&gt;
&lt;p&gt;Here's an example of the simple form:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-sql"&gt;CASE status
    WHEN 'pending' THEN 1
    WHEN 'active' THEN 2
    WHEN 'complete' THEN 3
    ELSE 0
END
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;In this &lt;code&gt;CASE&lt;/code&gt; expression, &lt;code&gt;status&lt;/code&gt; is evaluated once per row, and then its value is tested for equality with the values &lt;code&gt;'pending'&lt;/code&gt;, &lt;code&gt;'active'&lt;/code&gt;, and &lt;code&gt;'complete'&lt;/code&gt; in that order.
The &lt;code&gt;CASE&lt;/code&gt; expression evaluates to the value of the &lt;code&gt;THEN&lt;/code&gt; expression corresponding to the first matching &lt;code&gt;WHEN&lt;/code&gt; expression.&lt;/p&gt;
&lt;p&gt;The searched &lt;code&gt;CASE&lt;/code&gt; form is a more flexible variant.
It evaluates completely independent boolean expressions for each branch.
This allows you to test different columns with different operators per branch as shown in the following example:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-sql"&gt;CASE
    WHEN age &amp;gt; 65 THEN 'senior'
    WHEN childCount != 0 THEN 'parent'
    WHEN age &amp;lt; 21 THEN 'minor'
    ELSE 'adult'
END
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;In both forms, branches are evaluated sequentially with short-circuit semantics: for each row, once a &lt;code&gt;WHEN&lt;/code&gt; condition matches, the corresponding &lt;code&gt;THEN&lt;/code&gt; expression is evaluated.
Any further branches are not evaluated for that row.
This lazy evaluation model is critical for correctness.
It lets you safely write &lt;code&gt;CASE&lt;/code&gt; expressions like &lt;code&gt;CASE WHEN d != 0 THEN n / d ELSE NULL END&lt;/code&gt; that are guaranteed to not trigger divide-by-zero errors.&lt;/p&gt;
&lt;p&gt;Besides &lt;code&gt;CASE&lt;/code&gt;, there are a few &lt;a href="https://datafusion.apache.org/user-guide/sql/scalar_functions.html#conditional-functions"&gt;conditional scalar functions&lt;/a&gt; that provide similar, more restricted capabilities.
These include &lt;code&gt;COALESCE&lt;/code&gt;, &lt;code&gt;IFNULL&lt;/code&gt;, and &lt;code&gt;NVL2&lt;/code&gt;.
You can consider each of these functions as the equivalent of a macro for &lt;code&gt;CASE&lt;/code&gt;.
For example, &lt;code&gt;COALESCE(expr1, expr2, expr3)&lt;/code&gt; expands to:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-sql"&gt;CASE
  WHEN expr1 IS NOT NULL THEN expr1
  WHEN expr2 IS NOT NULL THEN expr2
  ELSE expr3
END
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;Since &lt;a href="https://datafusion.apache.org/"&gt;Apache DataFusion&lt;/a&gt; rewrites these conditional functions to their equivalent &lt;code&gt;CASE&lt;/code&gt; expression, any optimizations related to &lt;code&gt;CASE&lt;/code&gt; described in this post also apply to conditional function evaluation.&lt;/p&gt;
&lt;h2 id="case-evaluation-in-datafusion-5000"&gt;&lt;code&gt;CASE&lt;/code&gt; Evaluation in DataFusion 50.0.0&lt;a class="headerlink" href="#case-evaluation-in-datafusion-5000" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;For the remainder of this post, we'll be looking at 'searched CASE' evaluation.
'Simple CASE' uses a distinct, but very similar implementation.
The same set of improvements has been applied to both.&lt;/p&gt;
&lt;p&gt;The baseline implementation in DataFusion 50.0.0 evaluated &lt;code&gt;CASE&lt;/code&gt; using a common, straightforward approach:&lt;/p&gt;
&lt;ol&gt;
&lt;li&gt;Start with an output array &lt;code&gt;out&lt;/code&gt; with the same length as the input batch, filled with nulls. Additionally, create a bit vector &lt;code&gt;remainder&lt;/code&gt; with the same length and each value set to &lt;code&gt;true&lt;/code&gt;.&lt;/li&gt;
&lt;li&gt;For each &lt;code&gt;WHEN&lt;/code&gt;/&lt;code&gt;THEN&lt;/code&gt; branch:&lt;ul&gt;
&lt;li&gt;Evaluate the &lt;code&gt;WHEN&lt;/code&gt; condition for the remaining unmatched rows using &lt;a href="https://docs.rs/datafusion/latest/datafusion/physical_expr/trait.PhysicalExpr.html#method.evaluate_selection"&gt;&lt;code&gt;PhysicalExpr::evaluate_selection&lt;/code&gt;&lt;/a&gt;, passing in the input batch and the &lt;code&gt;remainder&lt;/code&gt; mask.&lt;/li&gt;
&lt;li&gt;If any rows matched, evaluate the &lt;code&gt;THEN&lt;/code&gt; expression for those rows using &lt;code&gt;PhysicalExpr::evaluate_selection&lt;/code&gt;.&lt;/li&gt;
&lt;li&gt;Merge the results into the &lt;code&gt;out&lt;/code&gt; array using the &lt;a href="https://docs.rs/arrow/latest/arrow/compute/kernels/zip/fn.zip.html"&gt;&lt;code&gt;zip&lt;/code&gt;&lt;/a&gt; kernel.&lt;/li&gt;
&lt;li&gt;Update the &lt;code&gt;remainder&lt;/code&gt; mask to exclude the matched rows.&lt;/li&gt;
&lt;/ul&gt;
&lt;/li&gt;
&lt;li&gt;If there's an &lt;code&gt;ELSE&lt;/code&gt; clause, evaluate it for any remaining unmatched rows and merge using &lt;a href="https://docs.rs/arrow/latest/arrow/compute/kernels/zip/fn.zip.html"&gt;&lt;code&gt;zip&lt;/code&gt;&lt;/a&gt;.&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;Here's a simplified version of the Rust code for the original loop:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-rust"&gt;let mut out = new_null_array(&amp;amp;return_type, batch.num_rows());
let mut remainder = BooleanArray::from(vec![true; batch.num_rows()]);

for (when_expr, then_expr) in &amp;amp;self.when_then_expr {
    // Determine for which remaining rows the WHEN condition matches
    let when = when_expr.evaluate_selection(batch, &amp;amp;remainder)?
        .into_array(batch.num_rows())?;
    // Ensure any `NULL` values are treated as false
    let when_and_rem = and(&amp;amp;when, &amp;amp;remainder)?;

    if when_and_rem.true_count() == 0 {
        continue;
    }

    // Evaluate the THEN expression for matching rows
    let then = then_expr.evaluate_selection(batch, &amp;amp;when_and_rem)?;
    // Merge results into output array
    out = zip(&amp;amp;when_and_rem, &amp;amp;then_value, &amp;amp;out)?;
    // Update remainder mask to exclude matched rows
    remainder = and_not(&amp;amp;remainder, &amp;amp;when_and_rem)?;
}
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;Let's examine one iteration of this loop for the following &lt;code&gt;CASE&lt;/code&gt; expression:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-sql"&gt;CASE
    WHEN col = 'b' THEN 100
    ELSE 200
END
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;Schematically, it will look as follows:&lt;/p&gt;
&lt;figure&gt;
&lt;img alt="Schematic representation of data flow in the original CASE implementation" class="img-fluid" src="/blog/images/case/original_loop.svg" width="100%"/&gt;
&lt;figcaption&gt;One iteration of the `CASE` evaluation loop&lt;/figcaption&gt;
&lt;/figure&gt;
&lt;p&gt;This implementation works perfectly fine, but there's significant room for optimization, mostly related to the usage of &lt;code&gt;evaluate_selection&lt;/code&gt;.
To understand why, we need to dig a little deeper into the implementation of that function.
Here's a simplified version of it that captures the relevant parts:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-rust"&gt;pub trait PhysicalExpr {
    fn evaluate_selection(
        &amp;amp;self,
        batch: &amp;amp;RecordBatch,
        selection: &amp;amp;BooleanArray,
    ) -&amp;gt; Result&amp;lt;ColumnarValue&amp;gt; {
        // Reduce record batch to only include rows that match selection
        let filtered_batch = filter_record_batch(batch, selection)?;
        // Perform regular evaluation on filtered batch
        let filtered_result = self.evaluate(&amp;amp;filtered_batch)?;
        // Expand result array to match original batch length
        scatter(selection, filtered_result)
    }
}
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;Going back to the same example as before, the data flow in &lt;code&gt;evaluate_selection&lt;/code&gt; looks like this:&lt;/p&gt;
&lt;figure&gt;
&lt;img alt="Schematic representation of `evaluate_selection` evaluation" class="img-fluid" src="/blog/images/case/evaluate_selection.svg" width="100%"/&gt;
&lt;figcaption&gt;evaluate_selection data flow&lt;/figcaption&gt;
&lt;/figure&gt;
&lt;p&gt;The &lt;code&gt;evaluate_selection&lt;/code&gt; method first filters the input batch to only include rows that match the &lt;code&gt;selection&lt;/code&gt; mask.
It then calls the regular &lt;code&gt;evaluate&lt;/code&gt; method using the filtered batch as input.
Finally, to return a result array with the same number of rows as &lt;code&gt;batch&lt;/code&gt;, the &lt;code&gt;scatter&lt;/code&gt; function is called.
This function produces a new array padded with &lt;code&gt;null&lt;/code&gt; values for any rows that didn't match the &lt;code&gt;selection&lt;/code&gt; mask.&lt;/p&gt;
&lt;p&gt;So how can we improve the performance of the simple evaluation strategy and use of &lt;code&gt;evaluate_selection&lt;/code&gt;?&lt;/p&gt;
&lt;h3 id="opportunity-1-early-exit"&gt;Opportunity 1: Early Exit&lt;a class="headerlink" href="#opportunity-1-early-exit" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;The &lt;code&gt;CASE&lt;/code&gt; evaluation loop always iterates through all branches, even when every row has already been matched.
In queries where early branches match all rows, this results in unnecessary work being done for the remaining branches.&lt;/p&gt;
&lt;h3 id="opportunity-2-optimize-repeated-filtering-scattering-and-merging"&gt;Opportunity 2: Optimize Repeated Filtering, Scattering, and Merging&lt;a class="headerlink" href="#opportunity-2-optimize-repeated-filtering-scattering-and-merging" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;Each iteration performs a number of operations that are very well-optimized, but still take up a significant amount of CPU time:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;strong&gt;Filtering&lt;/strong&gt;: &lt;code&gt;PhysicalExpr::evaluate_selection&lt;/code&gt; filters the entire &lt;code&gt;RecordBatch&lt;/code&gt; for each branch. For the &lt;code&gt;WHEN&lt;/code&gt; expression, this is done even if the selection mask was entirely empty.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Scattering&lt;/strong&gt;: &lt;code&gt;PhysicalExpr::evaluate_selection&lt;/code&gt; scatters the filtered result back to the original &lt;code&gt;RecordBatch&lt;/code&gt; length.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Merging&lt;/strong&gt;: The &lt;code&gt;zip&lt;/code&gt; kernel is called once per branch to merge partial results into the output array&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;Each of these operations needs to allocate memory for new arrays and shuffle quite a bit of data around.&lt;/p&gt;
&lt;h3 id="opportunity-3-filter-only-necessary-columns"&gt;Opportunity 3: Filter only Necessary Columns&lt;a class="headerlink" href="#opportunity-3-filter-only-necessary-columns" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;The &lt;code&gt;PhysicalExpr::evaluate_selection&lt;/code&gt; method filters the entire record batch, including columns that the current branch's &lt;code&gt;WHEN&lt;/code&gt; and &lt;code&gt;THEN&lt;/code&gt; expressions don't reference.
For wide tables (many columns) with narrow expressions (few column references), this is wasteful.&lt;/p&gt;
&lt;p&gt;Suppose you have a table with 26 columns named &lt;code&gt;a&lt;/code&gt; through &lt;code&gt;z&lt;/code&gt;, and the following simple &lt;code&gt;CASE&lt;/code&gt; expression:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-sql"&gt;CASE
  WHEN a &amp;gt; 1000 THEN 'large'
  WHEN a &amp;gt;= 0 THEN 'positive'
  ELSE 'negative'
END
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;The implementation would filter all 26 columns even though only a single column is needed for the entire &lt;code&gt;CASE&lt;/code&gt; expression evaluation.
Again this involves a non-negligible amount of allocation and data copying.&lt;/p&gt;
&lt;h2 id="performance-optimizations"&gt;Performance Optimizations&lt;a class="headerlink" href="#performance-optimizations" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;h3 id="optimization-1-short-circuit-early-exit"&gt;Optimization 1: Short-Circuit Early Exit&lt;a class="headerlink" href="#optimization-1-short-circuit-early-exit" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;The first optimization is straightforward.
As soon as we detect that all rows of the batch have been matched, we break out of the evaluation loop:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-rust"&gt;let mut remainder_count = batch.num_rows();

for (when_expr, then_expr) in &amp;amp;self.when_then_expr {
    if remainder_count == 0 {
        break;  // All rows matched, exit early
    }

    // ... evaluate branch ...

    let when_match_count = when_value.true_count();
    remainder_count -= when_match_count;
}
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;Additionally, we avoid evaluating the &lt;code&gt;ELSE&lt;/code&gt; clause when no rows remain:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-rust"&gt;if let Some(else_expr) = &amp;amp;self.else_expr {
    remainder = or(&amp;amp;base_nulls, &amp;amp;remainder)?;
    if remainder.true_count() &amp;gt; 0 {
        // ... evaluate else ...
    }
}
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;For queries where early branches match all rows, this eliminates unnecessary branch evaluations and &lt;code&gt;ELSE&lt;/code&gt; clause processing.&lt;/p&gt;
&lt;p&gt;This optimization was implemented by Pepijn Van Eeckhoudt (&lt;a href="https://github.com/pepijnve"&gt;&lt;code&gt;@pepijnve&lt;/code&gt;&lt;/a&gt;) in &lt;a href="https://github.com/apache/datafusion/pull/17898"&gt;PR #17898&lt;/a&gt;&lt;/p&gt;
&lt;h3 id="optimization-2-optimized-result-merging"&gt;Optimization 2: Optimized Result Merging&lt;a class="headerlink" href="#optimization-2-optimized-result-merging" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;The second optimization fundamentally restructures how the results of each loop iteration will be merged.
The diagram below illustrates the optimized data flow when evaluating the &lt;code&gt;CASE WHEN col = 'b' THEN 100 ELSE 200 END&lt;/code&gt; from before:&lt;/p&gt;
&lt;figure&gt;
&lt;img alt="Schematic representation of optimized evaluation loop" class="img-fluid" src="/blog/images/case/merging.svg" width="100%"/&gt;
&lt;figcaption&gt;optimized evaluation loop&lt;/figcaption&gt;
&lt;/figure&gt;
&lt;p&gt;In the reworked implementation, the &lt;code&gt;evaluate_selection&lt;/code&gt; function is no longer used.
The key insight is that we can defer all merging until the end of the evaluation loop by tracking result provenance.
This was implemented with the following changes:&lt;/p&gt;
&lt;ol&gt;
&lt;li&gt;Augment the input batch with a column containing row indices.&lt;/li&gt;
&lt;li&gt;Reduce the augmented batch after each loop iteration to only contain the remaining rows.&lt;/li&gt;
&lt;li&gt;Use the row index column to track which partial result array contains the value for each row.&lt;/li&gt;
&lt;li&gt;Perform a single merge operation at the end instead of a &lt;code&gt;zip&lt;/code&gt; operation after each loop iteration.&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;These changes make it unnecessary to &lt;code&gt;scatter&lt;/code&gt; and &lt;code&gt;zip&lt;/code&gt; results in each loop iteration.
Instead, when all rows have been matched, we then merge the partial results using &lt;a href="https://docs.rs/arrow-select/57.1.0/arrow_select/merge/fn.merge_n.html"&gt;&lt;code&gt;arrow_select::merge::merge_n&lt;/code&gt;&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;The diagram below illustrates how &lt;code&gt;merge_n&lt;/code&gt; works for an example where three &lt;code&gt;WHEN/THEN&lt;/code&gt; branches produced results.
The first branch produced the result &lt;code&gt;A&lt;/code&gt; for row 2, the second produced &lt;code&gt;B&lt;/code&gt; for row 1, and the third produced &lt;code&gt;C&lt;/code&gt; and &lt;code&gt;D&lt;/code&gt; for rows 4 and 5.&lt;/p&gt;
&lt;figure&gt;
&lt;img alt="Schematic illustration of the merge_n algorithm" class="img-fluid" src="/blog/images/case/merge_n.svg" width="100%"/&gt;
&lt;figcaption&gt;merge_n example&lt;/figcaption&gt;
&lt;/figure&gt;
&lt;p&gt;The &lt;code&gt;merge_n&lt;/code&gt; algorithm scans through the indices array.
For each non-empty cell, it takes one value from the corresponding values array.
In the example above, we first encounter &lt;code&gt;1&lt;/code&gt;.
This takes the first element from the values array with index &lt;code&gt;1&lt;/code&gt;, resulting in &lt;code&gt;B&lt;/code&gt;.
The next cell contains &lt;code&gt;0&lt;/code&gt; which takes &lt;code&gt;A&lt;/code&gt;, from the first array.
Finally, we encounter &lt;code&gt;2&lt;/code&gt; twice.
This takes the first and second element from the last values array respectively.&lt;/p&gt;
&lt;p&gt;This algorithm was initially implemented in DataFusion for the &lt;code&gt;CASE&lt;/code&gt; implementation, but in the meantime has been generalized and moved into the &lt;code&gt;arrow-rs&lt;/code&gt; crate as &lt;a href="https://docs.rs/arrow-select/57.1.0/arrow_select/merge/fn.merge_n.html"&gt;&lt;code&gt;arrow_select::merge::merge_n&lt;/code&gt;&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;This optimization was implemented by Pepijn Van Eeckhoudt (&lt;a href="https://github.com/pepijnve"&gt;&lt;code&gt;@pepijnve&lt;/code&gt;&lt;/a&gt;) in &lt;a href="https://github.com/apache/datafusion/pull/18152"&gt;PR #18152&lt;/a&gt;&lt;/p&gt;
&lt;h3 id="optimization-3-column-projection"&gt;Optimization 3: Column Projection&lt;a class="headerlink" href="#optimization-3-column-projection" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;The third optimization addresses the "filtering unused columns" overhead through projection.&lt;/p&gt;
&lt;p&gt;Look at the following query example where the &lt;code&gt;mailing_address&lt;/code&gt; table has the columns &lt;code&gt;name&lt;/code&gt;, &lt;code&gt;surname&lt;/code&gt;, &lt;code&gt;street&lt;/code&gt;, &lt;code&gt;number&lt;/code&gt;, &lt;code&gt;city&lt;/code&gt;, &lt;code&gt;state&lt;/code&gt;, &lt;code&gt;country&lt;/code&gt;:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-sql"&gt;SELECT *, CASE WHEN country = 'USA' THEN state ELSE country END AS region
FROM mailing_address 
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;You can see that the &lt;code&gt;CASE&lt;/code&gt; expression only references the columns &lt;code&gt;country&lt;/code&gt; and &lt;code&gt;state&lt;/code&gt;, but because all columns are being queried, projection pushdown cannot reduce the number of columns being fed in to the projection operator.&lt;/p&gt;
&lt;figure&gt;
&lt;img alt="Schematic illustration of CASE evaluation without projection" class="img-fluid" src="/blog/images/case/no_projection.svg" width="100%"/&gt;
&lt;figcaption&gt;CASE evaluation without projection&lt;/figcaption&gt;
&lt;/figure&gt;
&lt;p&gt;During &lt;code&gt;CASE&lt;/code&gt; evaluation, the batch must be filtered using the &lt;code&gt;WHEN&lt;/code&gt; expression to evaluate the &lt;code&gt;THEN&lt;/code&gt; expression values.
As the diagram above shows, this filtering creates a reduced copy of all columns.&lt;/p&gt;
&lt;p&gt;This unnecessary copying can be avoided by first narrowing the batch to only include the columns that are actually needed.&lt;/p&gt;
&lt;figure&gt;
&lt;img alt="Schematic illustration of CASE evaluation with projection" class="img-fluid" src="/blog/images/case/projection.svg" width="100%"/&gt;
&lt;figcaption&gt;CASE evaluation with projection&lt;/figcaption&gt;
&lt;/figure&gt;
&lt;p&gt;At first glance, this might not seem beneficial, since we're introducing an additional processing step.
Luckily projection of a record batch only requires a shallow copy of the record batch.
The column arrays themselves are not copied, and the only work that is actually done is incrementing the reference counts of the columns.&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Impact&lt;/strong&gt;: For wide tables with narrow CASE expressions, this dramatically reduces filtering overhead by removing the copying of unused columns.&lt;/p&gt;
&lt;p&gt;This optimization was implemented by Pepijn Van Eeckhoudt (&lt;a href="https://github.com/pepijnve"&gt;&lt;code&gt;@pepijnve&lt;/code&gt;&lt;/a&gt;) in &lt;a href="https://github.com/apache/datafusion/pull/18329"&gt;PR #18329&lt;/a&gt;&lt;/p&gt;
&lt;h3 id="optimization-4-eliminating-scatter-in-two-branch-case"&gt;Optimization 4: Eliminating Scatter in Two-Branch Case&lt;a class="headerlink" href="#optimization-4-eliminating-scatter-in-two-branch-case" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;Some of the earlier examples in this post use expressions of the form &lt;code&gt;CASE WHEN condition THEN expr1 ELSE expr2 END&lt;/code&gt; to explain how the general evaluation loop works.
For this kind of two-branch &lt;code&gt;CASE&lt;/code&gt; expression, &lt;a href="https://datafusion.apache.org/"&gt;Apache DataFusion&lt;/a&gt; has a more optimized implementation that unrolls the loop.
This specialized &lt;code&gt;ExpressionOrExpression&lt;/code&gt; fast path still used &lt;code&gt;evaluate_selection()&lt;/code&gt; for both branches which uses &lt;code&gt;scatter&lt;/code&gt; and &lt;code&gt;zip&lt;/code&gt; to combine the results incurring the same performance overhead as the general implementation.&lt;/p&gt;
&lt;p&gt;The revised implementation eliminates the use of &lt;code&gt;evaluate_selection&lt;/code&gt; as follows:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-rust"&gt;// Compute the `WHEN` condition for the entire batch
let when_filter = create_filter(&amp;amp;when_value);

// Compute a compact array of `THEN` values for the matching rows
let then_batch = filter_record_batch(batch, &amp;amp;when_filter)?;
let then_value = then_expr.evaluate(&amp;amp;then_batch)?;

// Compute a compact array of `ELSE` values for the non-matching rows
let else_filter = create_filter(&amp;amp;not(&amp;amp;when_value)?);
let else_batch = filter_record_batch(batch, &amp;amp;else_filter)?;
let else_value = else_expr.evaluate(&amp;amp;else_batch)?;
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;This produces two compact arrays, one for the THEN values and one for the ELSE values, which are then merged with the &lt;code&gt;merge&lt;/code&gt; function.
In contrast to &lt;code&gt;zip&lt;/code&gt;, &lt;code&gt;merge&lt;/code&gt; does not require both of its value inputs to have the same length.
Instead it requires that the sum of the length of the value inputs matches the length of the mask array.&lt;/p&gt;
&lt;figure&gt;
&lt;img alt="Schematic illustration of the merge algorithm" class="img-fluid" src="/blog/images/case/merge.svg" width="100%"/&gt;
&lt;figcaption&gt;merge example&lt;/figcaption&gt;
&lt;/figure&gt;
&lt;p&gt;This eliminates unnecessary &lt;code&gt;scatter&lt;/code&gt; operations and memory allocations for one of the most common &lt;code&gt;CASE&lt;/code&gt; expression patterns.&lt;/p&gt;
&lt;p&gt;Just like &lt;code&gt;merge_n&lt;/code&gt;, this operation has been moved into &lt;code&gt;arrow-rs&lt;/code&gt; as &lt;a href="https://docs.rs/arrow-select/57.1.0/arrow_select/merge/fn.merge.html"&gt;&lt;code&gt;arrow_select::merge::merge&lt;/code&gt;&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;This optimization was implemented by Pepijn Van Eeckhoudt (&lt;a href="https://github.com/pepijnve"&gt;&lt;code&gt;@pepijnve&lt;/code&gt;&lt;/a&gt;) in &lt;a href="https://github.com/apache/datafusion/pull/18444"&gt;PR #18444&lt;/a&gt;&lt;/p&gt;
&lt;h3 id="optimization-5-table-lookup-of-constants"&gt;Optimization 5: Table Lookup of Constants&lt;a class="headerlink" href="#optimization-5-table-lookup-of-constants" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;Up until now, we've discussed the implementations for generic &lt;code&gt;CASE&lt;/code&gt; expressions that use non-constant expressions for both &lt;code&gt;WHEN&lt;/code&gt; and &lt;code&gt;THEN&lt;/code&gt;.
Another common use of &lt;code&gt;CASE&lt;/code&gt; is to perform a mapping from one set of constants to another.
For instance, you can expand numeric constants to human-readable strings using the following &lt;code&gt;CASE&lt;/code&gt; example.&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-sql"&gt;CASE status
  WHEN 0 THEN 'idle'
  WHEN 1 THEN 'running'
  WHEN 2 THEN 'paused'
  WHEN 3 THEN 'stopped'
  ELSE 'unknown'
END
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;A final &lt;code&gt;CASE&lt;/code&gt; optimization recognizes this pattern and compiles the &lt;code&gt;CASE&lt;/code&gt; expression into a hash table.
Rather than evaluating the &lt;code&gt;WHEN&lt;/code&gt; and &lt;code&gt;THEN&lt;/code&gt; expressions, the input expression is evaluated once, and the result array is computed using a vectorized hash table lookup.
This approach avoids the need to filter the input batch and combine partial results entirely.
The result array is computed in a single pass over the input values, and the computation time does not grow significantly with the number of &lt;code&gt;WHEN&lt;/code&gt; branches in the &lt;code&gt;CASE&lt;/code&gt; expression.&lt;/p&gt;
&lt;p&gt;This optimization was implemented by Raz Luvaton (&lt;a href="https://github.com/rluvaton"&gt;&lt;code&gt;@rluvaton&lt;/code&gt;&lt;/a&gt;) in &lt;a href="https://github.com/apache/datafusion/pull/18183"&gt;PR #18183&lt;/a&gt;&lt;/p&gt;
&lt;h2 id="results"&gt;Results&lt;a class="headerlink" href="#results" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;The degree to which the performance optimizations described in this post will benefit your queries is highly dependent on both your data and your queries.
To give some idea of the impact, we ran the following query on the TPC_H &lt;code&gt;orders&lt;/code&gt; table with a scale factor of 100:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-sql"&gt;SELECT
    *,
    case o_orderstatus
        when 'O' then 'ordered'
        when 'F' then 'filled'
        when 'P' then 'pending'
        else 'other'
    end
from orders
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;This query was first run with DataFusion 50.0.0 to get a baseline measurement.
The same query was then run with each optimization applied in turn.
The recorded times are presented as the blue series in the chart below.
The green series shows the time measurement for the &lt;code&gt;SELECT * FROM orders&lt;/code&gt; to give an idea of the cost the addition of a &lt;code&gt;CASE&lt;/code&gt; expression in a query incurs.
All measurements were made with a target partition count of &lt;code&gt;1&lt;/code&gt;.&lt;/p&gt;
&lt;figure&gt;
&lt;img alt="Performance measurements chart" class="img-fluid" src="/blog/images/case/results.png" width="100%"/&gt;
&lt;figcaption&gt;Performance measurements&lt;/figcaption&gt;
&lt;/figure&gt;
&lt;p&gt;What you can see in the chart is that the effect of the various optimizations compounds up to the &lt;code&gt;project&lt;/code&gt; measurement.
Up to that point these results are applicable to any &lt;code&gt;CASE&lt;/code&gt; expression.
The final improvement in the &lt;code&gt;hash&lt;/code&gt; measurement is only applicable to simple &lt;code&gt;CASE&lt;/code&gt; expressions with constant &lt;code&gt;WHEN&lt;/code&gt; and &lt;code&gt;THEN&lt;/code&gt; expressions.&lt;/p&gt;
&lt;p&gt;The cumulative effect of these optimizations is a 63-71% reduction in CPU time spent evaluating &lt;code&gt;CASE&lt;/code&gt; expressions compared to the baseline.&lt;/p&gt;
&lt;h2 id="summary"&gt;Summary&lt;a class="headerlink" href="#summary" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;Through a number of targeted optimizations, we've transformed &lt;code&gt;CASE&lt;/code&gt; expression evaluation from a simple, but unoptimized implementation into a highly optimized one.
The optimizations described in this post compound: a &lt;code&gt;CASE&lt;/code&gt; expression on a wide table with multiple branches and early matches benefits from all four optimizations simultaneously.
The result is significantly reduced CPU time and memory allocation in SQL constructs that are essential for ETL-like queries.&lt;/p&gt;
&lt;h2 id="about-datafusion"&gt;About DataFusion&lt;a class="headerlink" href="#about-datafusion" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;&lt;a href="https://datafusion.apache.org/"&gt;Apache DataFusion&lt;/a&gt; is an extensible query engine, written in &lt;a href="https://www.rust-lang.org/"&gt;Rust&lt;/a&gt;, that uses &lt;a href="https://arrow.apache.org"&gt;Apache Arrow&lt;/a&gt; as its in-memory format. DataFusion is used by developers to create new, fast, data-centric systems such as databases, dataframe libraries,
and machine learning and streaming applications.
While &lt;a href="https://datafusion.apache.org/user-guide/introduction.html#project-goals"&gt;DataFusion’s primary design goal&lt;/a&gt; is to accelerate the creation of other data-centric systems, it provides a reasonable experience directly out of the box as a &lt;a href="https://datafusion.apache.org/user-guide/dataframe.html"&gt;dataframe library&lt;/a&gt;, &lt;a href="https://datafusion.apache.org/python/"&gt;Python library&lt;/a&gt;, and &lt;a href="https://datafusion.apache.org/user-guide/cli/"&gt;command-line SQL tool&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;DataFusion's core thesis is that, as a community, together we can build much more advanced technology than any of us as individuals or companies could build alone.
Without DataFusion, highly performant vectorized query engines would remain the domain of a few large companies and world-class research institutions.
With DataFusion, we can all build on top of a shared foundation and focus on what makes our projects unique.&lt;/p&gt;
&lt;h2 id="how-to-get-involved"&gt;How to Get Involved&lt;a class="headerlink" href="#how-to-get-involved" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;DataFusion is not a project built or driven by a single person, company, or foundation.
Rather, our community of users and contributors works together to build a shared technology that none of us could have built alone.&lt;/p&gt;
&lt;p&gt;If you are interested in joining us, we would love to have you. You can try out DataFusion on some of your own data and projects and let us know how it goes, contribute suggestions, documentation, bug reports, or a PR with documentation, tests, or code.
A list of open issues suitable for beginners is &lt;a href="https://github.com/apache/arrow-datafusion/issues?q=is%3Aissue+is%3Aopen+label%3A%22good+first+issue%22"&gt;here&lt;/a&gt;, and you can find out how to reach us on the &lt;a href="https://datafusion.apache.org/contributor-guide/communication.html"&gt;communication doc&lt;/a&gt;.&lt;/p&gt;</content><category term="blog"/></entry><entry><title>Apache DataFusion Comet 0.13.0 Release</title><link href="https://datafusion.apache.org/blog/2026/01/30/datafusion-comet-0.13.0" rel="alternate"/><published>2026-01-30T00:00:00+00:00</published><updated>2026-01-30T00:00:00+00:00</updated><author><name>pmc</name></author><id>tag:datafusion.apache.org,2026-01-30:/blog/2026/01/30/datafusion-comet-0.13.0</id><summary type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;p&gt;The Apache DataFusion PMC is pleased to announce version 0.13.0 of the &lt;a href="https://datafusion.apache.org/comet/"&gt;Comet&lt;/a&gt; subproject.&lt;/p&gt;
&lt;p&gt;Comet is an accelerator for Apache Spark that translates Spark physical plans to DataFusion physical plans for
improved performance and efficiency without requiring any code changes.&lt;/p&gt;
&lt;p&gt;This release covers approximately eight weeks of development …&lt;/p&gt;</summary><content type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;p&gt;The Apache DataFusion PMC is pleased to announce version 0.13.0 of the &lt;a href="https://datafusion.apache.org/comet/"&gt;Comet&lt;/a&gt; subproject.&lt;/p&gt;
&lt;p&gt;Comet is an accelerator for Apache Spark that translates Spark physical plans to DataFusion physical plans for
improved performance and efficiency without requiring any code changes.&lt;/p&gt;
&lt;p&gt;This release covers approximately eight weeks of development work and is the result of merging 169 PRs from 15
contributors. See the &lt;a href="https://github.com/apache/datafusion-comet/blob/main/dev/changelog/0.13.0.md"&gt;change log&lt;/a&gt; for more information.&lt;/p&gt;
&lt;h2 id="key-features"&gt;Key Features&lt;a class="headerlink" href="#key-features" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;h3 id="native-parquet-write-support-experimental"&gt;Native Parquet Write Support (Experimental)&lt;a class="headerlink" href="#native-parquet-write-support-experimental" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;This release introduces experimental native Parquet write capabilities, allowing Comet to intercept and execute Parquet write operations natively through DataFusion. Key capabilities include:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;File commit protocol support for reliable writes&lt;/li&gt;
&lt;li&gt;Remote HDFS writing via OpenDAL integration&lt;/li&gt;
&lt;li&gt;Complex type support (arrays, maps, structs)&lt;/li&gt;
&lt;li&gt;Proper handling of object store settings&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;To enable native Parquet writes, set:&lt;/p&gt;
&lt;pre&gt;&lt;code&gt;spark.comet.allowIncompatibleOp.DataWritingCommandExec=true
spark.comet.parquet.write.enabled=true
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;&lt;strong&gt;Note&lt;/strong&gt;: This feature is highly experimental and should not be used in production environments. It is currently categorized as a testing feature and is disabled by default.&lt;/p&gt;
&lt;h3 id="native-iceberg-improvements"&gt;Native Iceberg Improvements&lt;a class="headerlink" href="#native-iceberg-improvements" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;Comet's fully-native Iceberg integration received significant enhancements in this release:&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;REST Catalog Support&lt;/strong&gt;: Native Iceberg scans now support REST catalogs, enabling integration with catalog services like Apache Polaris and Tabular. Configure with:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-shell"&gt;--conf spark.sql.catalog.rest_cat=org.apache.iceberg.spark.SparkCatalog
--conf spark.sql.catalog.rest_cat.catalog-impl=org.apache.iceberg.rest.RESTCatalog
--conf spark.sql.catalog.rest_cat.uri=http://localhost:8181
--conf spark.comet.scan.icebergNative.enabled=true
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;&lt;strong&gt;Session Token Authentication&lt;/strong&gt;: Added support for session tokens in native Iceberg scans for secure S3 access.&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Performance Optimizations&lt;/strong&gt;:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;Deduplicated serialized metadata reducing memory overhead&lt;/li&gt;
&lt;li&gt;Switched from JSON to protobuf for partition value serialization&lt;/li&gt;
&lt;li&gt;Removed IcebergFileStream in favor of iceberg-rust's built-in parallelization&lt;/li&gt;
&lt;li&gt;Reduced metadata serialization points&lt;/li&gt;
&lt;li&gt;Added SchemaAdapter caching&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;To enable fully-native Iceberg scanning:&lt;/p&gt;
&lt;pre&gt;&lt;code&gt;spark.comet.scan.icebergNative.enabled=true
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;The native reader supports Iceberg table spec v1 and v2, all primitive and complex types, schema evolution, time travel, positional and equality deletes, filter pushdown, and various storage backends (local, HDFS, S3).&lt;/p&gt;
&lt;h3 id="native-csv-reading-experimental"&gt;Native CSV Reading (Experimental)&lt;a class="headerlink" href="#native-csv-reading-experimental" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;Experimental support for native CSV file reading has been added, expanding Comet's file format capabilities beyond Parquet.&lt;/p&gt;
&lt;h3 id="new-expressions"&gt;New Expressions&lt;a class="headerlink" href="#new-expressions" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;The release adds support for numerous expressions:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;Array functions: &lt;code&gt;explode&lt;/code&gt;, &lt;code&gt;explode_outer&lt;/code&gt;, &lt;code&gt;size&lt;/code&gt;&lt;/li&gt;
&lt;li&gt;Date/time functions: &lt;code&gt;unix_date&lt;/code&gt;, &lt;code&gt;date_format&lt;/code&gt;, &lt;code&gt;datediff&lt;/code&gt;, &lt;code&gt;last_day&lt;/code&gt;, &lt;code&gt;unix_timestamp&lt;/code&gt;&lt;/li&gt;
&lt;li&gt;String functions: &lt;code&gt;left&lt;/code&gt;&lt;/li&gt;
&lt;li&gt;JSON functions: &lt;code&gt;from_json&lt;/code&gt; (partial support)&lt;/li&gt;
&lt;/ul&gt;
&lt;h3 id="ansi-mode-support"&gt;ANSI Mode Support&lt;a class="headerlink" href="#ansi-mode-support" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;Sum and average aggregate expressions now support ANSI mode for both integer and decimal inputs, enabling overflow checking in strict SQL mode.&lt;/p&gt;
&lt;h3 id="native-shuffle-improvements"&gt;Native Shuffle Improvements&lt;a class="headerlink" href="#native-shuffle-improvements" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;ul&gt;
&lt;li&gt;Round-robin partitioning is now supported in native shuffle&lt;/li&gt;
&lt;li&gt;Spill metrics are now reported correctly&lt;/li&gt;
&lt;li&gt;Configurable shuffle writer buffer size via &lt;code&gt;spark.comet.shuffle.write.bufferSize&lt;/code&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;h2 id="performance-improvements"&gt;Performance Improvements&lt;a class="headerlink" href="#performance-improvements" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;This release includes extensive performance optimizations:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;strong&gt;String to integer casting&lt;/strong&gt;: Significant speedups through optimized parsing&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;String functions&lt;/strong&gt;: Optimized &lt;code&gt;lpad&lt;/code&gt;/&lt;code&gt;rpad&lt;/code&gt; to remove unnecessary memory allocations&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Date operations&lt;/strong&gt;: Improved &lt;code&gt;normalize_nan&lt;/code&gt; and date truncate performance&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Query planning&lt;/strong&gt;: Cached query plans to avoid per-partition serialization overhead&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Memory efficiency&lt;/strong&gt;: Reduced GC pressure in protobuf serialization&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Hash operations&lt;/strong&gt;: Optimized complex-type hash implementations including murmur3 support for nested types&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Runtime efficiency&lt;/strong&gt;: Eliminated busy-polling of Tokio stream for plans without CometScan&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Metrics overhead&lt;/strong&gt;: Reduced timer and syscall overhead in native shuffle writer&lt;/li&gt;
&lt;/ul&gt;
&lt;h2 id="deprecations"&gt;Deprecations&lt;a class="headerlink" href="#deprecations" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;The &lt;code&gt;native_comet&lt;/code&gt; scan mode is now deprecated in favor of &lt;code&gt;native_iceberg_compat&lt;/code&gt; and will be removed in a future release. The &lt;code&gt;auto&lt;/code&gt; scan mode no longer falls back to &lt;code&gt;native_comet&lt;/code&gt;.&lt;/p&gt;
&lt;h2 id="compatibility"&gt;Compatibility&lt;a class="headerlink" href="#compatibility" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;This release upgrades to DataFusion 51, Arrow 57, and the latest iceberg-rust. The minimum supported Rust version is now 1.88.&lt;/p&gt;
&lt;p&gt;Supported platforms include Spark 3.4.3, 3.5.4-3.5.7, and Spark 4.0.x with various JDK and Scala combinations.&lt;/p&gt;
&lt;p&gt;The community encourages users to test Comet with existing Spark workloads and welcomes contributions to ongoing development.&lt;/p&gt;</content><category term="blog"/></entry><entry><title>Apache DataFusion 52.0.0 Released</title><link href="https://datafusion.apache.org/blog/2026/01/12/datafusion-52.0.0" rel="alternate"/><published>2026-01-12T00:00:00+00:00</published><updated>2026-01-12T00:00:00+00:00</updated><author><name>pmc</name></author><id>tag:datafusion.apache.org,2026-01-12:/blog/2026/01/12/datafusion-52.0.0</id><summary type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;p&gt;We are proud to announce the release of &lt;a href="https://crates.io/crates/datafusion/52.0.0"&gt;DataFusion 52.0.0&lt;/a&gt;. This post highlights
some of the major improvements since &lt;a href="https://datafusion.apache.org/blog/2025/11/25/datafusion-51.0.0/"&gt;DataFusion 51.0.0&lt;/a&gt;. The complete list of
changes is available in the &lt;a href="https://github.com/apache/datafusion/blob/branch-52/dev/changelog/52.0.0.md"&gt;changelog&lt;/a&gt;. Thanks to the &lt;a href="https://github.com/apache/datafusion/blob/branch-52/dev/changelog/52.0.0.md#credits"&gt;121 contributors&lt;/a&gt; for
making this release possible.&lt;/p&gt;
&lt;h2 id="performance-improvements"&gt;Performance Improvements 🚀&lt;a class="headerlink" href="#performance-improvements" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;We continue to …&lt;/p&gt;</summary><content type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;p&gt;We are proud to announce the release of &lt;a href="https://crates.io/crates/datafusion/52.0.0"&gt;DataFusion 52.0.0&lt;/a&gt;. This post highlights
some of the major improvements since &lt;a href="https://datafusion.apache.org/blog/2025/11/25/datafusion-51.0.0/"&gt;DataFusion 51.0.0&lt;/a&gt;. The complete list of
changes is available in the &lt;a href="https://github.com/apache/datafusion/blob/branch-52/dev/changelog/52.0.0.md"&gt;changelog&lt;/a&gt;. Thanks to the &lt;a href="https://github.com/apache/datafusion/blob/branch-52/dev/changelog/52.0.0.md#credits"&gt;121 contributors&lt;/a&gt; for
making this release possible.&lt;/p&gt;
&lt;h2 id="performance-improvements"&gt;Performance Improvements 🚀&lt;a class="headerlink" href="#performance-improvements" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;We continue to make significant performance improvements in DataFusion as explained below.&lt;/p&gt;
&lt;h3 id="faster-case-expressions"&gt;Faster &lt;code&gt;CASE&lt;/code&gt; Expressions&lt;a class="headerlink" href="#faster-case-expressions" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;DataFusion 52 has lookup-table-based evaluation for certain &lt;code&gt;CASE&lt;/code&gt; expressions
to avoid repeated evaluation for accelerating common ETL patterns such as&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-sql"&gt;CASE company
    WHEN 1 THEN 'Apple'
    WHEN 5 THEN 'Samsung'
    WHEN 2 THEN 'Motorola'
    WHEN 3 THEN 'LG'
    ELSE 'Other'
END
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;This is the final work in our &lt;code&gt;CASE&lt;/code&gt; performance epic (&lt;a href="https://github.com/apache/datafusion/issues/18075"&gt;#18075&lt;/a&gt;), which has
improved &lt;code&gt;CASE&lt;/code&gt; evaluation significantly. Related PRs &lt;a href="https://github.com/apache/datafusion/pull/18183"&gt;#18183&lt;/a&gt;. Thanks to
&lt;a href="https://github.com/rluvaton"&gt;rluvaton&lt;/a&gt; and &lt;a href="https://github.com/pepijnve"&gt;pepijnve&lt;/a&gt; for the implementation. See the
&lt;a href="/blog/2026/02/02/datafusion_case/"&gt;Optimizing SQL CASE Expression Evaluation&lt;/a&gt; blog post for more details.&lt;/p&gt;
&lt;h3 id="minmax-aggregate-dynamic-filters"&gt;&lt;code&gt;MIN&lt;/code&gt;/&lt;code&gt;MAX&lt;/code&gt; Aggregate Dynamic Filters&lt;a class="headerlink" href="#minmax-aggregate-dynamic-filters" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;DataFusion now creates dynamic filters for queries with &lt;code&gt;MIN&lt;/code&gt;/&lt;code&gt;MAX&lt;/code&gt; aggregates
that have filters, but no &lt;code&gt;GROUP BY&lt;/code&gt;. These dynamic filters are used during scan
to prune files and rows as tighter bounds are discovered during execution, as
explained in the &lt;a href="https://datafusion.apache.org/blog/2025/09/10/dynamic-filters/#hash-join-dynamic-filters"&gt;Dynamic Filtering Blog&lt;/a&gt;. For example, the following query:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-sql"&gt;SELECT min(l_shipdate)
FROM lineitem
WHERE l_returnflag = 'R';
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;Is now executed like this  &lt;/p&gt;
&lt;pre&gt;&lt;code class="language-sql"&gt;SELECT min(l_shipdate)
FROM lineitem
--  '__current_min' is updated dynamically during execution
WHERE l_returnflag = 'R' AND l_shipdate &amp;lt; __current_min;
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;Thanks to &lt;a href="https://github.com/2010YOUY01"&gt;2010YOUY01&lt;/a&gt; for implementing this feature, with reviews from
&lt;a href="https://github.com/martin-g"&gt;martin-g&lt;/a&gt;, &lt;a href="https://github.com/adriangb"&gt;adriangb&lt;/a&gt;, and &lt;a href="https://github.com/LiaCastaneda"&gt;LiaCastaneda&lt;/a&gt;. Related PRs: &lt;a href="https://github.com/apache/datafusion/pull/18644"&gt;#18644&lt;/a&gt;&lt;/p&gt;
&lt;h3 id="new-merge-join"&gt;New Merge Join&lt;a class="headerlink" href="#new-merge-join" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;DataFusion 52 includes a rewrite of the sort-merge join (SMJ) operator, with
speedups of three orders of magnitude in some pathological cases such as the
case in &lt;a href="https://github.com/apache/datafusion/issues/18487"&gt;#18487&lt;/a&gt;, which also affected &lt;a href="https://datafusion.apache.org/comet/"&gt;Apache Comet&lt;/a&gt; workloads. Benchmarks in
&lt;a href="https://github.com/apache/datafusion/pull/18875"&gt;#18875&lt;/a&gt; show dramatic gains for TPC-H Q21 (minutes to milliseconds) while
leaving other queries unchanged or modestly faster. Thanks to &lt;a href="https://github.com/mbutrovich"&gt;mbutrovich&lt;/a&gt; for
the implementation and reviews from &lt;a href="https://github.com/Dandandan"&gt;Dandandan&lt;/a&gt;.&lt;/p&gt;
&lt;h3 id="caching-improvements"&gt;Caching Improvements&lt;a class="headerlink" href="#caching-improvements" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;This release also includes several additional caching improvements.&lt;/p&gt;
&lt;p&gt;A new statistics cache for File Metadata avoids repeatedly (re)calculating
statistics for files. This significantly improves planning time
for certain queries. You can see the contents of the new cache using the
&lt;a href="https://datafusion.apache.org/user-guide/cli/functions.html#statistics-cache"&gt;statistics_cache&lt;/a&gt; function in the CLI:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-sql"&gt;select * from statistics_cache();
+------------------+---------------------+-----------------+------------------------+---------+-----------------+-------------+--------------------+-----------------------+
| path             | file_modified       | file_size_bytes | e_tag                  | version | num_rows        | num_columns | table_size_bytes   | statistics_size_bytes |
+------------------+---------------------+-----------------+------------------------+---------+-----------------+-------------+--------------------+-----------------------+
| .../hits.parquet | 2022-06-25T22:22:22 | 14779976446     | 0-5e24d1ee16380-370f48 | NULL    | Exact(99997497) | 105         | Exact(36445943240) | 0                     |
+------------------+---------------------+-----------------+------------------------+---------+-----------------+-------------+--------------------+-----------------------+
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;Thanks to &lt;a href="https://github.com/bharath-techie"&gt;bharath-techie&lt;/a&gt; and &lt;a href="https://github.com/nuno-faria"&gt;nuno-faria&lt;/a&gt; for implementing the statistics cache,
with reviews from &lt;a href="https://github.com/martin-g"&gt;martin-g&lt;/a&gt;, &lt;a href="https://github.com/alamb"&gt;alamb&lt;/a&gt;, and &lt;a href="https://github.com/alchemist51"&gt;alchemist51&lt;/a&gt;.
Related PRs: &lt;a href="https://github.com/apache/datafusion/pull/18971"&gt;#18971&lt;/a&gt;, &lt;a href="https://github.com/apache/datafusion/pull/19054"&gt;#19054&lt;/a&gt;&lt;/p&gt;
&lt;p&gt;A prefix-aware list-files cache accelerates evaluating partition predicates for
Hive partitioned tables.&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-sql"&gt;-- Read the hive partitioned dataset from Overture Maps (100s of Parquet files)
CREATE EXTERNAL TABLE overturemaps
STORED AS PARQUET LOCATION 's3://overturemaps-us-west-2/release/2025-12-17.0/';
-- Find all files where the path contains `theme=base without requiring another LIST call
select count(*) from overturemaps where theme='base';
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;You can see the
contents of the new cache using the &lt;a href="https://datafusion.apache.org/user-guide/cli/functions.html#list-files-cache"&gt;list_files_cache&lt;/a&gt; function in the CLI:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-sql"&gt;create external table overturemaps
stored as parquet
location 's3://overturemaps-us-west-2/release/2025-12-17.0/theme=base/type=infrastructure';
0 row(s) fetched.
&amp;gt; select table, path, metadata_size_bytes, expires_in, unnest(metadata_list)['file_size_bytes'] as file_size_bytes, unnest(metadata_list)['e_tag'] as e_tag from list_files_cache() limit 10;
+--------------+-----------------------------------------------------+---------------------+-----------------------------------+-----------------+---------------------------------------+
| table        | path                                                | metadata_size_bytes | expires_in                        | file_size_bytes | e_tag                                 |
+--------------+-----------------------------------------------------+---------------------+-----------------------------------+-----------------+---------------------------------------+
| overturemaps | release/2025-12-17.0/theme=base/type=infrastructure | 2750                | 0 days 0 hours 0 mins 25.264 secs | 999055952       | "35fc8fbe8400960b54c66fbb408c48e8-60" |
| overturemaps | release/2025-12-17.0/theme=base/type=infrastructure | 2750                | 0 days 0 hours 0 mins 25.264 secs | 975592768       | "8a16e10b722681cdc00242564b502965-59" |
...
| overturemaps | release/2025-12-17.0/theme=base/type=infrastructure | 2750                | 0 days 0 hours 0 mins 25.264 secs | 1016732378      | "6d70857a0473ed9ed3fc6e149814168b-61" |
| overturemaps | release/2025-12-17.0/theme=base/type=infrastructure | 2750                | 0 days 0 hours 0 mins 25.264 secs | 991363784       | "c9cafb42fcbb413f851691c895dd7c2b-60" |
| overturemaps | release/2025-12-17.0/theme=base/type=infrastructure | 2750                | 0 days 0 hours 0 mins 25.264 secs | 1032469715      | "7540252d0d67158297a67038a3365e0f-62" |
+--------------+-----------------------------------------------------+---------------------+-----------------------------------+-----------------+---------------------------------------+
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;Thanks to &lt;a href="https://github.com/BlakeOrth"&gt;BlakeOrth&lt;/a&gt; and &lt;a href="https://github.com/Yuvraj-cyborg"&gt;Yuvraj-cyborg&lt;/a&gt; for implementing the list-files cache work,
with reviews from &lt;a href="https://github.com/gabotechs"&gt;gabotechs&lt;/a&gt;, &lt;a href="https://github.com/alamb"&gt;alamb&lt;/a&gt;, &lt;a href="https://github.com/alchemist51"&gt;alchemist51&lt;/a&gt;, &lt;a href="https://github.com/martin-g"&gt;martin-g&lt;/a&gt;, and &lt;a href="https://github.com/BlakeOrth"&gt;BlakeOrth&lt;/a&gt;.
Related PRs: &lt;a href="https://github.com/apache/datafusion/pull/18146"&gt;#18146&lt;/a&gt;, &lt;a href="https://github.com/apache/datafusion/pull/18855"&gt;#18855&lt;/a&gt;, &lt;a href="https://github.com/apache/datafusion/pull/19366"&gt;#19366&lt;/a&gt;, &lt;a href="https://github.com/apache/datafusion/pull/19298"&gt;#19298&lt;/a&gt;, &lt;/p&gt;
&lt;h3 id="improved-hash-join-filter-pushdown"&gt;Improved Hash Join Filter Pushdown&lt;a class="headerlink" href="#improved-hash-join-filter-pushdown" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;Starting in DataFusion 51, filtering information from &lt;code&gt;HashJoinExec&lt;/code&gt; is passed
dynamically to scans, as explained in the &lt;a href="https://datafusion.apache.org/blog/2025/09/10/dynamic-filters/#hash-join-dynamic-filters"&gt;Dynamic Filtering Blog&lt;/a&gt; using a
technique referred to as &lt;a href="https://dl.acm.org/doi/10.1109/ICDE.2008.4497486"&gt;Sideways Information Passing&lt;/a&gt; in Database research
literature. The initial implementation passed min/max values for the join keys.
DataFusion 52 extends the optimization (&lt;a href="https://github.com/apache/datafusion/issues/17171"&gt;#17171&lt;/a&gt; / &lt;a href="https://github.com/apache/datafusion/pull/18393"&gt;#18393&lt;/a&gt;) to pass the
contents of the build side hash map. These filters are evaluated on the probe
side scan to prune files, row groups, and individual rows. When the build side
contains &lt;code&gt;20&lt;/code&gt; or fewer rows (configurable) the contents of the hash map are
transformed to an &lt;code&gt;IN&lt;/code&gt; expression and used for &lt;a href="https://docs.rs/datafusion/latest/datafusion/physical_optimizer/pruning/struct.PruningPredicate.html"&gt;statistics-based pruning&lt;/a&gt; which
can avoid reading entire files or row groups that contain no matching join keys.
Thanks to &lt;a href="https://github.com/adriangb"&gt;adriangb&lt;/a&gt; for implementing this feature, with reviews from
&lt;a href="https://github.com/LiaCastaneda"&gt;LiaCastaneda&lt;/a&gt;, &lt;a href="https://github.com/asolimando"&gt;asolimando&lt;/a&gt;, &lt;a href="https://github.com/comphead"&gt;comphead&lt;/a&gt;, and &lt;a href="https://github.com/mbutrovich"&gt;mbutrovich&lt;/a&gt;.&lt;/p&gt;
&lt;h2 id="major-features"&gt;Major Features ✨&lt;a class="headerlink" href="#major-features" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;h3 id="arrow-ipc-stream-file-support"&gt;Arrow IPC Stream file support&lt;a class="headerlink" href="#arrow-ipc-stream-file-support" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;DataFusion can now read Arrow IPC stream files (&lt;a href="https://github.com/apache/datafusion/pull/18457"&gt;#18457&lt;/a&gt;). This expands
interoperability with systems that emit Arrow streams directly, making it
simpler to ingest Arrow-native data without conversion. Thanks to &lt;a href="https://github.com/corasaurus-hex"&gt;corasaurus-hex&lt;/a&gt;
for implementing this feature, with reviews from &lt;a href="https://github.com/martin-g"&gt;martin-g&lt;/a&gt;, &lt;a href="https://github.com/Jefffrey"&gt;Jefffrey&lt;/a&gt;,
&lt;a href="https://github.com/jdcasale"&gt;jdcasale&lt;/a&gt;, &lt;a href="https://github.com/2010YOUY01"&gt;2010YOUY01&lt;/a&gt;, and &lt;a href="https://github.com/timsaucer"&gt;timsaucer&lt;/a&gt;.&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-sql"&gt;CREATE EXTERNAL TABLE ipc_events
STORED AS ARROW
LOCATION 's3://bucket/events.arrow';
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;Related PRs: &lt;a href="https://github.com/apache/datafusion/pull/18457"&gt;#18457&lt;/a&gt;&lt;/p&gt;
&lt;h3 id="more-extensible-sql-planning-with-relationplanner"&gt;More Extensible SQL Planning with &lt;code&gt;RelationPlanner&lt;/code&gt;&lt;a class="headerlink" href="#more-extensible-sql-planning-with-relationplanner" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;DataFusion now has an API for extending the SQL planner for relations, as
explained in the &lt;a href="https://datafusion.apache.org/blog/2026/01/12/extending-sql/"&gt;Extending SQL in DataFusion Blog&lt;/a&gt;. In addition to the existing
expression and types extension points, this new API now allows extending &lt;code&gt;FROM&lt;/code&gt;
clauses. Using these APIs it is straightforward to provide SQL support for
almost any dialect, including vendor-specific syntax. Example use cases include:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-sql"&gt;-- Postgres-style JSON operators
SELECT payload-&amp;gt;'user'-&amp;gt;&amp;gt;'id' FROM logs;
-- MySQL-specific types
SELECT DATETIME '2001-01-01 18:00:00';
-- Statistical sampling
SELECT * FROM sensor_data TABLESAMPLE BERNOULLI(10 PERCENT);
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;Thanks to &lt;a href="https://github.com/geoffreyclaude"&gt;geoffreyclaude&lt;/a&gt; for implementing relation planner extensions, and to
&lt;a href="https://github.com/theirix"&gt;theirix&lt;/a&gt;, &lt;a href="https://github.com/alamb"&gt;alamb&lt;/a&gt;, &lt;a href="https://github.com/NGA-TRAN"&gt;NGA-TRAN&lt;/a&gt;, and &lt;a href="https://github.com/gabotechs"&gt;gabotechs&lt;/a&gt; for reviews and feedback on the
design. Related PRs: &lt;a href="https://github.com/apache/datafusion/pull/17843"&gt;#17843&lt;/a&gt;&lt;/p&gt;
&lt;h3 id="expression-evaluation-pushdown-to-scans"&gt;Expression Evaluation Pushdown to Scans&lt;a class="headerlink" href="#expression-evaluation-pushdown-to-scans" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;DataFusion now pushes down expression evaluation into TableProviders using 
&lt;a href="https://docs.rs/datafusion/52.0.0/datafusion/physical_expr_adapter/trait.PhysicalExprAdapter.html"&gt;PhysicalExprAdapter&lt;/a&gt;, replacing the older SchemaAdapter approach (&lt;a href="https://github.com/apache/datafusion/issues/14993"&gt;#14993&lt;/a&gt;,
&lt;a href="https://github.com/apache/datafusion/issues/16800"&gt;#16800&lt;/a&gt;). Predicates and expressions can now be customized for each
individual file schema, opening additional optimization such as support for
&lt;a href="https://github.com/apache/datafusion/issues/16116"&gt;Variant shredding&lt;/a&gt;. Thanks to &lt;a href="https://github.com/adriangb"&gt;adriangb&lt;/a&gt; for implementing PhysicalExprAdapter
and reworking pushdown to use it. Related PRs: &lt;a href="https://github.com/apache/datafusion/pull/18998"&gt;#18998&lt;/a&gt;, &lt;a href="https://github.com/apache/datafusion/pull/19345"&gt;#19345&lt;/a&gt;&lt;/p&gt;
&lt;h3 id="sort-pushdown-to-scans"&gt;Sort Pushdown to Scans&lt;a class="headerlink" href="#sort-pushdown-to-scans" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;DataFusion can now push sorts into data sources (&lt;a href="https://github.com/apache/datafusion/issues/10433"&gt;#10433&lt;/a&gt;, &lt;a href="https://github.com/apache/datafusion/pull/19064"&gt;#19064&lt;/a&gt;).
This allows table provider implementations to optimize based on
sort knowledge for certain query patterns. For example, the provided Parquet
data source now reverses the scan order of row groups and files when queried
for the opposite of the file's natural sort (e.g. &lt;code&gt;DESC&lt;/code&gt; when the files are sorted &lt;code&gt;ASC&lt;/code&gt;).
This reversal, combined with dynamic filtering, allows top-K queries with &lt;code&gt;LIMIT&lt;/code&gt;
on pre-sorted data to find the requested rows very quickly, pruning more files and row groups
without even scanning them. We have seen a ~30x performance improvement on
benchmark queries with pre-sorted data.
Thanks to &lt;a href="https://github.com/zhuqi-lucas"&gt;zhuqi-lucas&lt;/a&gt; and &lt;a href="https://github.com/xudong963"&gt;xudong963&lt;/a&gt; for this feature, with reviews from
&lt;a href="https://github.com/martin-g"&gt;martin-g&lt;/a&gt;, &lt;a href="https://github.com/adriangb"&gt;adriangb&lt;/a&gt;, and &lt;a href="https://github.com/alamb"&gt;alamb&lt;/a&gt;.&lt;/p&gt;
&lt;h3 id="tableprovider-supports-delete-and-update-statements"&gt;&lt;code&gt;TableProvider&lt;/code&gt; supports &lt;code&gt;DELETE&lt;/code&gt; and &lt;code&gt;UPDATE&lt;/code&gt; statements&lt;a class="headerlink" href="#tableprovider-supports-delete-and-update-statements" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;The &lt;a href="https://docs.rs/datafusion/52.0.0/datafusion/datasource/trait.TableProvider.html"&gt;TableProvider&lt;/a&gt; trait now includes hooks for &lt;code&gt;DELETE&lt;/code&gt; and &lt;code&gt;UPDATE&lt;/code&gt;
statements and the basic MemTable implements them (&lt;a href="https://github.com/apache/datafusion/pull/19142"&gt;#19142&lt;/a&gt;). This lets
downstream implementations and storage engines plug in their own mutation logic.
See &lt;a href="https://docs.rs/datafusion/52.0.0/datafusion/datasource/trait.TableProvider.html#method.delete_from"&gt;TableProvider::delete_from&lt;/a&gt; and &lt;a href="https://docs.rs/datafusion/52.0.0/datafusion/datasource/trait.TableProvider.html#method.update"&gt;TableProvider::update&lt;/a&gt; for more details.&lt;/p&gt;
&lt;p&gt;Example:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-sql"&gt;DELETE FROM mem_table WHERE status = 'obsolete';
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;Thanks to &lt;a href="https://github.com/ethan-tyler"&gt;ethan-tyler&lt;/a&gt; for the implementation and &lt;a href="https://github.com/alamb"&gt;alamb&lt;/a&gt; and &lt;a href="https://github.com/adriangb"&gt;adriangb&lt;/a&gt; for
reviews.&lt;/p&gt;
&lt;h3 id="coalescebatchesexec-removed"&gt;&lt;code&gt;CoalesceBatchesExec&lt;/code&gt; Removed&lt;a class="headerlink" href="#coalescebatchesexec-removed" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;The standalone &lt;code&gt;CoalesceBatchesExec&lt;/code&gt; operator existed to ensure batches were
large enough for subsequent vectorized execution, and was inserted after
filter-like operators such as &lt;code&gt;FilterExec&lt;/code&gt;, &lt;code&gt;HashJoinExec&lt;/code&gt;, and
&lt;code&gt;RepartitionExec&lt;/code&gt;. However, using a separate operator also blocks other
optimizations such as pushing &lt;code&gt;LIMIT&lt;/code&gt; through joins and made optimizer rules
more complex. In this release, we  integrated the coalescing into the operators
themselves (&lt;a href="https://github.com/apache/datafusion/issues/18779"&gt;#18779&lt;/a&gt;) using Arrow's &lt;a href="https://docs.rs/arrow/57.2.0/arrow/compute/kernels/coalesce/"&gt;coalesce kernel&lt;/a&gt;. This reduces plan
complexity while keeping batch sizes efficient, and allows additional focused
optimization work in the Arrow kernel, such as &lt;a href="https://github.com/Dandandan"&gt;Dandandan&lt;/a&gt;'s recent work with
filtering in &lt;a href="https://github.com/apache/arrow-rs/pull/8951"&gt;arrow-rs/#8951&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;Related PRs: &lt;a href="https://github.com/apache/datafusion/pull/18540"&gt;#18540&lt;/a&gt;, &lt;a href="https://github.com/apache/datafusion/pull/18604"&gt;#18604&lt;/a&gt;, &lt;a href="https://github.com/apache/datafusion/pull/18630"&gt;#18630&lt;/a&gt;, &lt;a href="https://github.com/apache/datafusion/pull/18972"&gt;#18972&lt;/a&gt;, &lt;a href="https://github.com/apache/datafusion/pull/19002"&gt;#19002&lt;/a&gt;, &lt;a href="https://github.com/apache/datafusion/pull/19342"&gt;#19342&lt;/a&gt;, &lt;a href="https://github.com/apache/datafusion/pull/19239"&gt;#19239&lt;/a&gt;
Thanks to &lt;a href="https://github.com/Tim-53"&gt;Tim-53&lt;/a&gt;, &lt;a href="https://github.com/Dandandan"&gt;Dandandan&lt;/a&gt;, &lt;a href="https://github.com/jizezhang"&gt;jizezhang&lt;/a&gt;, and &lt;a href="https://github.com/feniljain"&gt;feniljain&lt;/a&gt; for implementing
this feature, with reviews from &lt;a href="https://github.com/Jefffrey"&gt;Jefffrey&lt;/a&gt;, &lt;a href="https://github.com/alamb"&gt;alamb&lt;/a&gt;, &lt;a href="https://github.com/martin-g"&gt;martin-g&lt;/a&gt;,
&lt;a href="https://github.com/geoffreyclaude"&gt;geoffreyclaude&lt;/a&gt;, &lt;a href="https://github.com/milenkovicm"&gt;milenkovicm&lt;/a&gt;, and &lt;a href="https://github.com/jizezhang"&gt;jizezhang&lt;/a&gt;.&lt;/p&gt;
&lt;h2 id="upgrade-guide-and-changelog"&gt;Upgrade Guide and Changelog&lt;a class="headerlink" href="#upgrade-guide-and-changelog" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;As always, upgrading to 52.0.0 should be straightforward for most users. Please review the
&lt;a href="https://datafusion.apache.org/library-user-guide/upgrading.html"&gt;Upgrade Guide&lt;/a&gt;
for details on breaking changes and code snippets to help with the transition.
For a comprehensive list of all changes, please refer to the &lt;a href="https://github.com/apache/datafusion/blob/branch-52/dev/changelog/52.0.0.md"&gt;changelog&lt;/a&gt;.&lt;/p&gt;
&lt;h2 id="about-datafusion"&gt;About DataFusion&lt;a class="headerlink" href="#about-datafusion" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;&lt;a href="https://datafusion.apache.org/"&gt;Apache DataFusion&lt;/a&gt; is an extensible query engine, written in &lt;a href="https://www.rust-lang.org/"&gt;Rust&lt;/a&gt;, that uses
&lt;a href="https://arrow.apache.org"&gt;Apache Arrow&lt;/a&gt; as its in-memory format. DataFusion is used by developers to
create new, fast, data-centric systems such as databases, dataframe libraries,
and machine learning and streaming applications. While &lt;a href="https://datafusion.apache.org/user-guide/introduction.html#project-goals"&gt;DataFusion's primary
design goal&lt;/a&gt; is to accelerate the creation of other data-centric systems, it
provides a reasonable experience directly out of the box as a &lt;a href="https://datafusion.apache.org/user-guide/dataframe.html"&gt;dataframe
library&lt;/a&gt;, &lt;a href="https://datafusion.apache.org/python/"&gt;Python library&lt;/a&gt;, and &lt;a href="https://datafusion.apache.org/user-guide/cli/"&gt;command-line SQL tool&lt;/a&gt;.&lt;/p&gt;
&lt;h2 id="how-to-get-involved"&gt;How to Get Involved&lt;a class="headerlink" href="#how-to-get-involved" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;DataFusion is not a project built or driven by a single person, company, or
foundation. Rather, our community of users and contributors works together to
build a shared technology that none of us could have built alone.&lt;/p&gt;
&lt;p&gt;If you are interested in joining us, we would love to have you. You can try out
DataFusion on some of your own data and projects and let us know how it goes,
contribute suggestions, documentation, bug reports, or a PR with documentation,
tests, or code. A list of open issues suitable for beginners is &lt;a href="https://github.com/apache/arrow-datafusion/issues?q=is%3Aissue+is%3Aopen+label%3A%22good+first+issue%22"&gt;here&lt;/a&gt;, and you
can find out how to reach us on the &lt;a href="https://datafusion.apache.org/contributor-guide/communication.html"&gt;communication doc&lt;/a&gt;.&lt;/p&gt;</content><category term="blog"/></entry><entry><title>Extending SQL in DataFusion: from -&gt;&gt; to TABLESAMPLE</title><link href="https://datafusion.apache.org/blog/2026/01/12/extending-sql" rel="alternate"/><published>2026-01-12T00:00:00+00:00</published><updated>2026-01-12T00:00:00+00:00</updated><author><name>Geoffrey Claude (Datadog)</name></author><id>tag:datafusion.apache.org,2026-01-12:/blog/2026/01/12/extending-sql</id><summary type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;p&gt;If you embed &lt;a href="https://datafusion.apache.org/"&gt;DataFusion&lt;/a&gt; in your product, your users will eventually run SQL that DataFusion does not recognize. Not because the query is unreasonable, but because SQL in practice includes many dialects and system-specific statements.&lt;/p&gt;
&lt;p&gt;Suppose you store data as Parquet files on S3 and want users to attach an …&lt;/p&gt;</summary><content type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;p&gt;If you embed &lt;a href="https://datafusion.apache.org/"&gt;DataFusion&lt;/a&gt; in your product, your users will eventually run SQL that DataFusion does not recognize. Not because the query is unreasonable, but because SQL in practice includes many dialects and system-specific statements.&lt;/p&gt;
&lt;p&gt;Suppose you store data as Parquet files on S3 and want users to attach an external catalog to query them. DataFusion has &lt;code&gt;CREATE EXTERNAL TABLE&lt;/code&gt; for individual tables, but no built-in equivalent for catalogs. DuckDB has &lt;code&gt;ATTACH&lt;/code&gt;, SQLite has its own variant, and maybe you really want something even more flexible:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-sql"&gt;CREATE EXTERNAL CATALOG my_lake
STORED AS iceberg
LOCATION 's3://my-bucket/warehouse'
OPTIONS ('region' 'eu-west-1');
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;This syntax does not exist in DataFusion today, but you can add it.&lt;/p&gt;
&lt;hr/&gt;
&lt;p&gt;At the same time, many dialect gaps are smaller and show up in everyday queries:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-sql"&gt;-- Postgres-style JSON operators
SELECT payload-&amp;gt;'user'-&amp;gt;&amp;gt;'id' FROM logs;

-- MySQL-specific types
SELECT DATETIME '2001-01-01 18:00:00';

-- Statistical sampling
SELECT * FROM sensor_data TABLESAMPLE BERNOULLI(10 PERCENT);
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;You can implement all of these &lt;em&gt;without forking&lt;/em&gt; DataFusion:&lt;/p&gt;
&lt;ol&gt;
&lt;li&gt;&lt;strong&gt;Parse&lt;/strong&gt; new syntax (custom statements / dialect quirks)&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Plan&lt;/strong&gt; new semantics (expressions, types, FROM-clause constructs)&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Execute&lt;/strong&gt; new operators when rewrites are not sufficient&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;This post explains where and how to hook into each stage. For complete, working code, see the linked &lt;code&gt;datafusion-examples&lt;/code&gt;.&lt;/p&gt;
&lt;hr/&gt;
&lt;h2 id="parse-plan-execute"&gt;Parse → Plan → Execute&lt;a class="headerlink" href="#parse-plan-execute" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;DataFusion turns SQL into executable work in stages:&lt;/p&gt;
&lt;ol&gt;
&lt;li&gt;&lt;strong&gt;Parse&lt;/strong&gt;: SQL text is parsed into an AST (&lt;a href="https://docs.rs/sqlparser/latest/sqlparser/ast/enum.Statement.html"&gt;Statement&lt;/a&gt; from &lt;a href="https://github.com/sqlparser-rs/sqlparser-rs"&gt;sqlparser-rs&lt;/a&gt;)&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Logical planning&lt;/strong&gt;: &lt;a href="https://docs.rs/datafusion/latest/datafusion/sql/planner/struct.SqlToRel.html"&gt;SqlToRel&lt;/a&gt; converts the AST into a &lt;a href="https://docs.rs/datafusion/latest/datafusion/logical_expr/enum.LogicalPlan.html"&gt;LogicalPlan&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Physical planning&lt;/strong&gt;: The &lt;a href="https://docs.rs/datafusion/latest/datafusion/physical_planner/trait.PhysicalPlanner.html"&gt;PhysicalPlanner&lt;/a&gt; turns the logical plan into an &lt;a href="https://docs.rs/datafusion/latest/datafusion/physical_plan/trait.ExecutionPlan.html"&gt;ExecutionPlan&lt;/a&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;Each stage has extension points.&lt;/p&gt;
&lt;figure&gt;
&lt;img alt="DataFusion SQL processing pipeline: SQL String flows through Parser to AST, then SqlToRel (with Extension Planners) to LogicalPlan, then PhysicalPlanner to ExecutionPlan" class="img-fluid" src="/blog/images/extending-sql/architecture.svg" width="100%"/&gt;
&lt;figcaption&gt;
&lt;b&gt;Figure 1:&lt;/b&gt; SQL flows through three stages: parsing, logical planning (via &lt;code&gt;SqlToRel&lt;/code&gt;, where the Extension Planners hook in), and physical planning. Each stage has extension points: wrap the parser, implement planner traits, or add physical operators.
  &lt;/figcaption&gt;
&lt;/figure&gt;
&lt;p&gt;To choose the right extension point, look at where the query fails.&lt;/p&gt;
&lt;table class="table"&gt;
&lt;thead&gt;
&lt;tr&gt;
&lt;th&gt;What fails?&lt;/th&gt;
&lt;th&gt;What it looks like&lt;/th&gt;
&lt;th&gt;Where to hook in&lt;/th&gt;
&lt;/tr&gt;
&lt;/thead&gt;
&lt;tbody&gt;
&lt;tr&gt;
&lt;td&gt;Parsing&lt;/td&gt;
&lt;td&gt;&lt;code&gt;Expected: TABLE, found: CATALOG&lt;/code&gt;&lt;/td&gt;
&lt;td&gt;configure dialect or wrap &lt;code&gt;DFParser&lt;/code&gt;&lt;/td&gt;
&lt;/tr&gt;
&lt;tr&gt;
&lt;td&gt;Planning&lt;/td&gt;
&lt;td&gt;&lt;code&gt;This feature is not implemented: DATETIME&lt;/code&gt;&lt;/td&gt;
&lt;td&gt;&lt;code&gt;ExprPlanner&lt;/code&gt;, &lt;code&gt;TypePlanner&lt;/code&gt;, &lt;code&gt;RelationPlanner&lt;/code&gt;&lt;/td&gt;
&lt;/tr&gt;
&lt;tr&gt;
&lt;td&gt;Execution&lt;/td&gt;
&lt;td&gt;&lt;code&gt;No physical plan for TableSample&lt;/code&gt;&lt;/td&gt;
&lt;td&gt;&lt;code&gt;ExtensionPlanner&lt;/code&gt; (+ physical operator)&lt;/td&gt;
&lt;/tr&gt;
&lt;/tbody&gt;
&lt;/table&gt;
&lt;p&gt;We will follow that pipeline order.&lt;/p&gt;
&lt;hr/&gt;
&lt;h2 id="1-extending-parsing-wrapping-dfparser-for-custom-statements"&gt;1) Extending parsing: wrapping &lt;code&gt;DFParser&lt;/code&gt; for custom statements&lt;a class="headerlink" href="#1-extending-parsing-wrapping-dfparser-for-custom-statements" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;The &lt;code&gt;CREATE EXTERNAL CATALOG&lt;/code&gt; syntax from the introduction fails at the parser because DataFusion only recognizes &lt;code&gt;CREATE EXTERNAL TABLE&lt;/code&gt;. To support new statement-level syntax, you can &lt;strong&gt;wrap &lt;code&gt;DFParser&lt;/code&gt;&lt;/strong&gt;. Peek ahead &lt;strong&gt;in the token stream&lt;/strong&gt; to detect your custom syntax, handle it yourself, and delegate everything else to DataFusion.&lt;/p&gt;
&lt;p&gt;The &lt;a href="https://github.com/apache/datafusion/blob/main/datafusion-examples/examples/sql_ops/custom_sql_parser.rs"&gt;&lt;code&gt;custom_sql_parser.rs&lt;/code&gt;&lt;/a&gt; example demonstrates this pattern:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-rust"&gt;struct CustomParser&amp;lt;'a&amp;gt; { df_parser: DFParser&amp;lt;'a&amp;gt; }

impl&amp;lt;'a&amp;gt; CustomParser&amp;lt;'a&amp;gt; {
  pub fn parse_statement(&amp;amp;mut self) -&amp;gt; Result&amp;lt;CustomStatement&amp;gt; {
    // Peek tokens to detect CREATE EXTERNAL CATALOG
    if self.is_create_external_catalog() {
      return self.parse_create_external_catalog();
    }
    // Delegate everything else to DataFusion
    Ok(CustomStatement::DFStatement(Box::new(
      self.df_parser.parse_statement()?,
    )))
  }
}
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;You do not need to implement a full SQL parser. Reuse DataFusion's tokenizer and parser helpers to consume tokens, parse identifiers, and handle options—the example shows how.&lt;/p&gt;
&lt;p&gt;Once parsed, the simplest integration is to treat custom statements as &lt;strong&gt;application commands&lt;/strong&gt;:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-rust"&gt;match parser.parse_statement()? {
  CustomStatement::DFStatement(stmt) =&amp;gt; ctx.sql(&amp;amp;stmt.to_string()).await?,
  CustomStatement::CreateExternalCatalog(stmt) =&amp;gt; {
    handle_create_external_catalog(&amp;amp;ctx, stmt).await?
  }
}
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;This keeps the extension logic in your embedding application. The example includes a complete &lt;code&gt;handle_create_external_catalog&lt;/code&gt; that registers tables from a location into a catalog, making them queryable immediately.&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Full working example:&lt;/strong&gt; &lt;a href="https://github.com/apache/datafusion/blob/main/datafusion-examples/examples/sql_ops/custom_sql_parser.rs"&gt;&lt;code&gt;custom_sql_parser.rs&lt;/code&gt;&lt;/a&gt;&lt;/p&gt;
&lt;hr/&gt;
&lt;h2 id="2-extending-expression-semantics-exprplanner"&gt;2) Extending expression semantics: &lt;code&gt;ExprPlanner&lt;/code&gt;&lt;a class="headerlink" href="#2-extending-expression-semantics-exprplanner" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;Once SQL &lt;em&gt;parses&lt;/em&gt;, the next failure is often that DataFusion does not know what a particular expression means.&lt;/p&gt;
&lt;p&gt;This is where dialect differences show up in day-to-day queries: operators like Postgres JSON arrows, vendor-specific functions, or small syntactic sugar that users expect to keep working when you switch engines.&lt;/p&gt;
&lt;p&gt;&lt;code&gt;ExprPlanner&lt;/code&gt; lets you define how specific SQL expressions become DataFusion &lt;code&gt;Expr&lt;/code&gt;. Common examples:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;Non-standard operators (JSON / geometry / regex operators)&lt;/li&gt;
&lt;li&gt;Custom function syntaxes&lt;/li&gt;
&lt;li&gt;Special identifier behavior&lt;/li&gt;
&lt;/ul&gt;
&lt;h3 id="example-postgres-json-operators-"&gt;Example: Postgres JSON operators (&lt;code&gt;-&amp;gt;&lt;/code&gt;, &lt;code&gt;-&amp;gt;&amp;gt;&lt;/code&gt;)&lt;a class="headerlink" href="#example-postgres-json-operators-" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;The Postgres &lt;code&gt;-&amp;gt;&lt;/code&gt; operator is a good illustration because it is widely used and parses only under the PostgreSQL dialect.&lt;/p&gt;
&lt;p&gt;Configure the dialect:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-rust"&gt;let config = SessionConfig::new()
    .set_str("datafusion.sql_parser.dialect", "postgres");
let ctx = SessionContext::new_with_config(config);
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;Then implement &lt;code&gt;ExprPlanner&lt;/code&gt; to map the parsed operator (&lt;code&gt;BinaryOperator::Arrow&lt;/code&gt;) to DataFusion semantics:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-rust"&gt;fn plan_binary_op(&amp;amp;self, expr: RawBinaryExpr, _schema: &amp;amp;DFSchema)
  -&amp;gt; Result&amp;lt;PlannerResult&amp;lt;RawBinaryExpr&amp;gt;&amp;gt; {
  match expr.op {
    BinaryOperator::Arrow =&amp;gt; Ok(Planned(/* your Expr */)),
    _ =&amp;gt; Ok(Original(expr)),
  }
}
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;Return &lt;code&gt;Planned(...)&lt;/code&gt; when you handled the expression; return &lt;code&gt;Original(...)&lt;/code&gt; to pass it to the next planner.&lt;/p&gt;
&lt;p&gt;For a complete JSON implementation, see &lt;a href="https://github.com/datafusion-contrib/datafusion-functions-json"&gt;datafusion-functions-json&lt;/a&gt;. For a minimal end-to-end example in the DataFusion repo, see &lt;a href="https://github.com/apache/datafusion/blob/main/datafusion/core/tests/user_defined/expr_planner.rs"&gt;&lt;code&gt;expr_planner_tests&lt;/code&gt;&lt;/a&gt;.&lt;/p&gt;
&lt;hr/&gt;
&lt;h2 id="3-extending-type-support-typeplanner"&gt;3) Extending type support: &lt;code&gt;TypePlanner&lt;/code&gt;&lt;a class="headerlink" href="#3-extending-type-support-typeplanner" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;After expressions, types are often the next thing to break. Schemas and DDL may reference types that DataFusion does not support out of the box, like MySQL's &lt;code&gt;DATETIME&lt;/code&gt;.&lt;/p&gt;
&lt;p&gt;Type planning tends to come up when interoperating with other systems. You want to accept DDL or infer schemas from external catalogs without forcing users to rewrite types.&lt;/p&gt;
&lt;p&gt;&lt;code&gt;TypePlanner&lt;/code&gt; maps SQL types to Arrow/DataFusion types:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-rust"&gt;impl TypePlanner for MyTypePlanner {
  fn plan_type(&amp;amp;self, sql_type: &amp;amp;ast::DataType) -&amp;gt; Result&amp;lt;Option&amp;lt;DataType&amp;gt;&amp;gt; {
    match sql_type {
      ast::DataType::Datetime(Some(3)) =&amp;gt; Ok(Some(DataType::Timestamp(TimeUnit::Millisecond, None))),
      _ =&amp;gt; Ok(None), // let the default planner handle it
    }
  }
}
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;It is installed when building session state:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-rust"&gt;let state = SessionStateBuilder::new()
  .with_default_features()
  .with_type_planner(Arc::new(MyTypePlanner))
  .build();
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;Once installed, if your &lt;code&gt;CREATE EXTERNAL CATALOG&lt;/code&gt; statement exposes tables with MySQL types, DataFusion can interpret them correctly.&lt;/p&gt;
&lt;hr/&gt;
&lt;h2 id="4-extending-the-from-clause-relationplanner"&gt;4) Extending the FROM clause: &lt;code&gt;RelationPlanner&lt;/code&gt;&lt;a class="headerlink" href="#4-extending-the-from-clause-relationplanner" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;Some extensions change what a &lt;em&gt;relation&lt;/em&gt; means, not just expressions or types. &lt;code&gt;RelationPlanner&lt;/code&gt; (available starting in DataFusion 52) intercepts FROM-clause constructs while SQL is being converted into a &lt;code&gt;LogicalPlan&lt;/code&gt;.&lt;/p&gt;
&lt;p&gt;Once you have &lt;code&gt;RelationPlanner&lt;/code&gt;, there are two main approaches to implementing your extension.&lt;/p&gt;
&lt;h3 id="strategy-a-rewrite-to-existing-operators-pivot-unpivot"&gt;Strategy A: rewrite to existing operators (PIVOT / UNPIVOT)&lt;a class="headerlink" href="#strategy-a-rewrite-to-existing-operators-pivot-unpivot" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;If you can translate your syntax into relational algebra that DataFusion already supports, you can implement the feature with &lt;strong&gt;no custom physical operator&lt;/strong&gt;.&lt;/p&gt;
&lt;p&gt;&lt;code&gt;PIVOT&lt;/code&gt; rotates rows into columns, and &lt;code&gt;UNPIVOT&lt;/code&gt; does the reverse. Neither requires new execution logic: &lt;code&gt;PIVOT&lt;/code&gt; is just &lt;code&gt;GROUP BY&lt;/code&gt; with &lt;code&gt;CASE&lt;/code&gt; expressions, and &lt;code&gt;UNPIVOT&lt;/code&gt; is a &lt;code&gt;UNION ALL&lt;/code&gt; of each column. The planner rewrites them accordingly:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-rust"&gt;match relation {
  TableFactor::Pivot { .. } =&amp;gt; /* rewrite to GROUP BY + CASE */,
  TableFactor::Unpivot { .. } =&amp;gt; /* rewrite to UNION ALL */,
  other =&amp;gt; Original(other),
}
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;Because the output is a standard &lt;code&gt;LogicalPlan&lt;/code&gt;, DataFusion's usual optimization and physical planning apply automatically.&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Full working example:&lt;/strong&gt; &lt;a href="https://github.com/apache/datafusion/blob/main/datafusion-examples/examples/relation_planner/pivot_unpivot.rs"&gt;&lt;code&gt;pivot_unpivot.rs&lt;/code&gt;&lt;/a&gt;&lt;/p&gt;
&lt;h3 id="strategy-b-custom-logical-physical-tablesample"&gt;Strategy B: custom logical + physical (TABLESAMPLE)&lt;a class="headerlink" href="#strategy-b-custom-logical-physical-tablesample" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;Sometimes rewriting is not sufficient. &lt;code&gt;TABLESAMPLE&lt;/code&gt; returns a random subset of rows from a table and is useful for approximations or debugging on large datasets. Because it requires runtime randomness, you cannot express it as a rewrite to existing operators. Instead, you need a custom logical node and physical operator to execute it.&lt;/p&gt;
&lt;p&gt;The approach (shown in &lt;a href="https://github.com/apache/datafusion/blob/main/datafusion-examples/examples/relation_planner/table_sample.rs"&gt;&lt;code&gt;table_sample.rs&lt;/code&gt;&lt;/a&gt;):&lt;/p&gt;
&lt;ol&gt;
&lt;li&gt;&lt;code&gt;RelationPlanner&lt;/code&gt; recognizes &lt;code&gt;TABLESAMPLE&lt;/code&gt; and produces a custom logical node&lt;/li&gt;
&lt;li&gt;That node gets wrapped in &lt;code&gt;LogicalPlan::Extension&lt;/code&gt;&lt;/li&gt;
&lt;li&gt;&lt;code&gt;ExtensionPlanner&lt;/code&gt; converts it to a custom &lt;code&gt;ExecutionPlan&lt;/code&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;In code:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-rust"&gt;// Logical planning: FROM t TABLESAMPLE (...)  -&amp;gt;  LogicalPlan::Extension(...)
let plan = LogicalPlan::Extension(Extension { node: Arc::new(TableSamplePlanNode { /* ... */ }) });
&lt;/code&gt;&lt;/pre&gt;
&lt;pre&gt;&lt;code class="language-rust"&gt;// Physical planning: TableSamplePlanNode  -&amp;gt;  SampleExec
if let Some(sample_node) = node.as_any().downcast_ref::&amp;lt;TableSamplePlanNode&amp;gt;() {
  return Ok(Some(Arc::new(SampleExec::try_new(input, /* bounds, seed */)?)));
}
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;This is the general pattern for custom FROM constructs that need runtime behavior.&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Full working example:&lt;/strong&gt; &lt;a href="https://github.com/apache/datafusion/blob/main/datafusion-examples/examples/relation_planner/table_sample.rs"&gt;&lt;code&gt;table_sample.rs&lt;/code&gt;&lt;/a&gt;&lt;/p&gt;
&lt;h3 id="background-origin-of-the-api"&gt;Background: Origin of the API&lt;a class="headerlink" href="#background-origin-of-the-api" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;&lt;code&gt;RelationPlanner&lt;/code&gt; originally came out of trying to build &lt;code&gt;MATCH_RECOGNIZE&lt;/code&gt; support in DataFusion as a Datadog hackathon project. &lt;code&gt;MATCH_RECOGNIZE&lt;/code&gt; is a complex SQL feature for detecting patterns in sequences of rows, and it made sense to prototype as an extension first. At the time, DataFusion had no extension point at the right stage of SQL-to-rel planning to intercept and reinterpret relations.&lt;/p&gt;
&lt;p&gt;&lt;a href="https://github.com/theirix"&gt;@theirix&lt;/a&gt;'s &lt;code&gt;TABLESAMPLE&lt;/code&gt; work (&lt;a href="https://github.com/apache/datafusion/issues/13563"&gt;#13563&lt;/a&gt;, &lt;a href="https://github.com/apache/datafusion/pull/17633"&gt;#17633&lt;/a&gt;) demonstrated exactly where the gap was: their extension only worked when &lt;code&gt;TABLESAMPLE&lt;/code&gt; appeared at the query root and any &lt;code&gt;TABLESAMPLE&lt;/code&gt; inside a CTE or JOIN would error. That limitation motivated &lt;a href="https://github.com/apache/datafusion/pull/17843"&gt;#17843&lt;/a&gt;, which introduced &lt;code&gt;RelationPlanner&lt;/code&gt; to intercept relations at any nesting level. The same hook now supports &lt;code&gt;PIVOT&lt;/code&gt;, &lt;code&gt;UNPIVOT&lt;/code&gt;, &lt;code&gt;TABLESAMPLE&lt;/code&gt;, and can translate dialect-specific FROM-clause syntax (for example, bridging Trino constructs into DataFusion plans).&lt;/p&gt;
&lt;p&gt;This is how Datadog approaches compatibility work: build features in real systems first, then upstream the building blocks. A full &lt;code&gt;MATCH_RECOGNIZE&lt;/code&gt; extension is now in progress, built on top of &lt;code&gt;RelationPlanner&lt;/code&gt;, with the &lt;a href="https://github.com/apache/datafusion/blob/main/datafusion-examples/examples/relation_planner/match_recognize.rs"&gt;&lt;code&gt;match_recognize.rs&lt;/code&gt;&lt;/a&gt; example as a starting point.&lt;/p&gt;
&lt;hr/&gt;
&lt;h2 id="summary-the-extensibility-workflow"&gt;Summary: The Extensibility Workflow&lt;a class="headerlink" href="#summary-the-extensibility-workflow" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;DataFusion's SQL extensibility follows its processing pipeline. When building your own dialect extension, work incrementally:&lt;/p&gt;
&lt;ol&gt;
&lt;li&gt;&lt;strong&gt;Parse&lt;/strong&gt;: Use a parser wrapper to intercept custom syntax in the token stream. Produce either a standard &lt;code&gt;Statement&lt;/code&gt; or your own application-specific command.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Plan&lt;/strong&gt;: Implement the planning traits (&lt;code&gt;ExprPlanner&lt;/code&gt;, &lt;code&gt;TypePlanner&lt;/code&gt;, &lt;code&gt;RelationPlanner&lt;/code&gt;) to give your syntax meaning.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Execute&lt;/strong&gt;: Prefer rewrites to existing operators (like &lt;code&gt;PIVOT&lt;/code&gt; to &lt;code&gt;CASE&lt;/code&gt;). Only add custom physical operators via &lt;code&gt;ExtensionPlanner&lt;/code&gt; when you need specific runtime behavior like randomness or specialized I/O.&lt;/li&gt;
&lt;/ol&gt;
&lt;hr/&gt;
&lt;h2 id="debugging-tips"&gt;Debugging tips&lt;a class="headerlink" href="#debugging-tips" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;h3 id="print-the-logical-plan"&gt;Print the logical plan&lt;a class="headerlink" href="#print-the-logical-plan" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;pre&gt;&lt;code class="language-rust"&gt;let df = ctx.sql("SELECT * FROM t TABLESAMPLE (10 PERCENT)").await?;
println!("{}", df.logical_plan().display_indent());
&lt;/code&gt;&lt;/pre&gt;
&lt;h3 id="use-explain"&gt;Use &lt;a href="https://datafusion.apache.org/user-guide/sql/explain.html"&gt;&lt;code&gt;EXPLAIN&lt;/code&gt;&lt;/a&gt;&lt;a class="headerlink" href="#use-explain" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;pre&gt;&lt;code class="language-sql"&gt;EXPLAIN SELECT * FROM t TABLESAMPLE (10 PERCENT);
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;If your extension is not being invoked, it is usually visible in the logical plan first.&lt;/p&gt;
&lt;hr/&gt;
&lt;h2 id="when-hooks-arent-enough"&gt;When hooks aren't enough&lt;a class="headerlink" href="#when-hooks-arent-enough" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;While these extension points cover the majority of dialect needs, some deep architectural areas still have limited or no hooks. If you are working in these parts of the SQL surface area, you may need to contribute upstream:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;Statement-level planning: &lt;a href="https://github.com/apache/datafusion/blob/main/datafusion/sql/src/statement.rs"&gt;&lt;code&gt;statement.rs&lt;/code&gt;&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;JOIN planning: &lt;a href="https://github.com/apache/datafusion/blob/main/datafusion/sql/src/relation/join.rs"&gt;&lt;code&gt;relation/join.rs&lt;/code&gt;&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;TOP / FETCH clauses: &lt;a href="https://github.com/apache/datafusion/blob/main/datafusion/sql/src/select.rs"&gt;&lt;code&gt;select.rs&lt;/code&gt;&lt;/a&gt;, &lt;a href="https://github.com/apache/datafusion/blob/main/datafusion/sql/src/query.rs"&gt;&lt;code&gt;query.rs&lt;/code&gt;&lt;/a&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;hr/&gt;
&lt;h2 id="ideas-to-try"&gt;Ideas to try&lt;a class="headerlink" href="#ideas-to-try" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;If you want to experiment with these extension points, here are a few suggestions:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;Geometry operators (for example &lt;code&gt;@&amp;gt;&lt;/code&gt;, &lt;code&gt;&amp;lt;@&lt;/code&gt;) via &lt;code&gt;ExprPlanner&lt;/code&gt;&lt;/li&gt;
&lt;li&gt;Oracle &lt;code&gt;NUMBER&lt;/code&gt; or SQL Server &lt;code&gt;MONEY&lt;/code&gt; via &lt;code&gt;TypePlanner&lt;/code&gt;&lt;/li&gt;
&lt;li&gt;&lt;code&gt;JSON_TABLE&lt;/code&gt; or semantic-layer style relations via &lt;code&gt;RelationPlanner&lt;/code&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;hr/&gt;
&lt;h2 id="see-also"&gt;See also&lt;a class="headerlink" href="#see-also" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;ul&gt;
&lt;li&gt;Extending SQL Guide: &lt;a href="https://datafusion.apache.org/library-user-guide/extending-sql.html"&gt;Extending SQL Guide&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;Parser wrapping example: &lt;a href="https://github.com/apache/datafusion/blob/main/datafusion-examples/examples/sql_ops/custom_sql_parser.rs"&gt;&lt;code&gt;custom_sql_parser.rs&lt;/code&gt;&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;
&lt;p&gt;RelationPlanner examples:&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;
&lt;p&gt;&lt;code&gt;PIVOT&lt;/code&gt; / &lt;code&gt;UNPIVOT&lt;/code&gt;: &lt;a href="https://github.com/apache/datafusion/blob/main/datafusion-examples/examples/relation_planner/pivot_unpivot.rs"&gt;&lt;code&gt;pivot_unpivot.rs&lt;/code&gt;&lt;/a&gt;&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;
&lt;p&gt;&lt;code&gt;TABLESAMPLE&lt;/code&gt;: &lt;a href="https://github.com/apache/datafusion/blob/main/datafusion-examples/examples/relation_planner/table_sample.rs"&gt;&lt;code&gt;table_sample.rs&lt;/code&gt;&lt;/a&gt;&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;
&lt;p&gt;ExprPlanner test examples: &lt;a href="https://github.com/apache/datafusion/blob/main/datafusion/core/tests/user_defined/expr_planner.rs"&gt;&lt;code&gt;expr_planner_tests&lt;/code&gt;&lt;/a&gt;&lt;/p&gt;
&lt;/li&gt;
&lt;/ul&gt;
&lt;h2 id="acknowledgements"&gt;Acknowledgements&lt;a class="headerlink" href="#acknowledgements" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;Thank you to &lt;a href="https://github.com/jayzhan211"&gt;@jayzhan211&lt;/a&gt; for designing and implementing the original &lt;code&gt;ExprPlanner&lt;/code&gt; API (&lt;a href="https://github.com/apache/datafusion/pull/11180"&gt;#11180&lt;/a&gt;), to &lt;a href="https://github.com/goldmedal"&gt;@goldmedal&lt;/a&gt; for adding &lt;code&gt;TypePlanner&lt;/code&gt; (&lt;a href="https://github.com/apache/datafusion/pull/13294"&gt;#13294&lt;/a&gt;), and to &lt;a href="https://github.com/theirix"&gt;@theirix&lt;/a&gt; for the &lt;code&gt;TABLESAMPLE&lt;/code&gt; work (&lt;a href="https://github.com/apache/datafusion/issues/13563"&gt;#13563&lt;/a&gt;, &lt;a href="https://github.com/apache/datafusion/pull/17633"&gt;#17633&lt;/a&gt;) that helped shape &lt;code&gt;RelationPlanner&lt;/code&gt;. Thank you to &lt;a href="https://github.com/alamb"&gt;@alamb&lt;/a&gt; for driving DataFusion's extensibility philosophy and for feedback on this post.&lt;/p&gt;
&lt;h2 id="get-involved"&gt;Get Involved&lt;a class="headerlink" href="#get-involved" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;ul&gt;
&lt;li&gt;&lt;strong&gt;Try it out&lt;/strong&gt;: Implement one of the extension points and share your experience&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;File issues or join the conversation&lt;/strong&gt;: &lt;a href="https://github.com/apache/datafusion/"&gt;GitHub&lt;/a&gt; for bugs and feature requests, &lt;a href="https://datafusion.apache.org/contributor-guide/communication.html"&gt;Slack or Discord&lt;/a&gt; for discussion&lt;/li&gt;
&lt;/ul&gt;
&lt;!-- Reference links --&gt;</content><category term="blog"/></entry><entry><title>Optimizing Repartitions in DataFusion: How I Went From Database Noob to Core Contribution</title><link href="https://datafusion.apache.org/blog/2025/12/15/avoid-consecutive-repartitions" rel="alternate"/><published>2025-12-15T00:00:00+00:00</published><updated>2025-12-15T00:00:00+00:00</updated><author><name>Gene Bordegaray</name></author><id>tag:datafusion.apache.org,2025-12-15:/blog/2025/12/15/avoid-consecutive-repartitions</id><summary type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;
&lt;div class="row align-items-center mb-3"&gt;
&lt;div class="col-md-7"&gt;

Databases are some of the most complex yet interesting pieces of software. They are amazing pieces of abstraction: query engines optimize and execute complex plans, storage engines provide sophisticated infrastructure as the backbone of the system, while intricate file formats lay the groundwork for particular workloads. All of this is …&lt;/div&gt;&lt;/div&gt;</summary><content type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;
&lt;div class="row align-items-center mb-3"&gt;
&lt;div class="col-md-7"&gt;

Databases are some of the most complex yet interesting pieces of software. They are amazing pieces of abstraction: query engines optimize and execute complex plans, storage engines provide sophisticated infrastructure as the backbone of the system, while intricate file formats lay the groundwork for particular workloads. All of this is exposed by a user-friendly interface and query languages (typically a dialect of SQL).
&lt;br/&gt;&lt;br/&gt;
Starting a journey learning about database internals can be daunting. With so many topics that are whole PhD degrees themselves, finding a place to start is difficult. In this blog post, I will share my early journey in the database world and a quick lesson on one of the first topics I dove into. If you are new to the space, this post will help you get your first foot into the database world, and if you are already a veteran, you may still learn something new.

&lt;/div&gt;
&lt;div class="col-md-5 text-center"&gt;
&lt;img alt="Database System Components" class="img-fluid" src="/blog/images/avoid-consecutive-repartitions/database_system_diagram.png"/&gt;
&lt;/div&gt;
&lt;/div&gt;
&lt;hr/&gt;
&lt;h2 id="who-am-i"&gt;&lt;strong&gt;Who Am I?&lt;/strong&gt;&lt;a class="headerlink" href="#who-am-i" title="Permanent link"&gt;&amp;para;&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;I am Gene Bordegaray (&lt;a href="https://www.linkedin.com/in/genebordegaray"&gt;LinkedIn&lt;/a&gt;, &lt;a href="https://github.com/gene-bordegaray"&gt;GitHub&lt;/a&gt;), a recent computer science graduate from UCLA and software engineer at Datadog. Before starting my job, I had no real exposure to databases, only enough SQL knowledge to send CRUD requests and choose between a relational or no-SQL model in a systems design interview.&lt;/p&gt;
&lt;p&gt;When I found out I would be on a team focusing on query engines and execution, I was excited but horrified. "Query engines?" From my experience, I typed SQL queries into pgAdmin and got responses without knowing the dark magic that happened under the hood.&lt;/p&gt;
&lt;p&gt;With what seemed like an impossible task at hand, I began my favorite few months of learning.&lt;/p&gt;
&lt;hr/&gt;
&lt;h2 id="starting-out"&gt;&lt;strong&gt;Starting Out&lt;/strong&gt;&lt;a class="headerlink" href="#starting-out" title="Permanent link"&gt;&amp;para;&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;I was no expert in databases or any of their subsystems, but I am someone who recently began learning about them. These are some tips I found useful when first starting.&lt;/p&gt;
&lt;h3 id="build-a-foundation"&gt;Build a Foundation&lt;a class="headerlink" href="#build-a-foundation" title="Permanent link"&gt;&amp;para;&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;The first thing I did, which I highly recommend, was watch Andy Pavlo's &lt;a href="https://15445.courses.cs.cmu.edu/fall2025/"&gt;Intro To Database Systems course&lt;/a&gt;. This laid a great foundation for understanding how a database works from end-to-end at a high-level. It touches on topics ranging from file formats to query optimization, and it was helpful to have a general context for the whole system before diving deep into a single sector.&lt;/p&gt;
&lt;h3 id="narrow-your-scope"&gt;Narrow Your Scope&lt;a class="headerlink" href="#narrow-your-scope" title="Permanent link"&gt;&amp;para;&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;The next crucial step is to pick your niche to focus on. Database systems are so vast that trying to tackle the whole beast at once is a lost cause. If you want to effectively contribute to this space, you need to deeply understand the system you are working on, and you will have much better luck narrowing your scope.&lt;/p&gt;
&lt;p&gt;When learning about the entire database stack at a high level, note what parts stick out as particularly interesting. For me, this focus is on query engines, more specifically, the physical planner and optimizer.&lt;/p&gt;
&lt;h3 id="a-slow-start"&gt;A "Slow" Start&lt;a class="headerlink" href="#a-slow-start" title="Permanent link"&gt;&amp;para;&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;The final piece of advice when starting, and I sound like a broken record, is to take your time to learn. This is not an easy sector of software to jump into; it will pay dividends to slow down, fully understand the system, and why it is designed the way it is.&lt;/p&gt;
&lt;p&gt;When making your first contributions to an open-source project, start very small but go as deep as you can. Don't leave any stone unturned. I did this by looking for simpler issues, such as formatting or simple bug fixes, and stepping through the entire data flow that relates to the issue, noting what each component is responsible for.&lt;/p&gt;
&lt;p&gt;This will give you familiarity with the codebase and using your tools, like your debugger, within the project.&lt;/p&gt;
&lt;div class="text-center"&gt;
&lt;img alt="Noot Noot Database Meme" class="img-fluid" src="/blog/images/avoid-consecutive-repartitions/noot_noot_database_meme.png" width="50%"/&gt;
&lt;/div&gt;
&lt;p&gt;&lt;br/&gt;&lt;/p&gt;
&lt;p&gt;Now that we have some general knowledge of database internals, a niche or subsystem we want to dive deeper into, and the mindset for acquiring knowledge before contributing, let's start with our first core issue.&lt;/p&gt;
&lt;hr/&gt;
&lt;h2 id="intro-to-datafusion"&gt;&lt;strong&gt;Intro to DataFusion&lt;/strong&gt;&lt;a class="headerlink" href="#intro-to-datafusion" title="Permanent link"&gt;&amp;para;&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;As mentioned, the database subsystem I decided to explore was query engines. The query engine is responsible for interpreting, optimizing, and executing queries, aiming to do so as efficiently as possible.&lt;/p&gt;
&lt;p&gt;My team was in full-swing of restructuring how query execution would work in our organization. The team decided we would use &lt;a href="https://datafusion.apache.org/"&gt;Apache DataFusion&lt;/a&gt; at the heart of our system, chosen for its blazing fast execution time for analytical workloads and vast extendability. DataFusion is written in Rust and builds on top of &lt;a href="https://arrow.apache.org/"&gt;Apache Arrow&lt;/a&gt; (another great project), a columnar memory format that enables it to efficiently process large volumes of data in memory.&lt;/p&gt;
&lt;p&gt;This project offered a perfect environment for my first steps into databases: clear, production-ready Rust programming, a manageable codebase, high performance for a specific use case, and a welcoming community.&lt;/p&gt;
&lt;h3 id="parallel-execution-in-datafusion"&gt;Parallel Execution in DataFusion&lt;a class="headerlink" href="#parallel-execution-in-datafusion" title="Permanent link"&gt;&amp;para;&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;Before discussing this issue, it is essential to understand how DataFusion handles parallel execution.&lt;/p&gt;
&lt;p&gt;DataFusion implements a vectorized &lt;a href="https://dl.acm.org/doi/10.1145/93605.98720"&gt;Volcano Model&lt;/a&gt;, similar to other state of the art engines such as ClickHouse. The Volcano Model is built on the idea that each operation is abstracted into an operator, and a DAG can represent an entire query. Each operator implements a &lt;code&gt;next()&lt;/code&gt; function that returns a batch of tuples or a &lt;code&gt;NULL&lt;/code&gt; marker if no data is available.&lt;/p&gt;
&lt;div class="text-center"&gt;
&lt;img alt="Vectorized Volcano Model Example" class="img-fluid" src="/blog/images/avoid-consecutive-repartitions/volcano_model_diagram.png" width="60%"/&gt;
&lt;/div&gt;
&lt;p&gt;&lt;br/&gt;
DataFusion achieves multi-core parallelism through the use of "exchange operators." Individual operators are implemented to use a single CPU core, and the &lt;code&gt;RepartitionExec&lt;/code&gt; operator is responsible for distributing work across multiple processors.&lt;/p&gt;
&lt;h3 id="what-is-repartitioning"&gt;What is Repartitioning?&lt;a class="headerlink" href="#what-is-repartitioning" title="Permanent link"&gt;&amp;para;&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;Partitioning is a "divide-and-conquer" approach to executing a query. Each partition is a subset of the data that is being processed on a single core. Repartitioning is an operation that redistributes data across different partitions to balance workloads, reduce data skew, and increase parallelism. Two repartitioning methods are used in DataFusion: round-robin and hash.&lt;/p&gt;
&lt;h4 id="round-robin-repartitioning"&gt;&lt;strong&gt;Round-Robin Repartitioning&lt;/strong&gt;&lt;a class="headerlink" href="#round-robin-repartitioning" title="Permanent link"&gt;&amp;para;&lt;/a&gt;&lt;/h4&gt;
&lt;div class="row align-items-start mb-3"&gt;
&lt;div class="col-md-9"&gt;

Round-robin repartitioning is the simplest partitioning strategy. Incoming data is processed in batches (chunks of rows), and these batches are distributed across partitions cyclically or sequentially, with each new batch assigned to the next available partition.
&lt;br/&gt;&lt;br/&gt;
Round-robin repartitioning is useful when the data grouping isn't known or when aiming for an even distribution across partitions. Because it simply assigns batches in order without inspecting their contents, it is a low-overhead way to increase parallelism for downstream operations.

&lt;/div&gt;
&lt;div class="col-md-3 text-center"&gt;
&lt;img alt="Round-Robin Repartitioning" class="img-fluid" src="/blog/images/avoid-consecutive-repartitions/round_robin_repartitioning.png"/&gt;
&lt;/div&gt;
&lt;/div&gt;
&lt;h4 id="hash-repartitioning"&gt;&lt;strong&gt;Hash Repartitioning&lt;/strong&gt;&lt;a class="headerlink" href="#hash-repartitioning" title="Permanent link"&gt;&amp;para;&lt;/a&gt;&lt;/h4&gt;
&lt;div class="row align-items-start mb-3"&gt;
&lt;div class="col-md-9"&gt;

Hash repartitioning distributes data based on a hash function applied to one or more columns, called the partitioning key. Rows with the same hash value are placed in the same partition.
&lt;br/&gt;&lt;br/&gt;
Hash repartitioning is useful when working with grouped data. Imagine you have a database containing information on company sales, and you are looking to find the total revenue each store produced. Hash repartitioning would make this query much more efficient. Rather than iterating over the data on a single thread and keeping a running sum for each store, it would be better to hash repartition on the store column and have multiple threads calculate individual store sales.

&lt;/div&gt;
&lt;div class="col-md-3 text-center"&gt;
&lt;img alt="Hash Repartitioning" class="img-fluid" src="/blog/images/avoid-consecutive-repartitions/hash_repartitioning.png"/&gt;
&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Note, the benefit of hash opposed to round-robin partitioning in this scenario. Hash repartitioning consolidates all rows with the same store value in distinct partitions. Because of this property we can compute the complete results for each store in parallel and merge them to get the final outcome. This parallel processing wouldn&amp;rsquo;t be possible with only round-robin partitioning as the same store value may be spread across multiple partitions, making the aggregation results partial, unable to merge them to produce a correct final outcome.&lt;/p&gt;
&lt;div class="text-center"&gt;
&lt;img alt="Hash Repartitioning Example" class="img-fluid" src="/blog/images/avoid-consecutive-repartitions/hash_repartitioning_example.png" width="100%"/&gt;
&lt;/div&gt;
&lt;hr/&gt;
&lt;h2 id="the-issue-consecutive-repartitions"&gt;&lt;strong&gt;The Issue: Consecutive Repartitions&lt;/strong&gt;&lt;a class="headerlink" href="#the-issue-consecutive-repartitions" title="Permanent link"&gt;&amp;para;&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;DataFusion contributors pointed out that consecutive repartition operators were being added to query plans, making them less efficient and more confusing to read (&lt;a href="https://github.com/apache/datafusion/issues/18341"&gt;link to issue&lt;/a&gt;). This issue had stood for over a year, with some attempts to resolve it, but they fell short.&lt;/p&gt;
&lt;p&gt;For some queries that required repartitioning, the plan would look along the lines of:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-sql"&gt;SELECT a, SUM(b) FROM data.parquet GROUP BY a;
&lt;/code&gt;&lt;/pre&gt;
&lt;div class="text-center"&gt;
&lt;img alt="Consecutive Repartition Query Plan" class="img-fluid" src="/blog/images/avoid-consecutive-repartitions/basic_before_query_plan.png" width="65%"/&gt;
&lt;/div&gt;
&lt;hr/&gt;
&lt;h2 id="why-dont-we-want-consecutive-repartitions"&gt;&lt;strong&gt;Why Don&amp;rsquo;t We Want Consecutive Repartitions?&lt;/strong&gt;&lt;a class="headerlink" href="#why-dont-we-want-consecutive-repartitions" title="Permanent link"&gt;&amp;para;&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;Repartitions would appear back-to-back in query plans, specifically a round-robin followed by a hash repartition.&lt;/p&gt;
&lt;p&gt;Why is this such a big deal? Well, repartitions do not process the data; their purpose is to redistribute it in ways that enable more efficient computation for other operators. Having consecutive repartitions is counterintuitive because we are redistributing data, then immediately redistributing it again, making the first repartition pointless. While this didn't create extreme overhead for queries, since round-robin repartitioning does not copy data, just the pointers to batches, the behavior was unclear and unnecessary.&lt;/p&gt;
&lt;div class="text-center"&gt;
&lt;img alt="Consecutive Repartition Query Plan With Data" class="img-fluid" src="/blog/images/avoid-consecutive-repartitions/in_depth_before_query_plan.png" width="65%"/&gt;
&lt;/div&gt;
&lt;p&gt;&lt;br/&gt;&lt;/p&gt;
&lt;p&gt;Optimally the plan should do one of two things:&lt;/p&gt;
&lt;ol&gt;
&lt;li&gt;If there is enough data to justify round-robin repartitioning, split the repartitions across a "worker" operator that leverages the redistributed data.&lt;/li&gt;
&lt;li&gt;Otherwise, don't use any round-robin repartition and keep the hash repartition only in the middle of the two-stage aggregation.&lt;/li&gt;
&lt;/ol&gt;
&lt;div class="text-center"&gt;
&lt;img alt="Optimal Query Plans" class="img-fluid" src="/blog/images/avoid-consecutive-repartitions/optimal_query_plans.png" width="100%"/&gt;
&lt;/div&gt;
&lt;p&gt;&lt;br/&gt;&lt;/p&gt;
&lt;p&gt;As shown in the diagram for a large query plan above, the round-robin repartition takes place before the partial aggregation. This increases parallelism for this processing, which will yield great performance benefits in larger datasets.&lt;/p&gt;
&lt;hr/&gt;
&lt;h2 id="identifying-the-bug"&gt;&lt;strong&gt;Identifying the Bug&lt;/strong&gt;&lt;a class="headerlink" href="#identifying-the-bug" title="Permanent link"&gt;&amp;para;&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;With an understanding of what the problem is, it is finally time to dive into isolating and identifying the bug.&lt;/p&gt;
&lt;h3 id="no-code"&gt;No Code!&lt;a class="headerlink" href="#no-code" title="Permanent link"&gt;&amp;para;&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;Before looking at any code, we can narrow the scope of where we should be looking. I found that tightening the boundaries of what you are looking for before reading any code is critical for being effective in large, complex codebases. If you are searching for a needle in a haystack, you will spend hours sifting through irrelevant code.&lt;/p&gt;
&lt;p&gt;We can use what we know about the issue and provided tools to pinpoint where our search should begin. So far, we know the bug only exists where repartitioning is needed. Let's see how else we can narrow down our search.&lt;/p&gt;
&lt;p&gt;From previous tickets, I was aware that DataFusion offered the &lt;code&gt;EXPLAIN VERBOSE&lt;/code&gt; keywords. When put before a query, the CLI prints the logical and physical plan at each step of planning and optimization. Running this query:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-sql"&gt;EXPLAIN VERBOSE SELECT a, SUM(b) FROM data.parquet GROUP BY a;
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;we find a critical piece of information.&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Physical Plan Before EnforceDistribution:&lt;/strong&gt;&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-text"&gt;1.OutputRequirementExec: order_by=[], dist_by=Unspecified
2.  AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[sum(parquet_data.b)]
3.    AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[sum(parquet_data.b)]
4.      DataSourceExec:
            file_groups={1 group: [[...]]}
            projection=[a, b]
            file_type=parquet
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;&lt;strong&gt;Physical Plan After EnforceDistribution:&lt;/strong&gt;&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-text"&gt;1.OutputRequirementExec: order_by=[], dist_by=Unspecified
2.  AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[sum(parquet_data.b)]
3.    RepartitionExec: partitioning=Hash([a@0], 16), input_partitions=16
4.      RepartitionExec: partitioning=RoundRobinBatch(16), input_partitions=1 &amp;lt;-- EXTRA REPARTITION!
5.        AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[sum(parquet_data.b)]
6.          DataSourceExec:
                file_groups={1 group: [[...]]}
                projection=[a, b]
                file_type=parquet
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;We have found the exact rule, &lt;a href="https://github.com/apache/datafusion/blob/944f7f2f2739a9d82ac66c330ea32a9c7479ee8b/datafusion/physical-optimizer/src/enforce_distribution.rs#L66-L184"&gt;EnforceDistribution&lt;/a&gt;, that is responsible for introducing the bug before reading a single line of code! For experienced maintainers of DataFusion, they would've known where to look before starting, but for a newbie, this is great information.&lt;/p&gt;
&lt;h3 id="the-root-cause"&gt;The Root Cause&lt;a class="headerlink" href="#the-root-cause" title="Permanent link"&gt;&amp;para;&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;With a single rule to read, isolating the issue is much simpler. The &lt;code&gt;EnforceDistribution&lt;/code&gt; rule takes a physical query plan as input, iterates over each child analyzing its requirements, and decides where adding repartition nodes is beneficial.&lt;/p&gt;
&lt;p&gt;A great place to start looking is before any repartitions are inserted, and where the program decides if adding a repartition above/below an operator is useful. With the help of handy function header comments, it was easy to identify that this is done in the &lt;a href="https://github.com/apache/datafusion/blob/944f7f2f2739a9d82ac66c330ea32a9c7479ee8b/datafusion/physical-optimizer/src/enforce_distribution.rs#L1108"&gt;get_repartition_requirement_status&lt;/a&gt; function. Here, DataFusion sets four fields indicating how the operator would benefit from repartitioning:&lt;/p&gt;
&lt;ol&gt;
&lt;li&gt;&lt;strong&gt;The operator's distribution requirement&lt;/strong&gt;: what type of partitioning does it need from its children (hash, single, or unknown)?&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;If round-robin is theoretically beneficial:&lt;/strong&gt; does the operator benefit from parallelism?&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;If our data indicates round-robin to be beneficial&lt;/strong&gt;: do we have enough data to justify the overhead of repartitioning?&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;If hash repartitioning is necessary&lt;/strong&gt;: is the parent an operator that requires all column values to be in the same partition, like an aggregate, and are we already hash-partitioned correctly?&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;Ok, great! We understand the different components DataFusion uses to indicate if repartitioning is beneficial. Now all that's left to do is see how repartitions are inserted.&lt;/p&gt;
&lt;p&gt;This logic takes place in the main loop of this rule. I find it helpful to draw algorithms like these into logic trees; this tends to make things much more straightforward and approachable:&lt;/p&gt;
&lt;div class="text-center"&gt;
&lt;img alt="Incorrect Logic Tree" class="img-fluid" src="/blog/images/avoid-consecutive-repartitions/logic_tree_before.png" width="100%"/&gt;
&lt;/div&gt;
&lt;p&gt;&lt;br/&gt;&lt;/p&gt;
&lt;p&gt;Boom! This is the root of our problem: we are inserting a round-robin repartition, then still inserting a hash repartition afterwards. This means that if an operator indicates it would benefit from both round-robin and hash repartitioning, consecutive repartitions will occur.&lt;/p&gt;
&lt;hr/&gt;
&lt;h2 id="the-fix"&gt;&lt;strong&gt;The Fix&lt;/strong&gt;&lt;a class="headerlink" href="#the-fix" title="Permanent link"&gt;&amp;para;&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;The logic shown before is, of course, incorrect, and the conditions for adding hash and round-robin repartitioning should be mutually exclusive since an operator will never benefit from shuffling data twice.&lt;/p&gt;
&lt;p&gt;Well, what is the correct logic?&lt;/p&gt;
&lt;p&gt;Based on our lesson on hash repartitioning and the heuristics DataFusion uses to determine when repartitioning can benefit an operator, the fix is easy. In the sub-tree where an operator's parent requires hash partitioning:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;If we are already hashed correctly, don't do anything. If we insert a round-robin, we will break out the partitioning.&lt;/li&gt;
&lt;li&gt;If a hash is required, just insert a hash repartition.&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;The new logic tree looks like this:&lt;/p&gt;
&lt;div class="text-center"&gt;
&lt;img alt="Correct Logic Tree" class="img-fluid" src="/blog/images/avoid-consecutive-repartitions/logic_tree_after.png" width="100%"/&gt;
&lt;/div&gt;
&lt;p&gt;&lt;br/&gt;&lt;/p&gt;
&lt;p&gt;All that deep digging paid off, one condition (see &lt;a href="https://github.com/apache/datafusion/pull/18521"&gt;the final PR&lt;/a&gt; for full details)!&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Condition before:&lt;/strong&gt;&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-rust"&gt; if add_roundrobin {
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;&lt;strong&gt;Condition after:&lt;/strong&gt;&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-rust"&gt;if add_roundrobin &amp;amp;&amp;amp; !hash_necessary {
&lt;/code&gt;&lt;/pre&gt;
&lt;hr/&gt;
&lt;h2 id="results"&gt;&lt;strong&gt;Results&lt;/strong&gt;&lt;a class="headerlink" href="#results" title="Permanent link"&gt;&amp;para;&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;This eliminated every consecutive repartition in the DataFusion test suite and benchmarks, reducing overhead, making plans clearer, and enabling further optimizations.&lt;/p&gt;
&lt;p&gt;Plans became simpler:&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Before:&lt;/strong&gt;&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-text"&gt;
1.ProjectionExec: expr=[env@0 as env, count(Int64(1))@1 as count(*)]
2.  AggregateExec: mode=FinalPartitioned, gby=[env@0 as env], aggr=[count(Int64(1))]
3.    CoalesceBatchesExec: target_batch_size=8192
4.      RepartitionExec: partitioning=Hash([env@0], 4), input_partitions=4
5.        RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 &amp;lt;-- EXTRA REPARTITION!
6.          AggregateExec: mode=Partial, gby=[env@0 as env], aggr=[count(Int64(1))]
7.            DataSourceExec:
                file_groups={1 group: [[...]}
                projection=[env]
                file_type=parquet

&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;&lt;strong&gt;After:&lt;/strong&gt;&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-text"&gt;1.ProjectionExec: expr=[env@0 as env, count(Int64(1))@1 as count(*)]
2.  AggregateExec: mode=FinalPartitioned, gby=[env@0 as env], aggr=[count(Int64(1))]
3.    CoalesceBatchesExec: target_batch_size=8192
4.      RepartitionExec: partitioning=Hash([env@0], 4), input_partitions=1
5.        AggregateExec: mode=Partial, gby=[env@0 as env], aggr=[count(Int64(1))]
6.          DataSourceExec:
                file_groups={1 group: [[...]]}
                projection=[env]
                file_type=parquet
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;For the benchmarking standard, TPCH, speedups were small but consistent:&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;TPCH Benchmark&lt;/strong&gt;&lt;/p&gt;
&lt;div class="text-left"&gt;
&lt;img alt="TPCH Benchmark Results" class="img-fluid" src="/blog/images/avoid-consecutive-repartitions/tpch_benchmark.png" width="60%"/&gt;
&lt;/div&gt;
&lt;p&gt;&lt;br/&gt;&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;TPCH10 Benchmark&lt;/strong&gt;&lt;/p&gt;
&lt;div class="text-left"&gt;
&lt;img alt="TPCH10 Benchmark Results" class="img-fluid" src="/blog/images/avoid-consecutive-repartitions/tpch10_benchmark.png" width="60%"/&gt;
&lt;/div&gt;
&lt;p&gt;&lt;br/&gt;&lt;/p&gt;
&lt;p&gt;And there it is, our first core contribution for a database system!&lt;/p&gt;
&lt;p&gt;From this experience there are two main points I would like to emphasize:&lt;/p&gt;
&lt;ol&gt;
&lt;li&gt;
&lt;p&gt;Deeply understand the system you are working on. It is not only fun to figure these things out, but it also pays off in the long run when having surface-level knowledge won't cut it.&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;
&lt;p&gt;Narrow down the scope of your work when starting your journey into databases. Find a project that you are interested in and provides an environment that enhances your early learning process. I have found that Apache DataFusion and its community has been an amazing first step and plan to continue learning about query engines here.&lt;/p&gt;
&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;I hope you gained something from my experience and have fun learning about databases.&lt;/p&gt;
&lt;hr/&gt;
&lt;h2 id="acknowledgements"&gt;&lt;strong&gt;Acknowledgements&lt;/strong&gt;&lt;a class="headerlink" href="#acknowledgements" title="Permanent link"&gt;&amp;para;&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;Thank you to &lt;a href="https://github.com/NGA-TRAN"&gt;Nga Tran&lt;/a&gt; for continuous mentorship and guidance, the DataFusion community, specifically &lt;a href="https://github.com/alamb"&gt;Andrew Lamb&lt;/a&gt;, for lending me support throughout my work, and Datadog for providing the opportunity to work on such interesting systems.&lt;/p&gt;</content><category term="blog"/></entry><entry><title>Apache DataFusion Comet 0.12.0 Release</title><link href="https://datafusion.apache.org/blog/2025/12/04/datafusion-comet-0.12.0" rel="alternate"/><published>2025-12-04T00:00:00+00:00</published><updated>2025-12-04T00:00:00+00:00</updated><author><name>pmc</name></author><id>tag:datafusion.apache.org,2025-12-04:/blog/2025/12/04/datafusion-comet-0.12.0</id><summary type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;p&gt;The Apache DataFusion PMC is pleased to announce version 0.12.0 of the &lt;a href="https://datafusion.apache.org/comet/"&gt;Comet&lt;/a&gt; subproject.&lt;/p&gt;
&lt;p&gt;Comet is an accelerator for Apache Spark that translates Spark physical plans to DataFusion physical plans for
improved performance and efficiency without requiring any code changes.&lt;/p&gt;
&lt;p&gt;This release covers approximately four weeks of development …&lt;/p&gt;</summary><content type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;p&gt;The Apache DataFusion PMC is pleased to announce version 0.12.0 of the &lt;a href="https://datafusion.apache.org/comet/"&gt;Comet&lt;/a&gt; subproject.&lt;/p&gt;
&lt;p&gt;Comet is an accelerator for Apache Spark that translates Spark physical plans to DataFusion physical plans for
improved performance and efficiency without requiring any code changes.&lt;/p&gt;
&lt;p&gt;This release covers approximately four weeks of development work and is the result of merging 105 PRs from 13
contributors. See the &lt;a href="https://github.com/apache/datafusion-comet/blob/main/dev/changelog/0.12.0.md"&gt;change log&lt;/a&gt; for more information.&lt;/p&gt;
&lt;h2 id="release-highlights"&gt;Release Highlights&lt;a class="headerlink" href="#release-highlights" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;h3 id="experimental-native-apache-iceberg-scan-support"&gt;Experimental Native Apache Iceberg Scan Support&lt;a class="headerlink" href="#experimental-native-apache-iceberg-scan-support" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;Comet has a new, experimental, &lt;a href="https://github.com/apache/datafusion-comet/pull/2528"&gt;native Iceberg scan&lt;/a&gt;. This work relies on &lt;a href="https://github.com/apache/iceberg-rust"&gt;iceberg-rust&lt;/a&gt; and the Parquet reader from &lt;a href="https://github.com/apache/arrow-rs"&gt;arrow-rs&lt;/a&gt; that Comet already uses to great effect. Comet’s &lt;a href="https://datafusion.apache.org/comet/user-guide/0.12/iceberg.html"&gt;existing Iceberg integration&lt;/a&gt; relies on a modified Iceberg Java build to accelerate Parquet decoding. This new approach allows unmodified Iceberg Java to handle query planning (&lt;em&gt;i.e.&lt;/em&gt;, catalog access, partition pruning, etc.), then Comet serializes Iceberg &lt;code&gt;FileScanTask&lt;/code&gt; objects directly to iceberg-rust, enabling native execution of Iceberg table scans through DataFusion.&lt;/p&gt;
&lt;p&gt;This represents a significant step forward in Comet's support for data lakehouse architectures and expands the range of workloads that can benefit from native acceleration. Please take a look at the PR and Comet’s documentation to understand the current limitations and try it on your workloads! We are eager for feedback on this approach.&lt;/p&gt;
&lt;h3 id="code-architecture-improvements"&gt;Code Architecture Improvements&lt;a class="headerlink" href="#code-architecture-improvements" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;This release includes significant refactoring to improve code maintainability and extensibility, and we will continue those efforts into 0.13.0 development:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;strong&gt;Unified operator serialization&lt;/strong&gt;: The &lt;a href="https://github.com/apache/datafusion-comet/pull/2768"&gt;CometExecRule refactor&lt;/a&gt; unifies CometNativeExec creation with serialization through the new &lt;code&gt;CometOperatorSerde&lt;/code&gt; trait&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Expression serde refactoring&lt;/strong&gt;: Multiple PRs (&lt;a href="https://github.com/apache/datafusion-comet/pull/2738"&gt;#2738&lt;/a&gt;, &lt;a href="https://github.com/apache/datafusion-comet/pull/2741"&gt;#2741&lt;/a&gt;, &lt;a href="https://github.com/apache/datafusion-comet/pull/2791"&gt;#2791&lt;/a&gt;) moved expression serialization logic out of &lt;code&gt;QueryPlanSerde&lt;/code&gt; into specialized traits&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Aggregate expression improvements&lt;/strong&gt;: &lt;a href="https://github.com/apache/datafusion-comet/pull/2777"&gt;Added getSupportLevel to CometAggregateExpressionSerde trait&lt;/a&gt; for better aggregate function handling&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;These architectural improvements make it easier for contributors to add new operators and expressions while reducing code complexity.&lt;/p&gt;
&lt;h3 id="new-sql-functions"&gt;New SQL Functions&lt;a class="headerlink" href="#new-sql-functions" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;The following SQL functions are now supported:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;a href="https://github.com/apache/datafusion-comet/pull/2604"&gt;&lt;code&gt;concat&lt;/code&gt;&lt;/a&gt; - String concatenation&lt;/li&gt;
&lt;li&gt;&lt;a href="https://github.com/apache/datafusion-comet/pull/2689"&gt;&lt;code&gt;abs&lt;/code&gt;&lt;/a&gt; - Absolute value&lt;/li&gt;
&lt;li&gt;&lt;a href="https://github.com/apache/datafusion-comet/pull/2471"&gt;&lt;code&gt;sha1&lt;/code&gt;&lt;/a&gt; - SHA-1 hash function&lt;/li&gt;
&lt;li&gt;&lt;a href="https://github.com/apache/datafusion-comet/pull/2755"&gt;&lt;code&gt;cot&lt;/code&gt;&lt;/a&gt; - Cotangent function&lt;/li&gt;
&lt;li&gt;&lt;a href="https://github.com/apache/datafusion-comet/pull/2784"&gt;Hyperbolic trigonometric functions&lt;/a&gt; - sinh, cosh, tanh, and their inverse functions&lt;/li&gt;
&lt;/ul&gt;
&lt;h3 id="new-operators"&gt;New Operators&lt;a class="headerlink" href="#new-operators" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;ul&gt;
&lt;li&gt;&lt;a href="https://github.com/apache/datafusion-comet/pull/2735"&gt;&lt;code&gt;CometLocalTableScanExec&lt;/code&gt;&lt;/a&gt; - Native support for local table scans, eliminating fallback to Spark for small, in-memory datasets&lt;/li&gt;
&lt;/ul&gt;
&lt;h3 id="configuration-and-usability-improvements"&gt;Configuration and Usability Improvements&lt;a class="headerlink" href="#configuration-and-usability-improvements" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;ul&gt;
&lt;li&gt;&lt;strong&gt;Simplified on-heap configuration&lt;/strong&gt;: &lt;a href="https://github.com/apache/datafusion-comet/pull/2599"&gt;Simplified on-heap memory configuration&lt;/a&gt; for easier setup&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Extended explain format&lt;/strong&gt;: &lt;a href="https://github.com/apache/datafusion-comet/pull/2644"&gt;Renamed and improved COMET_EXTENDED_EXPLAIN_FORMAT&lt;/a&gt; with better defaults&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Environment variable support&lt;/strong&gt;: &lt;a href="https://github.com/apache/datafusion-comet/pull/2722"&gt;Improved framework for setting configs with environment variables&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Native config passing&lt;/strong&gt;: &lt;a href="https://github.com/apache/datafusion-comet/pull/2801"&gt;All Comet configs now passed to native plan&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Config categorization&lt;/strong&gt;: &lt;a href="https://github.com/apache/datafusion-comet/pull/2740"&gt;Categorized testing configs&lt;/a&gt; and added notes about known timezone issues&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Removed legacy configs&lt;/strong&gt;: &lt;a href="https://github.com/apache/datafusion-comet/pull/2786"&gt;Removed COMET_EXPR_ALLOW_INCOMPATIBLE config&lt;/a&gt; to simplify configuration&lt;/li&gt;
&lt;/ul&gt;
&lt;h3 id="bug-fixes"&gt;Bug Fixes&lt;a class="headerlink" href="#bug-fixes" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;This release includes numerous bug fixes:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;a href="https://github.com/apache/datafusion-comet/pull/2606"&gt;Fixed None.get in stringDecode&lt;/a&gt; when binary child cannot be converted&lt;/li&gt;
&lt;li&gt;&lt;a href="https://github.com/apache/datafusion-comet/pull/2630"&gt;Proper fallback for lpad/rpad with unsupported arguments&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;&lt;a href="https://github.com/apache/datafusion-comet/pull/2634"&gt;Fixed trunc/date_trunc with unsupported format strings&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;&lt;a href="https://github.com/apache/datafusion-comet/pull/2675"&gt;Corrected single partition handling in native_datafusion&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;&lt;a href="https://github.com/apache/datafusion-comet/pull/2687"&gt;Fixed LeftSemi join handling&lt;/a&gt; - do not replace SMJ with HJ&lt;/li&gt;
&lt;li&gt;&lt;a href="https://github.com/apache/datafusion-comet/pull/2718"&gt;Fixed CometLiteral class cast exception with arrays&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;&lt;a href="https://github.com/apache/datafusion-comet/pull/2716"&gt;Fixed missing SortOrder fallback reason in range partitioning&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;&lt;a href="https://github.com/apache/datafusion-comet/pull/2728"&gt;Improved checkSparkMaybeThrows to compare results in success case&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;&lt;a href="https://github.com/apache/datafusion-comet/pull/2643"&gt;Fixed null handling in CometVector implementations&lt;/a&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;h3 id="documentation-improvements"&gt;Documentation Improvements&lt;a class="headerlink" href="#documentation-improvements" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;ul&gt;
&lt;li&gt;&lt;a href="https://github.com/apache/datafusion-comet/pull/2668"&gt;Added FFI documentation to contributor guide&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;&lt;a href="https://github.com/apache/datafusion-comet/pull/2704"&gt;Updated contributor guide for adding new expressions&lt;/a&gt; and &lt;a href="https://github.com/apache/datafusion-comet/pull/2758"&gt;operators&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;&lt;a href="https://github.com/apache/datafusion-comet/pull/2587"&gt;Improved documentation layout&lt;/a&gt; and &lt;a href="https://github.com/apache/datafusion-comet/pull/2597"&gt;navigation&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;&lt;a href="https://github.com/apache/datafusion-comet/pull/2783"&gt;Added prettier enforcement&lt;/a&gt; for consistent markdown formatting&lt;/li&gt;
&lt;li&gt;&lt;a href="https://github.com/apache/datafusion-comet/pull/2779"&gt;CI check to ensure generated docs are in sync&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;Various documentation updates for &lt;a href="https://github.com/apache/datafusion-comet/pull/2694"&gt;SortOrder expressions&lt;/a&gt;, &lt;a href="https://github.com/apache/datafusion-comet/pull/2742"&gt;LocalTableScan and WindowExec&lt;/a&gt;, and &lt;a href="https://github.com/apache/datafusion-comet/pull/2712"&gt;Spark SQL tests&lt;/a&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;h3 id="dependency-updates"&gt;Dependency Updates&lt;a class="headerlink" href="#dependency-updates" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;ul&gt;
&lt;li&gt;&lt;a href="https://github.com/apache/datafusion-comet/pull/2574"&gt;Upgraded to Spark 3.5.7&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;&lt;a href="https://github.com/apache/datafusion-comet/pull/2605"&gt;Upgraded to DataFusion 50.3.0&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;&lt;a href="https://github.com/apache/datafusion-comet/pull/2608"&gt;Upgraded Parquet from 56.0.0 to 56.2.0&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;Various other dependency updates via Dependabot&lt;/li&gt;
&lt;/ul&gt;
&lt;h3 id="spark-compatibility"&gt;Spark Compatibility&lt;a class="headerlink" href="#spark-compatibility" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;ul&gt;
&lt;li&gt;Spark 3.4.3 with JDK 11 &amp;amp; 17, Scala 2.12 &amp;amp; 2.13&lt;/li&gt;
&lt;li&gt;Spark 3.5.4 through 3.5.7 with JDK 11 &amp;amp; 17, Scala 2.12 &amp;amp; 2.13&lt;/li&gt;
&lt;li&gt;Spark 4.0.1 with JDK 17, Scala 2.13&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;We are looking for help from the community to fully support Spark 4.0.1. See &lt;a href="https://github.com/apache/datafusion-comet/issues/1637"&gt;EPIC: Support 4.0.0&lt;/a&gt; for more information.&lt;/p&gt;
&lt;h2 id="getting-involved"&gt;Getting Involved&lt;a class="headerlink" href="#getting-involved" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;The Comet project welcomes new contributors. We use the same &lt;a href="https://datafusion.apache.org/contributor-guide/communication.html#slack-and-discord"&gt;Slack and Discord&lt;/a&gt; channels as the main DataFusion
project and have a weekly &lt;a href="https://docs.google.com/document/d/1NBpkIAuU7O9h8Br5CbFksDhX-L9TyO9wmGLPMe0Plc8/edit?usp=sharing"&gt;DataFusion video call&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;The easiest way to get involved is to test Comet with your current Spark jobs and file issues for any bugs or
performance regressions that you find. See the &lt;a href="https://datafusion.apache.org/comet/user-guide/installation.html"&gt;Getting Started&lt;/a&gt; guide for instructions on downloading and installing
Comet.&lt;/p&gt;
&lt;p&gt;There are also many &lt;a href="https://github.com/apache/datafusion-comet/contribute"&gt;good first issues&lt;/a&gt; waiting for contributions.&lt;/p&gt;</content><category term="blog"/></entry><entry><title>Apache DataFusion 51.0.0 Released</title><link href="https://datafusion.apache.org/blog/2025/11/25/datafusion-51.0.0" rel="alternate"/><published>2025-11-25T00:00:00+00:00</published><updated>2025-11-25T00:00:00+00:00</updated><author><name>pmc</name></author><id>tag:datafusion.apache.org,2025-11-25:/blog/2025/11/25/datafusion-51.0.0</id><summary type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;h2 id="introduction"&gt;Introduction&lt;a class="headerlink" href="#introduction" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;We are proud to announce the release of &lt;a href="https://crates.io/crates/datafusion/51.0.0"&gt;DataFusion 51.0.0&lt;/a&gt;. This post highlights
some of the major improvements since &lt;a href="https://datafusion.apache.org/blog/2025/09/29/datafusion-50.0.0/"&gt;DataFusion 50.0.0&lt;/a&gt;. The complete list of
changes is available in the &lt;a href="https://github.com/apache/datafusion/blob/branch-51/dev/changelog/51.0.0.md"&gt;changelog&lt;/a&gt;. Thanks to the &lt;a href="https://github.com/apache/datafusion/blob/branch-51/dev/changelog/51.0.0.md#credits"&gt;128 contributors&lt;/a&gt; for
making this release possible.&lt;/p&gt;
&lt;h2 id="performance-improvements"&gt;Performance Improvements 🚀&lt;a class="headerlink" href="#performance-improvements" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;We continue …&lt;/p&gt;</summary><content type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;h2 id="introduction"&gt;Introduction&lt;a class="headerlink" href="#introduction" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;We are proud to announce the release of &lt;a href="https://crates.io/crates/datafusion/51.0.0"&gt;DataFusion 51.0.0&lt;/a&gt;. This post highlights
some of the major improvements since &lt;a href="https://datafusion.apache.org/blog/2025/09/29/datafusion-50.0.0/"&gt;DataFusion 50.0.0&lt;/a&gt;. The complete list of
changes is available in the &lt;a href="https://github.com/apache/datafusion/blob/branch-51/dev/changelog/51.0.0.md"&gt;changelog&lt;/a&gt;. Thanks to the &lt;a href="https://github.com/apache/datafusion/blob/branch-51/dev/changelog/51.0.0.md#credits"&gt;128 contributors&lt;/a&gt; for
making this release possible.&lt;/p&gt;
&lt;h2 id="performance-improvements"&gt;Performance Improvements 🚀&lt;a class="headerlink" href="#performance-improvements" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;We continue to make significant performance improvements in DataFusion, both in
the core engine and in the Parquet reader.&lt;/p&gt;
&lt;p&gt;&lt;img alt="Performance over time" class="img-fluid" src="/blog/images/datafusion-51.0.0/performance_over_time_clickbench.png" width="100%"/&gt;&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Figure 1&lt;/strong&gt;: Average and median normalized query execution times for ClickBench queries for DataFusion 51.0.0 compared to previous releases.
Query times are normalized using the ClickBench definition. See the
&lt;a href="https://alamb.github.io/datafusion-benchmarking/"&gt;DataFusion Benchmarking Page&lt;/a&gt;
for more details.&lt;/p&gt;
&lt;h3 id="faster-case-expression-evaluation"&gt;Faster &lt;code&gt;CASE&lt;/code&gt; expression evaluation&lt;a class="headerlink" href="#faster-case-expression-evaluation" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;This release builds on the &lt;a href="https://github.com/apache/datafusion/issues/18075"&gt;CASE performance epic&lt;/a&gt; with significant improvements.
Expressions short‑circuit earlier, reuse partial results, and avoid unnecessary
scattering, speeding up common ETL patterns. Thanks to &lt;a href="https://github.com/pepijnve"&gt;pepijnve&lt;/a&gt;, &lt;a href="https://github.com/chenkovsky"&gt;chenkovsky&lt;/a&gt;,
and &lt;a href="https://github.com/petern48"&gt;petern48&lt;/a&gt; for leading this effort. You can find more details in the
&lt;a href="/blog/2026/02/02/datafusion_case/"&gt;Optimizing SQL CASE Expression Evaluation&lt;/a&gt; blog post.&lt;/p&gt;
&lt;h3 id="better-defaults-for-remote-parquet-reads"&gt;Better Defaults for Remote Parquet Reads&lt;a class="headerlink" href="#better-defaults-for-remote-parquet-reads" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;By default, DataFusion now always fetches the last 512KB (configurable) of &lt;a href="https://parquet.apache.org/"&gt;Apache Parquet&lt;/a&gt; files
which usually includes the footer and metadata (&lt;a href="https://github.com/apache/datafusion/issues/18118"&gt;#18118&lt;/a&gt;). This 
change typically avoids 2 I/O requests for each Parquet. While this
setting has existed in DataFusion for many years, it was not previously enabled
by default. Users can tune the number of bytes fetched in the initial I/O
request via the &lt;code&gt;datafusion.execution.parquet.metadata_size_hint&lt;/code&gt; &lt;a href="https://datafusion.apache.org/user-guide/configs.html"&gt;config setting&lt;/a&gt;. Thanks to
&lt;a href="https://github.com/zhuqi-lucas"&gt;zhuqi-lucas&lt;/a&gt; for leading this effort.&lt;/p&gt;
&lt;h3 id="faster-parquet-metadata-parsing"&gt;Faster Parquet metadata parsing&lt;a class="headerlink" href="#faster-parquet-metadata-parsing" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;DataFusion 51 also includes the latest Parquet reader from
&lt;a href="https://arrow.apache.org/blog/2025/10/30/arrow-rs-57.0.0/"&gt;Arrow Rust 57.0.0&lt;/a&gt;, which parses Parquet metadata significantly faster. This is
especially beneficial for workloads with many small Parquet files and scenarios
where startup time or low latency is important. You can read more about the upstream work by
&lt;a href="https://github.com/etseidl"&gt;etseidl&lt;/a&gt; and &lt;a href="https://github.com/jhorstmann"&gt;jhorstmann&lt;/a&gt; that enabled these improvements in the &lt;a href="https://arrow.apache.org/blog/2025/10/23/rust-parquet-metadata/"&gt;Faster Apache Parquet Footer Metadata Using a Custom Thrift Parser&lt;/a&gt; blog.&lt;/p&gt;
&lt;p&gt;&lt;img alt="Metadata Parsing Performance Improvements in Arrow/Parquet 57" class="img-fluid" src="/blog/images/datafusion-51.0.0/arrow-57-metadata-parsing.png" width="100%"/&gt;&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Figure 2&lt;/strong&gt;: Metadata parsing performance improvements in Arrow/Parquet 57.0.0. &lt;/p&gt;
&lt;h2 id="new-features"&gt;New Features ✨&lt;a class="headerlink" href="#new-features" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;h3 id="decimal32decimal64-support"&gt;Decimal32/Decimal64 support&lt;a class="headerlink" href="#decimal32decimal64-support" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;The new Arrow types &lt;code&gt;Decimal32&lt;/code&gt; and &lt;code&gt;Decimal64&lt;/code&gt; are now supported in DataFusion
(&lt;a href="https://github.com/apache/datafusion/pull/17501"&gt;#17501&lt;/a&gt;), including aggregations such as &lt;code&gt;SUM&lt;/code&gt;, &lt;code&gt;AVG&lt;/code&gt;, &lt;code&gt;MIN/MAX&lt;/code&gt;, and window
functions. Thanks to &lt;a href="https://github.com/AdamGS"&gt;AdamGS&lt;/a&gt; for leading this effort.&lt;/p&gt;
&lt;h3 id="sql-pipe-operators"&gt;SQL Pipe Operators&lt;a class="headerlink" href="#sql-pipe-operators" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;DataFusion now supports the SQL pipe operator syntax
(&lt;a href="https://github.com/apache/datafusion/pull/17278"&gt;#17278&lt;/a&gt;), enabling inline transforms such as:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-sql"&gt;SELECT * FROM t
|&amp;gt; WHERE a &amp;gt; 10
|&amp;gt; ORDER BY b
|&amp;gt; LIMIT 5;
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;This syntax, &lt;a href="https://docs.cloud.google.com/bigquery/docs/reference/standard-sql/pipe-syntax"&gt;popularized by Google BigQuery&lt;/a&gt;, keeps multi-step transformations concise while preserving regular
SQL semantics. Thanks to &lt;a href="https://github.com/simonvandel"&gt;simonvandel&lt;/a&gt; for leading this effort.&lt;/p&gt;
&lt;h3 id="io-profiling-in-datafusion-cli"&gt;I/O Profiling in &lt;code&gt;datafusion-cli&lt;/code&gt;&lt;a class="headerlink" href="#io-profiling-in-datafusion-cli" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;&lt;a href="https://datafusion.apache.org/user-guide/cli/"&gt;datafusion-cli&lt;/a&gt; now has built-in instrumentation to trace object store calls
(&lt;a href="https://github.com/apache/datafusion/issues/17207"&gt;#17207&lt;/a&gt;). Toggle profiling
with the &lt;a href="https://datafusion.apache.org/user-guide/cli/usage.html#commands"&gt;\object_store_profiling command&lt;/a&gt; and inspect the exact &lt;code&gt;GET&lt;/code&gt;/&lt;code&gt;LIST&lt;/code&gt; requests issued during
query execution:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-sql"&gt;DataFusion CLI v51.0.0
&amp;gt; \object_store_profiling trace
ObjectStore Profile mode set to Trace
&amp;gt; select count(*) from 'https://datasets.clickhouse.com/hits_compatible/athena_partitioned/hits_1.parquet';
+----------+
| count(*) |
+----------+
| 1000000  |
+----------+
1 row(s) fetched.
Elapsed 0.367 seconds.

Object Store Profiling
Instrumented Object Store: instrument_mode: Trace, inner: HttpStore
2025-11-19T21:10:43.476121+00:00 operation=Head duration=0.069763s path=hits_compatible/athena_partitioned/hits_1.parquet
2025-11-19T21:10:43.545903+00:00 operation=Head duration=0.025859s path=hits_compatible/athena_partitioned/hits_1.parquet
2025-11-19T21:10:43.571768+00:00 operation=Head duration=0.025684s path=hits_compatible/athena_partitioned/hits_1.parquet
2025-11-19T21:10:43.597463+00:00 operation=Get duration=0.034194s size=524288 range: bytes=174440756-174965043 path=hits_compatible/athena_partitioned/hits_1.parquet
2025-11-19T21:10:43.705821+00:00 operation=Head duration=0.022029s path=hits_compatible/athena_partitioned/hits_1.parquet

Summaries:
+-----------+----------+-----------+-----------+-----------+-----------+-------+
| Operation | Metric   | min       | max       | avg       | sum       | count |
+-----------+----------+-----------+-----------+-----------+-----------+-------+
| Get       | duration | 0.034194s | 0.034194s | 0.034194s | 0.034194s | 1     |
| Get       | size     | 524288 B  | 524288 B  | 524288 B  | 524288 B  | 1     |
| Head      | duration | 0.022029s | 0.069763s | 0.035834s | 0.143335s | 4     |
| Head      | size     |           |           |           |           | 4     |
+-----------+----------+-----------+-----------+-----------+-----------+-------+
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;This makes it far easier to diagnose slow remote scans and validate caching
strategies. Thanks to &lt;a href="https://github.com/BlakeOrth"&gt;BlakeOrth&lt;/a&gt; for leading this effort.&lt;/p&gt;
&lt;h3 id="describe-query"&gt;&lt;code&gt;DESCRIBE &amp;lt;query&amp;gt;&lt;/code&gt;&lt;a class="headerlink" href="#describe-query" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;&lt;code&gt;DESCRIBE&lt;/code&gt; now works on arbitrary queries, returning the schema instead
of being an alias for &lt;code&gt;EXPLAIN&lt;/code&gt; (&lt;a href="https://github.com/apache/datafusion/issues/18234"&gt;#18234&lt;/a&gt;). This brings DataFusion in line with engines
like DuckDB and makes it easy to inspect the output schema of queries
without executing them. Thanks to &lt;a href="https://github.com/djanderson"&gt;djanderson&lt;/a&gt; for leading this effort.&lt;/p&gt;
&lt;p&gt;For example:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-sql"&gt;DataFusion CLI v51.0.0
&amp;gt; create table t(a int, b varchar, c float) as values (1, 'a', 2.0);
0 row(s) fetched.
Elapsed 0.002 seconds.

&amp;gt; DESCRIBE SELECT a, b, SUM(c) FROM t GROUP BY a, b;

+-------------+-----------+-------------+
| column_name | data_type | is_nullable |
+-------------+-----------+-------------+
| a           | Int32     | YES         |
| b           | Utf8View  | YES         |
| sum(t.c)    | Float64   | YES         |
+-------------+-----------+-------------+
3 row(s) fetched.
&lt;/code&gt;&lt;/pre&gt;
&lt;h3 id="named-arguments-in-sql-functions"&gt;Named arguments in SQL functions&lt;a class="headerlink" href="#named-arguments-in-sql-functions" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;DataFusion now understands &lt;a href="https://www.postgresql.org/docs/current/sql-syntax-calling-funcs.html"&gt;PostgreSQL-style named arguments&lt;/a&gt; (&lt;code&gt;param =&amp;gt; value&lt;/code&gt;)
for scalar, aggregate, and window functions (&lt;a href="https://github.com/apache/datafusion/issues/17379"&gt;#17379&lt;/a&gt;). You can mix positional and named
arguments in any order, and error messages now list parameter names to make
diagnostics clearer. UDF authors can also expose parameter names so their
functions benefit from the same syntax. Thanks to &lt;a href="https://github.com/timsaucer"&gt;timsaucer&lt;/a&gt; and &lt;a href="https://github.com/bubulalabu"&gt;bubulalabu&lt;/a&gt; for leading this effort.&lt;/p&gt;
&lt;p&gt;For example, you can pass arguments to functions like this:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-sql"&gt;SELECT power(exponent =&amp;gt; 3.0, base =&amp;gt; 2.0);
&lt;/code&gt;&lt;/pre&gt;
&lt;h3 id="metrics-improvements"&gt;Metrics improvements&lt;a class="headerlink" href="#metrics-improvements" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;The output of &lt;a href="https://datafusion.apache.org/user-guide/sql/explain.html#explain-analyze"&gt;EXPLAIN ANALYZE&lt;/a&gt; has been improved to include more metrics
about execution time and memory usage of each operator (&lt;a href="https://github.com/apache/datafusion/issues/18217"&gt;#18217&lt;/a&gt;).
You can learn more about these new metrics in the &lt;a href="https://datafusion.apache.org/user-guide/metrics.html"&gt;metrics user guide&lt;/a&gt;. Thanks to
&lt;a href="https://github.com/2010YOUY01"&gt;2010YOUY01&lt;/a&gt; for leading this effort.&lt;/p&gt;
&lt;p&gt;The &lt;code&gt;51.0.0&lt;/code&gt; release adds:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;strong&gt;Configuration&lt;/strong&gt;: adds a new option &lt;code&gt;datafusion.explain.analyze_level&lt;/code&gt;, which can be set to &lt;code&gt;summary&lt;/code&gt; for a concise output or &lt;code&gt;dev&lt;/code&gt; for the full set of metrics (the previous default).&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;For all major operators&lt;/strong&gt;: adds &lt;code&gt;output_bytes&lt;/code&gt;, reporting how many bytes of data each operator produces.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;FilterExec&lt;/strong&gt;: adds a &lt;code&gt;selectivity&lt;/code&gt; metric (&lt;code&gt;output_rows / input_rows&lt;/code&gt;) to show how effective the filter is.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;AggregateExec&lt;/strong&gt;: &lt;/li&gt;
&lt;li&gt;adds detailed timing metrics for group-ID computation, aggregate argument evaluation, aggregation work, and emitting final results.&lt;/li&gt;
&lt;li&gt;adds a &lt;code&gt;reduction_factor&lt;/code&gt; metric (&lt;code&gt;output_rows / input_rows&lt;/code&gt;) to show how much grouping reduces the data.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;NestedLoopJoinExec&lt;/strong&gt;: adds a &lt;code&gt;selectivity&lt;/code&gt; metric (&lt;code&gt;output_rows / (left_rows * right_rows)&lt;/code&gt;) to show how many combinations actually pass the join condition.&lt;/li&gt;
&lt;li&gt;Several display formatting improvements were added to make &lt;code&gt;EXPLAIN ANALYZE&lt;/code&gt; output easier to read.&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;For example, the following query:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-sql"&gt;set datafusion.explain.analyze_level = summary

explain analyze 
select count(*) 
from 'https://datasets.clickhouse.com/hits_compatible/athena_partitioned/hits_1.parquet' 
where "URL" &amp;lt;&amp;gt; '';
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;Now shows easier-to-understand metrics such as:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-text"&gt; metrics=[
   output_rows=1000000, 
   elapsed_compute=16ns, 
   output_bytes=222.5 MB, 
   files_ranges_pruned_statistics=16 total → 16 matched, 
   row_groups_pruned_statistics=3 total → 3 matched, 
   row_groups_pruned_bloom_filter=3 total → 3 matched, 
   page_index_rows_pruned=0 total → 0 matched,
   bytes_scanned=33661364,
   metadata_load_time=4.243098ms, 
]
&lt;/code&gt;&lt;/pre&gt;
&lt;h2 id="upgrade-guide-and-changelog"&gt;Upgrade Guide and Changelog&lt;a class="headerlink" href="#upgrade-guide-and-changelog" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;Upgrading to 51.0.0 should be straightforward for most users. Please review the
&lt;a href="https://datafusion.apache.org/library-user-guide/upgrading.html"&gt;Upgrade Guide&lt;/a&gt;
for details on breaking changes and code snippets to help with the transition.
For a comprehensive list of all changes, please refer to the &lt;a href="https://github.com/apache/datafusion/blob/branch-51/dev/changelog/51.0.0.md"&gt;changelog&lt;/a&gt;.&lt;/p&gt;
&lt;h2 id="about-datafusion"&gt;About DataFusion&lt;a class="headerlink" href="#about-datafusion" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;&lt;a href="https://datafusion.apache.org/"&gt;Apache DataFusion&lt;/a&gt; is an extensible query engine, written in &lt;a href="https://www.rust-lang.org/"&gt;Rust&lt;/a&gt;, that uses
&lt;a href="https://arrow.apache.org"&gt;Apache Arrow&lt;/a&gt; as its in-memory format. DataFusion is used by developers to
create new, fast, data-centric systems such as databases, dataframe libraries,
and machine learning and streaming applications. While &lt;a href="https://datafusion.apache.org/user-guide/introduction.html#project-goals"&gt;DataFusion’s primary
design goal&lt;/a&gt; is to accelerate the creation of other data-centric systems, it
provides a reasonable experience directly out of the box as a &lt;a href="https://datafusion.apache.org/user-guide/dataframe.html"&gt;dataframe
library&lt;/a&gt;, &lt;a href="https://datafusion.apache.org/python/"&gt;Python library&lt;/a&gt;, and &lt;a href="https://datafusion.apache.org/user-guide/cli/"&gt;command-line SQL tool&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;DataFusion's core thesis is that, as a community, together we can build much
more advanced technology than any of us as individuals or companies could build
alone. Without DataFusion, highly performant vectorized query engines would
remain the domain of a few large companies and world-class research
institutions. With DataFusion, we can all build on top of a shared foundation
and focus on what makes our projects unique.&lt;/p&gt;
&lt;h2 id="how-to-get-involved"&gt;How to Get Involved&lt;a class="headerlink" href="#how-to-get-involved" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;DataFusion is not a project built or driven by a single person, company, or
foundation. Rather, our community of users and contributors works together to
build a shared technology that none of us could have built alone.&lt;/p&gt;
&lt;p&gt;If you are interested in joining us, we would love to have you. You can try out
DataFusion on some of your own data and projects and let us know how it goes,
contribute suggestions, documentation, bug reports, or a PR with documentation,
tests, or code. A list of open issues suitable for beginners is &lt;a href="https://github.com/apache/arrow-datafusion/issues?q=is%3Aissue+is%3Aopen+label%3A%22good+first+issue%22"&gt;here&lt;/a&gt;, and you
can find out how to reach us on the &lt;a href="https://datafusion.apache.org/contributor-guide/communication.html"&gt;communication doc&lt;/a&gt;.&lt;/p&gt;</content><category term="blog"/></entry><entry><title>Apache DataFusion Comet 0.11.0 Release</title><link href="https://datafusion.apache.org/blog/2025/10/21/datafusion-comet-0.11.0" rel="alternate"/><published>2025-10-21T00:00:00+00:00</published><updated>2025-10-21T00:00:00+00:00</updated><author><name>pmc</name></author><id>tag:datafusion.apache.org,2025-10-21:/blog/2025/10/21/datafusion-comet-0.11.0</id><summary type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;p&gt;The Apache DataFusion PMC is pleased to announce version 0.11.0 of the &lt;a href="https://datafusion.apache.org/comet/"&gt;Comet&lt;/a&gt; subproject.&lt;/p&gt;
&lt;p&gt;Comet is an accelerator for Apache Spark that translates Spark physical plans to DataFusion physical plans for
improved performance and efficiency without requiring any code changes.&lt;/p&gt;
&lt;p&gt;This release covers approximately five weeks of development …&lt;/p&gt;</summary><content type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;p&gt;The Apache DataFusion PMC is pleased to announce version 0.11.0 of the &lt;a href="https://datafusion.apache.org/comet/"&gt;Comet&lt;/a&gt; subproject.&lt;/p&gt;
&lt;p&gt;Comet is an accelerator for Apache Spark that translates Spark physical plans to DataFusion physical plans for
improved performance and efficiency without requiring any code changes.&lt;/p&gt;
&lt;p&gt;This release covers approximately five weeks of development work and is the result of merging 131 PRs from 15
contributors. See the &lt;a href="https://github.com/apache/datafusion-comet/blob/main/dev/changelog/0.11.0.md"&gt;change log&lt;/a&gt; for more information.&lt;/p&gt;
&lt;h2 id="release-highlights"&gt;Release Highlights&lt;a class="headerlink" href="#release-highlights" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;h3 id="parquet-modular-encryption-support"&gt;Parquet Modular Encryption Support&lt;a class="headerlink" href="#parquet-modular-encryption-support" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;Spark supports Parquet Modular Encryption to independently encrypt column values and metadata. Furthermore, Spark supports custom encryption factories for users to provide their own key-management service (KMS) implementations. Thanks to &lt;a href="https://github.com/apache/arrow-rs/issues/7278"&gt;a&lt;/a&gt; &lt;a href="https://github.com/apache/datafusion/issues/15216"&gt;number&lt;/a&gt; &lt;a href="https://github.com/apache/datafusion/pull/16351"&gt;of&lt;/a&gt; &lt;a href="https://github.com/apache/datafusion/pull/16779"&gt;contributions&lt;/a&gt; in upstream DataFusion and arrow-rs, Comet now &lt;a href="https://github.com/apache/datafusion-comet/pull/2447"&gt;supports Parquet Modular Encryption with Spark KMS&lt;/a&gt; for native readers, enabling secure reading of encrypted Parquet files in production environments.&lt;/p&gt;
&lt;h3 id="improved-memory-management"&gt;Improved Memory Management&lt;a class="headerlink" href="#improved-memory-management" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;Comet 0.11.0 introduces significant improvements to memory management, making it easier to deploy and more resilient to out-of-memory conditions:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;strong&gt;Changed default memory pool&lt;/strong&gt;: The default off-heap memory pool has been &lt;a href="https://github.com/apache/datafusion-comet/pull/2526"&gt;changed from &lt;code&gt;greedy_unified&lt;/code&gt; to &lt;code&gt;fair_unified&lt;/code&gt;&lt;/a&gt;, providing better memory fairness across operations&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Off-heap deployment recommended&lt;/strong&gt;: To simplify configuration and improve performance, Comet now expects to be deployed with Spark's off-heap memory configuration. &lt;a href="https://github.com/apache/datafusion-comet/pull/2554"&gt;On-heap memory is still available&lt;/a&gt; for development and debugging, but is not recommended for deployment&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Better disk management&lt;/strong&gt;: The &lt;a href="https://github.com/apache/datafusion-comet/pull/2479"&gt;DiskManager &lt;code&gt;max_temp_directory_size&lt;/code&gt; is now configurable&lt;/a&gt; for better control over temporary disk usage&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Enhanced safety&lt;/strong&gt;: Memory pool operations now &lt;a href="https://github.com/apache/datafusion-comet/pull/2455"&gt;use checked arithmetic operations&lt;/a&gt; to prevent overflow issues&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;These changes make Comet significantly easier to configure and deploy in production environments.&lt;/p&gt;
&lt;h3 id="improved-apache-spark-40-support"&gt;Improved Apache Spark 4.0 Support&lt;a class="headerlink" href="#improved-apache-spark-40-support" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;Comet has improved its support for Apache Spark 4.0.1 with several important enhancements:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;a href="https://github.com/apache/datafusion-comet/pull/2414"&gt;Updated support from Spark 4.0.0 to Spark 4.0.1&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;&lt;a href="https://github.com/apache/datafusion-comet/pull/2514"&gt;Spark 4.0 is now included in the release build script&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;Expanded ANSI mode compatibility with several new implementations:&lt;/li&gt;
&lt;li&gt;&lt;a href="https://github.com/apache/datafusion-comet/pull/2136"&gt;ANSI evaluation mode arithmetic operations&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;&lt;a href="https://github.com/apache/datafusion-comet/pull/2421"&gt;ANSI mode integral divide&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;&lt;a href="https://github.com/apache/datafusion-comet/pull/2542"&gt;ANSI mode rounding functions&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;&lt;a href="https://github.com/apache/datafusion-comet/pull/2556"&gt;ANSI mode remainder function&lt;/a&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;Spark 4.0 compatible jar files are now available on Maven Central. See the &lt;a href="https://datafusion.apache.org/comet/user-guide/0.11/installation.html#using-a-published-jar-file"&gt;installation guide&lt;/a&gt; for instructions on using published jar files.&lt;/p&gt;
&lt;h3 id="complex-types-for-columnar-shuffle"&gt;Complex Types for Columnar Shuffle&lt;a class="headerlink" href="#complex-types-for-columnar-shuffle" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;&lt;a href="https://github.com/ashdnazg"&gt;ashdnazg&lt;/a&gt; submitted a &lt;a href="https://github.com/apache/datafusion-comet/pull/2571"&gt;fantastic refactoring PR&lt;/a&gt; that simplified the logic for writing rows in Comet’s JVM-based, columnar shuffle. A benefit of this refactoring is better support for complex types (&lt;em&gt;e.g.,&lt;/em&gt; structs, lists, and arrays) in columnar shuffle. Comet no longer falls back to Spark to shuffle these types, enabling native acceleration for queries involving nested data structures. This enhancement significantly expands the range of queries that can benefit from Comet's columnar shuffle implementation.&lt;/p&gt;
&lt;h3 id="rangepartitioning-for-native-shuffle"&gt;RangePartitioning for Native Shuffle&lt;a class="headerlink" href="#rangepartitioning-for-native-shuffle" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;Comet's native shuffle now &lt;a href="https://github.com/apache/datafusion-comet/pull/2258"&gt;supports RangePartitioning&lt;/a&gt;, providing better performance for operations that require range-based data distribution. Comet now matches Spark behavior for computing and distributing range boundaries, and serializes them to native execution for faster shuffle operations.&lt;/p&gt;
&lt;h3 id="new-functionality"&gt;New Functionality&lt;a class="headerlink" href="#new-functionality" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;The following SQL functions are now supported:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;a href="https://github.com/apache/datafusion-comet/pull/2411"&gt;&lt;code&gt;weekday&lt;/code&gt;&lt;/a&gt; - Extract day of week from date&lt;/li&gt;
&lt;li&gt;&lt;a href="https://github.com/apache/datafusion-comet/pull/2102"&gt;&lt;code&gt;lpad&lt;/code&gt;&lt;/a&gt; - Left pad a string with column support for pad length&lt;/li&gt;
&lt;li&gt;&lt;a href="https://github.com/apache/datafusion-comet/pull/2099"&gt;&lt;code&gt;rpad&lt;/code&gt;&lt;/a&gt; - Right pad a string with &lt;a href="https://github.com/apache/datafusion-comet/pull/2436"&gt;column support and additional character support&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;&lt;a href="https://github.com/apache/datafusion-comet/pull/2481"&gt;&lt;code&gt;reverse&lt;/code&gt;&lt;/a&gt; - Support for ArrayType input in addition to strings&lt;/li&gt;
&lt;li&gt;&lt;a href="https://github.com/apache/datafusion-comet/pull/2429"&gt;&lt;code&gt;count(distinct)&lt;/code&gt;&lt;/a&gt; - Native support without falling back to Spark&lt;/li&gt;
&lt;li&gt;&lt;a href="https://github.com/apache/datafusion-comet/pull/2466"&gt;&lt;code&gt;bit_get&lt;/code&gt;&lt;/a&gt; - Get bit value at position&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;New expression capabilities include:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;a href="https://github.com/apache/datafusion-comet/pull/2181"&gt;Nested array literal support&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;&lt;a href="https://github.com/apache/datafusion-comet/pull/2425"&gt;Array-to-string cast support&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;&lt;a href="https://github.com/apache/datafusion-comet/pull/2472"&gt;Spark-compatible cast from integral to decimal types&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;&lt;a href="https://github.com/apache/datafusion-comet/pull/2490"&gt;Support for decimal type to boolean cast&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;&lt;a href="https://github.com/apache/datafusion-comet/pull/2316"&gt;More date part expressions&lt;/a&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;h3 id="performance-improvements"&gt;Performance Improvements&lt;a class="headerlink" href="#performance-improvements" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;ul&gt;
&lt;li&gt;&lt;a href="https://github.com/apache/datafusion-comet/pull/2417"&gt;Improved BroadcastExchangeExec conversion&lt;/a&gt; for better broadcast join performance&lt;/li&gt;
&lt;li&gt;&lt;a href="https://github.com/apache/datafusion-comet/pull/2407"&gt;Use of DataFusion's native &lt;code&gt;count_udaf&lt;/code&gt;&lt;/a&gt; instead of &lt;code&gt;SUM(IF(expr IS NOT NULL, 1, 0))&lt;/code&gt;&lt;/li&gt;
&lt;li&gt;&lt;a href="https://github.com/apache/datafusion-comet/pull/2402"&gt;New configuration from shared conf&lt;/a&gt; to reduce overhead&lt;/li&gt;
&lt;li&gt;&lt;a href="https://github.com/apache/datafusion-comet/pull/2579"&gt;Buffered index writes&lt;/a&gt; to reduce system calls in shuffle operations&lt;/li&gt;
&lt;/ul&gt;
&lt;h3 id="comet-0110-tpc-h-performance"&gt;Comet 0.11.0 TPC-H Performance&lt;a class="headerlink" href="#comet-0110-tpc-h-performance" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;Comet 0.11.0 continues to deliver significant performance improvements over Spark. In our &lt;a href="https://github.com/apache/datafusion-comet/pull/2596"&gt;TPC-H benchmarks&lt;/a&gt;, Comet reduced overall query runtime from 687 seconds to 302 seconds when processing 100 GB of Parquet data using a single 8-core executor, achieving a &lt;strong&gt;2.2x speedup&lt;/strong&gt;.&lt;/p&gt;
&lt;p&gt;&lt;img alt="TPC-H Overall Performance" class="img-fluid" src="/blog/images/comet-0.11.0/tpch_allqueries.png" width="100%"/&gt;&lt;/p&gt;
&lt;p&gt;The performance gains are consistent across individual queries, with most queries showing substantial improvements:&lt;/p&gt;
&lt;p&gt;&lt;img alt="TPC-H Query-by-Query Comparison" class="img-fluid" src="/blog/images/comet-0.11.0/tpch_queries_compare.png" width="100%"/&gt;&lt;/p&gt;
&lt;p&gt;You can reproduce these benchmarks using our &lt;a href="https://datafusion.apache.org/comet/contributor-guide/benchmarking.html"&gt;Comet Benchmarking Guide&lt;/a&gt;. We encourage you to run your own performance tests with your workloads.&lt;/p&gt;
&lt;h3 id="apache-iceberg-support"&gt;Apache Iceberg Support&lt;a class="headerlink" href="#apache-iceberg-support" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;ul&gt;
&lt;li&gt;&lt;a href="https://github.com/apache/datafusion-comet/pull/2386"&gt;Updated support for Apache Iceberg 1.9.1&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;&lt;a href="https://github.com/apache/datafusion-comet/pull/2442"&gt;Additional Parquet-independent API improvements&lt;/a&gt; for Iceberg integration&lt;/li&gt;
&lt;li&gt;&lt;a href="https://github.com/apache/datafusion-comet/pull/2510"&gt;Improved resource management&lt;/a&gt; in Iceberg reader instances&lt;/li&gt;
&lt;/ul&gt;
&lt;h3 id="ux-improvements"&gt;UX Improvements&lt;a class="headerlink" href="#ux-improvements" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;ul&gt;
&lt;li&gt;&lt;a href="https://github.com/apache/datafusion-comet/pull/2412"&gt;Added plan conversion statistics to extended explain info&lt;/a&gt; for better observability&lt;/li&gt;
&lt;li&gt;&lt;a href="https://github.com/apache/datafusion-comet/pull/2450"&gt;Improved fallback information&lt;/a&gt; to help users understand when and why Comet falls back to Spark&lt;/li&gt;
&lt;li&gt;&lt;a href="https://github.com/apache/datafusion-comet/pull/2515"&gt;Added &lt;code&gt;backtrace&lt;/code&gt; feature&lt;/a&gt; to simplify enabling native backtraces in &lt;code&gt;CometNativeException&lt;/code&gt;&lt;/li&gt;
&lt;li&gt;&lt;a href="https://github.com/apache/datafusion-comet/pull/2379"&gt;Native log level is now configurable&lt;/a&gt; via Comet configuration&lt;/li&gt;
&lt;/ul&gt;
&lt;h3 id="bug-fixes"&gt;Bug Fixes&lt;a class="headerlink" href="#bug-fixes" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;ul&gt;
&lt;li&gt;&lt;a href="https://github.com/apache/datafusion-comet/pull/2398"&gt;Resolved issues with reused broadcast plans in non-AQE mode&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;&lt;a href="https://github.com/apache/datafusion-comet/pull/2420"&gt;Fixed thread safety in setNumPartitions&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;&lt;a href="https://github.com/apache/datafusion-comet/pull/2440"&gt;Improved error handling when resolving S3 bucket region&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;&lt;a href="https://github.com/apache/datafusion-comet/pull/2432"&gt;Fixed byte array literal casting issues&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;&lt;a href="https://github.com/apache/datafusion-comet/pull/2438"&gt;Corrected subquery filter pushdown behavior for native_datafusion scan&lt;/a&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;h3 id="documentation-updates"&gt;Documentation Updates&lt;a class="headerlink" href="#documentation-updates" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;ul&gt;
&lt;li&gt;&lt;a href="https://github.com/apache/datafusion-comet/pull/2487"&gt;Updated documentation for native shuffle configuration and tuning&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;&lt;a href="https://github.com/apache/datafusion-comet/pull/2496"&gt;Added documentation for ANSI mode support&lt;/a&gt; in various functions&lt;/li&gt;
&lt;li&gt;&lt;a href="https://github.com/apache/datafusion-comet/pull/2474"&gt;Improved EC2 benchmarking guide&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;&lt;a href="https://github.com/apache/datafusion-comet/pull/2568"&gt;Split configuration guide into different sections&lt;/a&gt; (scan, exec, shuffle, etc.) for better organization&lt;/li&gt;
&lt;li&gt;Various clarifications and improvements throughout the documentation&lt;/li&gt;
&lt;/ul&gt;
&lt;h3 id="spark-compatibility"&gt;Spark Compatibility&lt;a class="headerlink" href="#spark-compatibility" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;ul&gt;
&lt;li&gt;Spark 3.4.3 with JDK 11 &amp;amp; 17, Scala 2.12 &amp;amp; 2.13&lt;/li&gt;
&lt;li&gt;Spark 3.5.4 through 3.5.6 with JDK 11 &amp;amp; 17, Scala 2.12 &amp;amp; 2.13&lt;/li&gt;
&lt;li&gt;Spark 4.0.1 with JDK 17, Scala 2.13&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;We are looking for help from the community to fully support Spark 4.0.1. See &lt;a href="https://github.com/apache/datafusion-comet/issues/1637"&gt;EPIC: Support 4.0.0&lt;/a&gt; for more information.&lt;/p&gt;
&lt;h2 id="getting-involved"&gt;Getting Involved&lt;a class="headerlink" href="#getting-involved" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;The Comet project welcomes new contributors. We use the same &lt;a href="https://datafusion.apache.org/contributor-guide/communication.html#slack-and-discord"&gt;Slack and Discord&lt;/a&gt; channels as the main DataFusion
project and have a weekly &lt;a href="https://docs.google.com/document/d/1NBpkIAuU7O9h8Br5CbFksDhX-L9TyO9wmGLPMe0Plc8/edit?usp=sharing"&gt;DataFusion video call&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;The easiest way to get involved is to test Comet with your current Spark jobs and file issues for any bugs or
performance regressions that you find. See the &lt;a href="https://datafusion.apache.org/comet/user-guide/installation.html"&gt;Getting Started&lt;/a&gt; guide for instructions on downloading and installing
Comet.&lt;/p&gt;
&lt;p&gt;There are also many &lt;a href="https://github.com/apache/datafusion-comet/contribute"&gt;good first issues&lt;/a&gt; waiting for contributions.&lt;/p&gt;</content><category term="blog"/></entry><entry><title>Apache DataFusion 50.0.0 Released</title><link href="https://datafusion.apache.org/blog/2025/09/29/datafusion-50.0.0" rel="alternate"/><published>2025-09-29T00:00:00+00:00</published><updated>2025-09-29T00:00:00+00:00</updated><author><name>pmc</name></author><id>tag:datafusion.apache.org,2025-09-29:/blog/2025/09/29/datafusion-50.0.0</id><summary type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;!-- see https://github.com/apache/datafusion/issues/16347 for details --&gt;
&lt;h2 id="introduction"&gt;Introduction&lt;a class="headerlink" href="#introduction" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;We are proud to announce the release of &lt;a href="https://crates.io/crates/datafusion/50.0.0"&gt;DataFusion 50.0.0&lt;/a&gt;. This blog post
highlights some of the major improvements since the release of &lt;a href="https://datafusion.apache.org/blog/2025/07/28/datafusion-49.0.0/"&gt;DataFusion
49.0.0&lt;/a&gt;. The complete list of changes is available in the &lt;a href="https://github.com/apache/datafusion/blob/branch-50/dev/changelog/50.0.0.md"&gt;changelog&lt;/a&gt;.
Thanks to &lt;a href="https://github.com/apache/datafusion/blob/branch-50/dev/changelog/50.0.0.md#credits"&gt;numerous contributors&lt;/a&gt; for making this release possible!&lt;/p&gt;
&lt;h2 id="performance-improvements"&gt;Performance …&lt;/h2&gt;</summary><content type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;!-- see https://github.com/apache/datafusion/issues/16347 for details --&gt;
&lt;h2 id="introduction"&gt;Introduction&lt;a class="headerlink" href="#introduction" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;We are proud to announce the release of &lt;a href="https://crates.io/crates/datafusion/50.0.0"&gt;DataFusion 50.0.0&lt;/a&gt;. This blog post
highlights some of the major improvements since the release of &lt;a href="https://datafusion.apache.org/blog/2025/07/28/datafusion-49.0.0/"&gt;DataFusion
49.0.0&lt;/a&gt;. The complete list of changes is available in the &lt;a href="https://github.com/apache/datafusion/blob/branch-50/dev/changelog/50.0.0.md"&gt;changelog&lt;/a&gt;.
Thanks to &lt;a href="https://github.com/apache/datafusion/blob/branch-50/dev/changelog/50.0.0.md#credits"&gt;numerous contributors&lt;/a&gt; for making this release possible!&lt;/p&gt;
&lt;h2 id="performance-improvements"&gt;Performance Improvements 🚀&lt;a class="headerlink" href="#performance-improvements" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;DataFusion continues to focus on enhancing performance, as shown in ClickBench
and other benchmark results.&lt;/p&gt;
&lt;p&gt;&lt;img alt="ClickBench performance results over time for DataFusion" class="img-fluid" src="/blog/images/datafusion-50.0.0/performance_over_time_clickbench.png" width="100%"/&gt;&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Figure 1&lt;/strong&gt;: Average and median normalized query execution times for ClickBench queries for each git revision.
Query times are normalized using the ClickBench definition. See the 
&lt;a href="https://alamb.github.io/datafusion-benchmarking/"&gt;DataFusion Benchmarking Page&lt;/a&gt; 
for more details.&lt;/p&gt;
&lt;p&gt;Here are some noteworthy optimizations added since DataFusion 49:&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Dynamic Filter Pushdown Improvements&lt;/strong&gt;&lt;/p&gt;
&lt;p&gt;The dynamic filter pushdown optimization, which allows runtime filters to cut
down on the amount of data read, has been extended to support &lt;strong&gt;inner hash
joins&lt;/strong&gt;, dramatically improving performance when one relation is relatively
small or filtered by a highly selective predicate. More details can be found in
the &lt;a href="#dynamic-filter-pushdown-for-hash-joins"&gt;Dynamic Filter Pushdown for Hash Joins&lt;/a&gt; section below.
The dynamic filters in the TopK operator have also been improved in DataFusion
50.0.0, further increasing the effectiveness and efficiency of the optimization.
More details can be found in this
&lt;a href="https://github.com/apache/datafusion/pull/16433"&gt;ticket&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Nested Loop Join Optimization&lt;/strong&gt;&lt;/p&gt;
&lt;p&gt;The nested loop join operator has been rewritten to reduce execution time and memory
usage by adopting a finer-grained approach. Specifically, we now limit the 
intermediate data size to around a single &lt;code&gt;RecordBatch&lt;/code&gt; for better memory
efficiency, and we have eliminated redundant conversions from the old 
implementation to further improve execution speed.
When evaluating this new approach in a microbenchmark, we measured up to a 5x
improvement in execution time and a 99% reduction in memory usage. More details and
results can be found in this
&lt;a href="https://github.com/apache/datafusion/pull/16996"&gt;ticket&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Parquet Metadata Caching&lt;/strong&gt;&lt;/p&gt;
&lt;p&gt;DataFusion now automatically caches the metadata of Parquet files (statistics,
page indexes, etc.), to avoid unnecessary disk/network round-trips. This is
especially useful when querying the same table multiple times over relatively
slow networks, allowing us to achieve an order of magnitude faster execution
time when running many small reads over large files. More information can be
found in the &lt;a href="#parquet-metadata-cache"&gt;Parquet Metadata Cache&lt;/a&gt; section.&lt;/p&gt;
&lt;h2 id="community-growth"&gt;Community Growth 📈&lt;a class="headerlink" href="#community-growth" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;Between &lt;code&gt;49.0.0&lt;/code&gt; and &lt;code&gt;50.0.0&lt;/code&gt;, we continue to see our community grow:&lt;/p&gt;
&lt;ol&gt;
&lt;li&gt;Qi Zhu (&lt;a href="https://github.com/zhuqi-lucas"&gt;zhuqi-lucas&lt;/a&gt;) and Yoav Cohen
   (&lt;a href="https://github.com/yoavcloud"&gt;yoavcloud&lt;/a&gt;) became committers. See the
   &lt;a href="https://lists.apache.org/list.html?dev@datafusion.apache.org"&gt;mailing list&lt;/a&gt; for more details.&lt;/li&gt;
&lt;li&gt;In the &lt;a href="https://github.com/apache/arrow-datafusion"&gt;core DataFusion repo&lt;/a&gt; alone, we reviewed and accepted 318 PRs
   from 79 different committers, created over 235 issues, and closed 197 of them
   🚀. All changes are listed in the detailed &lt;a href="https://github.com/apache/datafusion/tree/main/dev/changelog"&gt;changelogs&lt;/a&gt;.&lt;/li&gt;
&lt;li&gt;DataFusion published several blogs, including &lt;em&gt;&lt;a href="https://datafusion.apache.org/blog/2025/08/15/external-parquet-indexes/"&gt;Using External Indexes, Metadata Stores, Catalogs and
   Caches to Accelerate Queries on Apache Parquet&lt;/a&gt;&lt;/em&gt;, &lt;em&gt;&lt;a href="https://datafusion.apache.org/blog/2025/09/10/dynamic-filters/"&gt;Dynamic Filters:
   Passing Information Between Operators During Execution for 25x Faster
   Queries&lt;/a&gt;&lt;/em&gt;, and &lt;em&gt;&lt;a href="https://datafusion.apache.org/blog/2025/09/21/custom-types-using-metadata/"&gt;Implementing User Defined Types and Custom Metadata 
   in DataFusion&lt;/a&gt;&lt;/em&gt;.&lt;/li&gt;
&lt;/ol&gt;
&lt;!--
# Unique committers
$ git shortlog -sn 49.0.0..50.0.0  . | wc -l
    79
# commits
$ git log --pretty=oneline 49.0.0..50.0.0  . | wc -l
    318

https://crates.io/crates/datafusion/49.0.0
DataFusion 49 released July 25, 2025

https://crates.io/crates/datafusion/50.0.0
DataFusion 50 released September 16, 2025

Issues created in this time: 117 open, 118 closed = 235 total
https://github.com/apache/datafusion/issues?q=is%3Aissue+created%3A2025-07-25..2025-09-16

Issues closed: 197
https://github.com/apache/arrow-datafusion/issues?q=is%3Aissue+closed%3A2025-07-25..2025-09-16

PRs merged in this time 371
https://github.com/apache/arrow-datafusion/pulls?q=is%3Apr+merged%3A2025-07-25..2025-09-16
--&gt;
&lt;h2 id="new-features"&gt;New Features ✨&lt;a class="headerlink" href="#new-features" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;h3 id="improved-spilling-sorts-for-larger-than-memory-datasets"&gt;Improved Spilling Sorts for Larger-than-Memory Datasets&lt;a class="headerlink" href="#improved-spilling-sorts-for-larger-than-memory-datasets" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;DataFusion has long been able to sort datasets that do not fit entirely in memory,
but still struggled with particularly large inputs or highly memory-constrained 
setups. Larger-than-memory sorts in DataFusion 50.0.0 have been improved with the recent introduction
of multi-level merge sorts (more details in the respective
&lt;a href="https://github.com/apache/datafusion/pull/15700"&gt;ticket&lt;/a&gt;). It is now
possible to execute almost any sorting query that would have previously triggered &lt;em&gt;out-of-memory&lt;/em&gt;
errors, by relying on disk spilling. Thanks to &lt;a href="https://github.com/rluvaton"&gt;Raz Luvaton&lt;/a&gt;, &lt;a href="https://github.com/2010YOUY01"&gt;Yongting You&lt;/a&gt;, and
&lt;a href="https://github.com/ding-young"&gt;ding-young&lt;/a&gt; for delivering this feature.&lt;/p&gt;
&lt;h3 id="dynamic-filter-pushdown-for-hash-joins"&gt;Dynamic Filter Pushdown for Hash Joins&lt;a class="headerlink" href="#dynamic-filter-pushdown-for-hash-joins" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;The &lt;a href="https://datafusion.apache.org/blog/2025/09/10/dynamic-filters/"&gt;dynamic filter pushdown
optimization&lt;/a&gt;
has been extended to inner hash joins, dramatically reducing the amount of
scanned data in some workloads—a technique sometimes referred to as
&lt;a href="https://www.cs.cmu.edu/~15721-f24/papers/Sideways_Information_Passing.pdf"&gt;&lt;em&gt;Sideways Information Passing&lt;/em&gt;&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;These filters are automatically applied to inner hash joins, while future work
will introduce them to other join types. &lt;/p&gt;
&lt;p&gt;For example, given a query that looks for a specific customer and
their orders, DataFusion can now filter the &lt;code&gt;orders&lt;/code&gt; relation based on the
&lt;code&gt;c_custkey&lt;/code&gt; of the target customer, reducing the amount of data
read from disk by orders of magnitude.&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-sql"&gt;-- retrieve the orders of the customer with c_phone = '25-989-741-2988'
SELECT *
FROM customer
JOIN orders ON c_custkey = o_custkey
WHERE c_phone = '25-989-741-2988';
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;The following shows an execution plan in DataFusion 50.0.0 with this optimization:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-sql"&gt;HashJoinExec
    DataSourceExec: &amp;lt;-- read customer
      predicate=c_phone@4 = 25-989-741-2988
      metrics=[output_rows=1, ...]
    DataSourceExec: &amp;lt;-- read orders
      -- dynamic filter is added here, filtering directly at scan time
      predicate=DynamicFilterPhysicalExpr [ o_custkey@1 &amp;gt;= 1 AND o_custkey@1 &amp;lt;= 1 ]
      -- the number of output rows is kept to a minimum
      metrics=[output_rows=11, ...]
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;Because there is a single customer in this query,
almost all rows from &lt;code&gt;orders&lt;/code&gt; are filtered out by the join. 
In previous versions of DataFusion, the entire &lt;code&gt;orders&lt;/code&gt; relation would be
scanned to join with the target customer, but now the dynamic filter pushdown can
filter it right at the source, minimizing the amount of data decoded.&lt;/p&gt;
&lt;p&gt;More information can be found in the respective
&lt;a href="https://github.com/apache/datafusion/pull/16445"&gt;ticket&lt;/a&gt; and the next step will be to
&lt;a href="https://github.com/apache/datafusion/issues/16973"&gt;extend the dynamic filters to other types of joins&lt;/a&gt;, such as &lt;code&gt;LEFT&lt;/code&gt; and
&lt;code&gt;RIGHT&lt;/code&gt; outer joins. Thanks to &lt;a href="https://github.com/adriangb"&gt;Adrian Garcia Badaracco&lt;/a&gt;, &lt;a href="https://github.com/zhuqi-lucas"&gt;Qi Zhu&lt;/a&gt;, &lt;a href="https://github.com/xudong963"&gt;xudong963&lt;/a&gt;, &lt;a href="https://github.com/Dandandan"&gt;Daniël Heres&lt;/a&gt;, and &lt;a href="https://github.com/LiaCastaneda"&gt;Lía Adriana&lt;/a&gt;
for delivering this feature.&lt;/p&gt;
&lt;h3 id="parquet-metadata-cache"&gt;Parquet Metadata Cache&lt;a class="headerlink" href="#parquet-metadata-cache" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;The metadata of Parquet files (statistics, page indexes, etc.) is now
automatically cached when using the built-in &lt;a href="https://docs.rs/datafusion/latest/datafusion/datasource/listing/struct.ListingTable.html"&gt;ListingTable&lt;/a&gt;, which reduces disk/network round-trips and repeated decoding
of the same information. With a simple microbenchmark that executes point reads
(e.g., &lt;code&gt;SELECT v FROM t WHERE k = x&lt;/code&gt;) over large files, we measured a 12x
improvement in execution time (more details can be found in the respective
&lt;a href="https://github.com/apache/datafusion/pull/16971"&gt;ticket&lt;/a&gt;). This optimization
is production ready and enabled by default (more details in the
&lt;a href="https://github.com/apache/datafusion/issues/17000"&gt;Epic&lt;/a&gt;).
Thanks to &lt;a href="https://github.com/nuno-faria"&gt;Nuno Faria&lt;/a&gt;, &lt;a href="https://github.com/jonathanc-n"&gt;Jonathan Chen&lt;/a&gt;, &lt;a href="https://github.com/shehabgamin"&gt;Shehab Amin&lt;/a&gt;, &lt;a href="https://github.com/comphead"&gt;Oleks V&lt;/a&gt;, &lt;a href="https://github.com/timsaucer"&gt;Tim Saucer&lt;/a&gt;, and &lt;a href="https://github.com/BlakeOrth"&gt;Blake Orth&lt;/a&gt; for delivering this feature.&lt;/p&gt;
&lt;p&gt;Here is an example of the metadata cache in action:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-sql"&gt;-- disabling the metadata cache
&amp;gt; SET datafusion.runtime.metadata_cache_limit = '0M';

-- simple query (t.parquet: 100M rows, 3 cols)
&amp;gt; EXPLAIN ANALYZE SELECT * FROM 't.parquet' LIMIT 1;
DataSourceExec: ... metrics=[..., metadata_load_time=229.196422ms, ...]
Elapsed 0.246 seconds.

-- enabling the metadata cache
&amp;gt; SET datafusion.runtime.metadata_cache_limit = '50M';

&amp;gt; EXPLAIN ANALYZE SELECT * FROM 't.parquet' LIMIT 1;
DataSourceExec: ... metrics=[..., metadata_load_time=228.612µs, ...]
Elapsed 0.003 seconds. -- 82x improvement in this specific query
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;The cache can be configured with the following runtime parameter:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-sql"&gt;datafusion.runtime.metadata_cache_limit
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;The default &lt;a href="https://docs.rs/datafusion/latest/datafusion/execution/cache/cache_manager/trait.FileMetadataCache.html"&gt;&lt;code&gt;FileMetadataCache&lt;/code&gt;&lt;/a&gt; uses a
least-recently-used eviction algorithm and up to 50MB of memory.
If the underlying file changes, the cache is automatically invalidated.
Setting the limit to 0 will disable any metadata caching. As with most APIs in
DataFusion, users can provide their own behavior using a custom
&lt;a href="https://docs.rs/datafusion/50.0.0/datafusion/execution/cache/cache_manager/trait.FileMetadataCache.html"&gt;&lt;code&gt;FileMetadataCache&lt;/code&gt;&lt;/a&gt;
implementation when setting up the &lt;a href="https://docs.rs/datafusion/latest/datafusion/execution/runtime_env/struct.RuntimeEnv.html"&gt;&lt;code&gt;RuntimeEnv&lt;/code&gt;&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;For users with custom &lt;a href="https://docs.rs/datafusion/latest/datafusion/catalog/trait.TableProvider.html"&gt;&lt;code&gt;TableProvider&lt;/code&gt;&lt;/a&gt;:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;
&lt;p&gt;If the custom provider uses the
&lt;a href="https://docs.rs/datafusion/latest/datafusion/datasource/file_format/parquet/struct.ParquetFormat.html"&gt;&lt;code&gt;ParquetFormat&lt;/code&gt;&lt;/a&gt;, caching will work
without any changes.&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;
&lt;p&gt;Otherwise the
&lt;a href="https://docs.rs/datafusion/latest/datafusion/datasource/physical_plan/parquet/struct.CachedParquetFileReaderFactory.html"&gt;&lt;code&gt;CachedParquetFileReaderFactory&lt;/code&gt;&lt;/a&gt;
can be provided when creating a
&lt;a href="https://docs.rs/datafusion/latest/datafusion/datasource/physical_plan/struct.ParquetSource.html"&gt;&lt;code&gt;ParquetSource&lt;/code&gt;&lt;/a&gt;.&lt;/p&gt;
&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;Users can inspect the cache contents through the
&lt;a href="https://docs.rs/datafusion/latest/datafusion/execution/cache/cache_manager/trait.FileMetadataCache.html#tymethod.list_entries"&gt;&lt;code&gt;FileMetadataCache::list_entries&lt;/code&gt;&lt;/a&gt;
method, or with the
&lt;a href="https://datafusion.apache.org/user-guide/cli/functions.html#metadata-cache"&gt;&lt;code&gt;metadata_cache()&lt;/code&gt;&lt;/a&gt;
function in &lt;code&gt;datafusion-cli&lt;/code&gt;:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-sql"&gt;&amp;gt; SELECT * FROM metadata_cache();
+---------------+-------------------------+-----------------+--------------------------+---------+---------------------+------+-----------------+
| path          | file_modified           | file_size_bytes | e_tag                    | version | metadata_size_bytes | hits | extra           |
+---------------+-------------------------+-----------------+--------------------------+---------+---------------------+------+-----------------+
| .../t.parquet | 2025-09-21T17:40:13.650 | 420827020       | 0-63f5331fb4458-19154f8c | NULL    | 44480534            | 27   | page_index=true |
+---------------+-------------------------+-----------------+--------------------------+---------+---------------------+------+-----------------+
1 row(s) fetched.
Elapsed 0.003 seconds.
&lt;/code&gt;&lt;/pre&gt;
&lt;h3 id="qualify-clause"&gt;&lt;code&gt;QUALIFY&lt;/code&gt; Clause&lt;a class="headerlink" href="#qualify-clause" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;DataFusion now supports the &lt;code&gt;QUALIFY&lt;/code&gt; SQL clause
(&lt;a href="https://github.com/apache/datafusion/pull/16933"&gt;#16933&lt;/a&gt;), which simplifies
filtering window function output (similar to how &lt;code&gt;HAVING&lt;/code&gt; filters
aggregation output).&lt;/p&gt;
&lt;p&gt;For example, filtering the output of the &lt;code&gt;rank()&lt;/code&gt; function previously
required a query like this:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-sql"&gt;SELECT a, b, c
FROM (
   SELECT a, b, c, rank() OVER(PARTITION BY a ORDER BY b) as rk
   FROM t
)
WHERE rk = 1
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;The same query can now be written like this:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-sql"&gt;SELECT a, b, c, rank() OVER(PARTITION BY a ORDER BY b) as rk
FROM t
QUALIFY rk = 1
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;Although it is not part of the SQL standard (yet), it has been gaining
adoption in several SQL analytical systems such as DuckDB, Snowflake, and
BigQuery. Thanks to &lt;a href="https://github.com/haohuaijin"&gt;Huaijin&lt;/a&gt; and &lt;a href="https://github.com/jonahgao"&gt;Jonah Gao&lt;/a&gt; for delivering this feature.&lt;/p&gt;
&lt;h3 id="filter-support-for-window-functions"&gt;&lt;code&gt;FILTER&lt;/code&gt; Support for Window Functions&lt;a class="headerlink" href="#filter-support-for-window-functions" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;Continuing the theme, the &lt;code&gt;FILTER&lt;/code&gt; clause has been extended to support
&lt;a href="https://github.com/apache/datafusion/pull/17378"&gt;aggregate window functions&lt;/a&gt;.
It allows these functions to apply to specific rows without having to
rely on &lt;code&gt;CASE&lt;/code&gt; expressions, similar to what was already possible with regular
aggregate functions.&lt;/p&gt;
&lt;p&gt;For example, we can gather multiple distinct sets of values matching different
criteria with a single pass over the input:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-sql"&gt;SELECT 
  ARRAY_AGG(c2) FILTER (WHERE c2 &amp;gt;= 2) OVER (...)     -- e.g. [2, 3, 4]
  ARRAY_AGG(CASE WHEN c2 &amp;gt;= 2 THEN c2 END) OVER (...) -- e.g. [NULL, NULL, 2, 3, 4]
...
FROM table
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;Thanks to &lt;a href="https://github.com/geoffreyclaude"&gt;Geoffrey Claude&lt;/a&gt; and &lt;a href="https://github.com/Jefffrey"&gt;Jeffrey Vo&lt;/a&gt; for delivering this feature.&lt;/p&gt;
&lt;h3 id="configoptions-now-available-to-functions"&gt;&lt;code&gt;ConfigOptions&lt;/code&gt; Now Available to Functions&lt;a class="headerlink" href="#configoptions-now-available-to-functions" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;DataFusion 50.0.0 now passes session configuration parameters to User-Defined
Functions (UDFs) via
&lt;a href="https://docs.rs/datafusion/latest/datafusion/logical_expr/struct.ScalarFunctionArgs.html"&gt;ScalarFunctionArgs&lt;/a&gt;
(&lt;a href="https://github.com/apache/datafusion/pull/16970"&gt;#16970&lt;/a&gt;). This allows
behavior that varies based on runtime state; for example, time UDFs can use the
session-specified time zone instead of just UTC.&lt;/p&gt;
&lt;p&gt;Thanks to &lt;a href="https://github.com/Omega359"&gt;Bruce Ritchie&lt;/a&gt;, &lt;a href="https://github.com/findepi"&gt;Piotr Findeisen&lt;/a&gt;, &lt;a href="https://github.com/comphead"&gt;Oleks V&lt;/a&gt;, and &lt;a href="https://github.com/alamb"&gt;Andrew Lamb&lt;/a&gt; for delivering this feature.&lt;/p&gt;
&lt;h3 id="additional-apache-spark-compatible-functions"&gt;Additional Apache Spark Compatible Functions&lt;a class="headerlink" href="#additional-apache-spark-compatible-functions" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;Finally, due to Apache Spark's impact on analytical processing, many DataFusion
users desire Spark compatibility in their workloads, so DataFusion provides a
set of Spark-compatible functions in the &lt;a href="https://crates.io/crates/datafusion-spark"&gt;datafusion-spark&lt;/a&gt; crate.
You can read more about this project in the &lt;a href="https://datafusion.apache.org/blog/2025/07/16/datafusion-48.0.0/#new-datafusion-spark-crate"&gt;announcement&lt;/a&gt; and &lt;a href="https://github.com/apache/datafusion/issues/15914"&gt;epic&lt;/a&gt;.
DataFusion 50.0.0 adds several new such functions:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;a href="https://github.com/apache/datafusion/pull/16936"&gt;&lt;code&gt;array&lt;/code&gt;&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;&lt;a href="https://github.com/apache/datafusion/pull/16942"&gt;&lt;code&gt;bit_get/bit_count&lt;/code&gt;&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;&lt;a href="https://github.com/apache/datafusion/pull/17179"&gt;&lt;code&gt;bitmap_count&lt;/code&gt;&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;&lt;a href="https://github.com/apache/datafusion/pull/17032"&gt;&lt;code&gt;crc32/sha1&lt;/code&gt;&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;&lt;a href="https://github.com/apache/datafusion/pull/17024"&gt;&lt;code&gt;date_add/date_sub&lt;/code&gt;&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;&lt;a href="https://github.com/apache/datafusion/pull/16946"&gt;&lt;code&gt;if&lt;/code&gt;&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;&lt;a href="https://github.com/apache/datafusion/pull/16828"&gt;&lt;code&gt;last_day&lt;/code&gt;&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;&lt;a href="https://github.com/apache/datafusion/pull/16962"&gt;&lt;code&gt;like/ilike&lt;/code&gt;&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;&lt;a href="https://github.com/apache/datafusion/pull/16848"&gt;&lt;code&gt;luhn_check&lt;/code&gt;&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;&lt;a href="https://github.com/apache/datafusion/pull/16829"&gt;&lt;code&gt;mod/pmod&lt;/code&gt;&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;&lt;a href="https://github.com/apache/datafusion/pull/16780"&gt;&lt;code&gt;next_day&lt;/code&gt;&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;&lt;a href="https://github.com/apache/datafusion/pull/16937"&gt;&lt;code&gt;parse_url&lt;/code&gt;&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;&lt;a href="https://github.com/apache/datafusion/pull/16924"&gt;&lt;code&gt;rint&lt;/code&gt;&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;&lt;a href="https://github.com/apache/datafusion/pull/17331"&gt;&lt;code&gt;width_bucket&lt;/code&gt;&lt;/a&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;Thanks to &lt;a href="https://github.com/davidlghellin"&gt;David López&lt;/a&gt;, &lt;a href="https://github.com/chenkovsky"&gt;Chen Chongchen&lt;/a&gt;, &lt;a href="https://github.com/Standing-Man"&gt;Alan Tang&lt;/a&gt;, &lt;a href="https://github.com/petern48"&gt;Peter Nguyen&lt;/a&gt;, and &lt;a href="https://github.com/SparkApplicationMaster"&gt;Evgenii Glotov&lt;/a&gt; for delivering these functions. We are looking for additional help
reviewing and implementing more functions; please reach out on the &lt;a href="https://github.com/apache/datafusion/issues/15914"&gt;epic&lt;/a&gt; if you are interested.&lt;/p&gt;
&lt;h2 id="known-issues-patchset"&gt;Known Issues / Patchset&lt;a class="headerlink" href="#known-issues-patchset" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;As DataFusion continues to mature, we regularly release patch versions to fix issues 
in major releases. Since the release of &lt;code&gt;50.0.0&lt;/code&gt;, we have identified a few
issues, and expect to release &lt;code&gt;50.1.0&lt;/code&gt; to address them. You can track progress
in this &lt;a href="https://github.com/apache/datafusion/issues/17594"&gt;ticket&lt;/a&gt;. &lt;/p&gt;
&lt;h2 id="upgrade-guide-and-changelog"&gt;Upgrade Guide and Changelog&lt;a class="headerlink" href="#upgrade-guide-and-changelog" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;Upgrading to 50.0.0 should be straightforward for most users. Please review the
&lt;a href="https://datafusion.apache.org/library-user-guide/upgrading.html"&gt;Upgrade Guide&lt;/a&gt;
for details on breaking changes and code snippets to help with the transition.
Recently, some users have reported success automatically upgrading DataFusion by
pairing AI tools with the upgrade guide. For a comprehensive list of all
changes, please refer to the &lt;a href="https://github.com/apache/datafusion/blob/branch-50/dev/changelog/50.0.0.md"&gt;changelog&lt;/a&gt;.&lt;/p&gt;
&lt;h2 id="about-datafusion"&gt;About DataFusion&lt;a class="headerlink" href="#about-datafusion" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;&lt;a href="https://datafusion.apache.org/"&gt;Apache DataFusion&lt;/a&gt; is an extensible query engine, written in &lt;a href="https://www.rust-lang.org/"&gt;Rust&lt;/a&gt;, that uses
&lt;a href="https://arrow.apache.org"&gt;Apache Arrow&lt;/a&gt; as its in-memory format. DataFusion is used by developers to
create new, fast, data-centric systems such as databases, dataframe libraries,
and machine learning and streaming applications. While &lt;a href="https://datafusion.apache.org/user-guide/introduction.html#project-goals"&gt;DataFusion’s primary
design goal&lt;/a&gt; is to accelerate the creation of other data-centric systems, it
provides a reasonable experience directly out of the box as a &lt;a href="https://datafusion.apache.org/user-guide/dataframe.html"&gt;dataframe
library&lt;/a&gt;, &lt;a href="https://datafusion.apache.org/python/"&gt;Python library&lt;/a&gt;, and &lt;a href="https://datafusion.apache.org/user-guide/cli/"&gt;command-line SQL tool&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;DataFusion's core thesis is that, as a community, together we can build much
more advanced technology than any of us as individuals or companies could build
alone. Without DataFusion, highly performant vectorized query engines would
remain the domain of a few large companies and world-class research
institutions. With DataFusion, we can all build on top of a shared foundation
and focus on what makes our projects unique.&lt;/p&gt;
&lt;h2 id="how-to-get-involved"&gt;How to Get Involved&lt;a class="headerlink" href="#how-to-get-involved" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;DataFusion is not a project built or driven by a single person, company, or
foundation. Rather, our community of users and contributors works together to
build a shared technology that none of us could have built alone.&lt;/p&gt;
&lt;p&gt;If you are interested in joining us, we would love to have you. You can try out
DataFusion on some of your own data and projects and let us know how it goes,
contribute suggestions, documentation, bug reports, or a PR with documentation,
tests, or code. A list of open issues suitable for beginners is &lt;a href="https://github.com/apache/arrow-datafusion/issues?q=is%3Aissue+is%3Aopen+label%3A%22good+first+issue%22"&gt;here&lt;/a&gt;, and you
can find out how to reach us on the &lt;a href="https://datafusion.apache.org/contributor-guide/communication.html"&gt;communication doc&lt;/a&gt;.&lt;/p&gt;</content><category term="blog"/></entry><entry><title>Implementing User Defined Types and Custom Metadata in DataFusion</title><link href="https://datafusion.apache.org/blog/2025/09/21/custom-types-using-metadata" rel="alternate"/><published>2025-09-21T00:00:00+00:00</published><updated>2025-09-21T00:00:00+00:00</updated><author><name>Tim Saucer(rerun.io), Dewey Dunnington(Wherobots), Andrew Lamb(InfluxData)</name></author><id>tag:datafusion.apache.org,2025-09-21:/blog/2025/09/21/custom-types-using-metadata</id><summary type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}x
--&gt;

&lt;p&gt;&lt;a href="https://datafusion.apache.org/blog/2025/07/16/datafusion-48.0.0/"&gt;Apache DataFusion&lt;/a&gt; significantly improves support for user
defined types and metadata. The user defined function APIs let users access
metadata on the input columns to functions and produce metadata in the output.&lt;/p&gt;
&lt;h2 id="user-defined-types-extension-types"&gt;User defined types == extension types&lt;a class="headerlink" href="#user-defined-types-extension-types" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;DataFusion directly uses &lt;a href="https://arrow.apache.org"&gt;Apache Arrow&lt;/a&gt;'s &lt;a href="https://docs.rs/arrow/latest/arrow/datatypes/enum.DataType.html"&gt;DataTypes&lt;/a&gt; as its type system. This
has …&lt;/p&gt;</summary><content type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}x
--&gt;

&lt;p&gt;&lt;a href="https://datafusion.apache.org/blog/2025/07/16/datafusion-48.0.0/"&gt;Apache DataFusion&lt;/a&gt; significantly improves support for user
defined types and metadata. The user defined function APIs let users access
metadata on the input columns to functions and produce metadata in the output.&lt;/p&gt;
&lt;h2 id="user-defined-types-extension-types"&gt;User defined types == extension types&lt;a class="headerlink" href="#user-defined-types-extension-types" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;DataFusion directly uses &lt;a href="https://arrow.apache.org"&gt;Apache Arrow&lt;/a&gt;'s &lt;a href="https://docs.rs/arrow/latest/arrow/datatypes/enum.DataType.html"&gt;DataTypes&lt;/a&gt; as its type system. This
has several benefits including being simple to explain, supports a rich set of
both scalar and nested types, true zero copy interoperability with other Arrow
implementations, and world-class library support (via &lt;a href="https://github.com/apache/arrow-rs"&gt;arrow-rs&lt;/a&gt;). However, one
challenge of directly using the Arrow type system is there is no distinction
between logical types and physical types. For example, the Arrow type system
contains multiple types which can store "String"s (sequences of UTF8 encoded
bytes) such as &lt;code&gt;Utf8&lt;/code&gt;, &lt;code&gt;LargeUTF8&lt;/code&gt;, &lt;code&gt;Dictionary(Utf8)&lt;/code&gt;, and &lt;code&gt;Utf8View&lt;/code&gt;. &lt;/p&gt;
&lt;p&gt;However, Apache Arrow does provide &lt;a href="https://arrow.apache.org/docs/format/Columnar.html#format-metadata-extension-types"&gt;extension types&lt;/a&gt;, a version of logical type
information, which describe how to interpret data stored in one of the existing
physical types. With the improved support for metadata in DataFusion 48.0.0, it
is now easier to implement user defined types using Arrow extension types.&lt;/p&gt;
&lt;h2 id="metadata-in-apache-arrow-fields"&gt;Metadata in Apache Arrow &lt;code&gt;Field&lt;/code&gt;s&lt;a class="headerlink" href="#metadata-in-apache-arrow-fields" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;The &lt;a href="https://arrow.apache.org/docs/format/Columnar.html"&gt;Arrow specification&lt;/a&gt; defines Metadata as a map of key-value pairs of
strings. This metadata is used to attach extension types and use case-specific
context to a column of values. The Rust implementation of Apache Arrow,
&lt;a href="https://github.com/apache/arrow-rs"&gt;arrow-rs&lt;/a&gt;, stores metadata on &lt;a href="https://arrow.apache.org/docs/format/Glossary.html#term-field"&gt;Field&lt;/a&gt;s, but prior to DataFusion 48.0.0, many of
DataFusion's internal APIs used &lt;a href="https://docs.rs/arrow/latest/arrow/datatypes/enum.DataType.html"&gt;DataTypes&lt;/a&gt; directly, and thus did not propagate
metadata through all operations.&lt;/p&gt;
&lt;p&gt;In previous versions of DataFusion &lt;code&gt;Field&lt;/code&gt; metadata was propagated through certain
operations (e.g., renaming or selecting a column) but was not 
others (e.g., scalar, window, or aggregate function calls). In DataFusion 48.0.0, 
and later, all user defined functions are passed the full
input &lt;code&gt;Field&lt;/code&gt; information and can return &lt;code&gt;Field&lt;/code&gt; information to the caller.&lt;/p&gt;
&lt;p&gt;Supporting extension types was a key motivation for adding metadata to the
function processing, the same mechanism can store arbitrary metadata on the
input and output fields, which supports other interesting use cases as we describe
later in this post.&lt;/p&gt;
&lt;h2 id="metadata-handling"&gt;Metadata handling&lt;a class="headerlink" href="#metadata-handling" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;Data in Arrow record batches carry a &lt;a href="https://docs.rs/arrow/latest/arrow/datatypes/struct.Schema.html"&gt;Schema&lt;/a&gt; in addition to the Arrow arrays. Each
&lt;a href="https://arrow.apache.org/docs/format/Glossary.html#term-field"&gt;Field&lt;/a&gt; in this &lt;code&gt;Schema&lt;/code&gt; contains a name, data type, nullability, and metadata. The
metadata is specified as a map of key-value pairs of strings.  In the new
implementation, during processing of all user defined functions we pass the input
field information.&lt;/p&gt;
&lt;figure&gt;
&lt;img alt="Relationship between a Record Batch, it's schema, and the underlying arrays. There is a one to one relationship between each Field in the Schema and Array entry in the Columns." class="img-fluid" src="/blog/images/metadata-handling/arrow_record_batch.png" width="100%"/&gt;
&lt;figcaption&gt;
&lt;b&gt;Figure 1:&lt;/b&gt; Relationship between a Record Batch, it's schema, and the underlying arrays. There is a one to one relationship between each Field in the Schema and Array entry in the Columns.
  &lt;/figcaption&gt;
&lt;/figure&gt;
&lt;p&gt;It is often desirable to write a generic function for reuse. Prior versions of
user defined functions only had access to the &lt;code&gt;DataType&lt;/code&gt; of the input columns.
This works well for some features that only rely on the types of data, but other
use cases may need additional information that describes the data.&lt;/p&gt;
&lt;p&gt;For example, suppose I wish to write a function that takes in a UUID and returns a string
of the &lt;a href="https://www.ietf.org/rfc/rfc9562.html#section-4.1"&gt;variant&lt;/a&gt; of the input field. We would want this function to be able to handle
all of the string types and also a binary encoded UUID. The Arrow specification does not
contain a unsigned 128 bit value, it is common to encode a UUID as a fixed sized binary
array where each element is 16 bytes long. With the metadata handling in [DataFusion 48.0.0]
we can validate during planning that the input data not only has the correct underlying
data type, but that it also represents the right &lt;em&gt;kind&lt;/em&gt; of data. The UUID example is a
common one, and it is included in the &lt;a href="https://arrow.apache.org/docs/format/CanonicalExtensions.html"&gt;canonical extension types&lt;/a&gt; that are now
supported in DataFusion.&lt;/p&gt;
&lt;p&gt;Another common application of metadata handling is understanding encoding of a blob of data.
Suppose you have a column that contains image data. Most likely this data is stored as
an array of &lt;code&gt;u8&lt;/code&gt; data. Without knowing a priori what the encoding of that blob of data is,
you cannot ensure you are using the correct methods for decoding it. You may work around
this by adding another column to your data source indicating the encoding, but this can be
wasteful for systems where the encoding never changes. Instead, you could use metadata to
specify the encoding for the entire column.&lt;/p&gt;
&lt;h2 id="how-to-use-metadata-in-user-defined-functions"&gt;How to use metadata in user defined functions&lt;a class="headerlink" href="#how-to-use-metadata-in-user-defined-functions" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;When working with metadata for &lt;a href="https://docs.rs/datafusion/latest/datafusion/logical_expr/trait.ScalarUDFImpl.html"&gt;user defined scalar functions&lt;/a&gt;, there are typically two
places in the function definition that require implementation.&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;Computing the return field from the arguments&lt;/li&gt;
&lt;li&gt;Invocation&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;During planning, we will attempt to call the function &lt;a href="https://docs.rs/datafusion/latest/datafusion/logical_expr/trait.ScalarUDFImpl.html#method.return_field_from_args"&gt;return_field_from_args()&lt;/a&gt;. This will
provide a list of input fields to the function and return the output field. To evaluate
metadata on the input side, you can write a functions similar to this example:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-rust"&gt;fn return_field_from_args(
    &amp;amp;self,
    args: ReturnFieldArgs,
) -&amp;gt; datafusion::common::Result&amp;lt;FieldRef&amp;gt; {
    if args.arg_fields.len() != 1 {
        return exec_err!("Incorrect number of arguments for uuid_version");
    }

    let input_field = &amp;amp;args.arg_fields[0];
    if &amp;amp;DataType::FixedSizeBinary(16) == input_field.data_type() {
        let Ok(CanonicalExtensionType::Uuid(_)) = input_field.try_canonical_extension_type()
        else {
            return exec_err!("Input field must contain the UUID canonical extension type");
        };
    }

    let is_nullable = args.arg_fields[0].is_nullable();

    Ok(Arc::new(Field::new(self.name(), DataType::UInt32, is_nullable)))
}
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;In this example, we take advantage of the fact that we already have support for extension
types that evaluate metadata. If you were attempting to check for metadata other than
extension type support, we could have instead written a snippet such as:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-rust"&gt;    if &amp;amp;DataType::FixedSizeBinary(16) == input_field.data_type() {
        let _ = input_field
            .metadata()
            .get("ARROW:extension:metadata")
            .ok_or(exec_datafusion_err!("Input field must contain the UUID canonical extension type"))?;
        };
    }
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;If you are writing a user defined function that will instead return metadata on output
you can add this directly into the &lt;code&gt;Field&lt;/code&gt; that is the output of the &lt;code&gt;return_field_from_args&lt;/code&gt;
call. In our above example, we could change the return line to:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-rust"&gt;    Ok(Arc::new(
        Field::new(self.name(), DataType::UInt32, is_nullable).with_metadata(
            [("my_key".to_string(), "my_value".to_string())]
                .into_iter()
                .collect(),
        ),
    ))
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;By checking the metadata during the planning process, we can identify errors early in
the query process. There are cases were we wish to have access to this metadata during
execution as well. The function &lt;a href="https://docs.rs/datafusion/latest/datafusion/logical_expr/trait.ScalarUDFImpl.html#tymethod.invoke_with_args"&gt;invoke_with_args&lt;/a&gt; in the user defined function takes
the updated struct &lt;a href="https://docs.rs/datafusion/latest/datafusion/logical_expr/struct.ScalarFunctionArgs.html"&gt;ScalarFunctionArgs&lt;/a&gt;. This now contains the input fields, which can
be used to check for metadata. For example, you can do the following:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-rust"&gt;fn invoke_with_args(&amp;amp;self, args: ScalarFunctionArgs) -&amp;gt; Result&amp;lt;ColumnarValue&amp;gt; {
    assert_eq!(args.arg_fields.len(), 1);
    let my_value = args.arg_fields[0]
        .metadata()
        .get("encoding_type");
    ...
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;In this snippet we have extracted an &lt;code&gt;Option&amp;lt;String&amp;gt;&lt;/code&gt; from the input field metadata
which we can then use to determine which functions we might want to call. We could
then parse the returned value to determine what type of encoding to use when
evaluating the array in the arguments. Since &lt;code&gt;return_field_from_args&lt;/code&gt; is not &lt;code&gt;&amp;amp;mut self&lt;/code&gt;
this check could not be performed during the planning stage.&lt;/p&gt;
&lt;p&gt;The description in this section applies to scalar user defined functions, but equivalent
support exists for aggregate and window functions.&lt;/p&gt;
&lt;h2 id="extension-types"&gt;Extension types&lt;a class="headerlink" href="#extension-types" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;Extension types are one of the primary motivations for this  enhancement in
[Datafusion 48.0.0]. The official Rust implementation of Apache Arrow, &lt;a href="https://github.com/apache/arrow-rs"&gt;arrow-rs&lt;/a&gt;,
already contains support for the &lt;a href="https://arrow.apache.org/docs/format/CanonicalExtensions.html"&gt;canonical extension types&lt;/a&gt;. This support includes
helper functions such as &lt;code&gt;try_canonical_extension_type()&lt;/code&gt; in the earlier example.&lt;/p&gt;
&lt;p&gt;For a concrete example of how extension types can be used in DataFusion functions,
there is an &lt;a href="https://github.com/timsaucer/datafusion_extension_type_examples"&gt;example repository&lt;/a&gt; that demonstrates using UUIDs. The UUID extension
type specifies that the data are stored as a Fixed Size Binary of length 16. In the
DataFusion core functions, we have the ability to generate string representations of
UUIDs that match the version 4 specification. These are helpful, but a user may
wish to do additional work with UUIDs where having them in the dense representation
is preferable. Alternatively, the user may already have data with the binary encoding
and we want to extract values such as the version, timestamp, or string
representation.&lt;/p&gt;
&lt;p&gt;In the example repository we have created three user defined functions: &lt;code&gt;UuidVersion&lt;/code&gt;,
&lt;code&gt;StringToUuid&lt;/code&gt;, and &lt;code&gt;UuidToString&lt;/code&gt;. Each of these implements &lt;code&gt;ScalarUDFImpl&lt;/code&gt; and can
be used thusly:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-rust"&gt;async fn main() -&amp;gt; Result&amp;lt;()&amp;gt; {
    let ctx = create_context()?;

    // get a DataFrame from the context
    let mut df = ctx.table("t").await?;

    // Create the string UUIDs
    df = df.select(vec![uuid().alias("string_uuid")])?;

    // Convert string UUIDs to canonical extension UUIDs
    let string_to_uuid = ScalarUDF::new_from_impl(StringToUuid::default());
    df = df.with_column("uuid", string_to_uuid.call(vec![col("string_uuid")]))?;

    // Extract version number from canonical extension UUIDs
    let version = ScalarUDF::new_from_impl(UuidVersion::default());
    df = df.with_column("version", version.call(vec![col("uuid")]))?;

    // Convert back to a string
    let uuid_to_string = ScalarUDF::new_from_impl(UuidToString::default());
    df = df.with_column("string_round_trip", uuid_to_string.call(vec![col("uuid")]))?;

    df.show().await?;

    Ok(())
}
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;The &lt;a href="https://github.com/timsaucer/datafusion_extension_type_examples"&gt;example repository&lt;/a&gt; also contains a crate that demonstrates how to expose these
UDFs to &lt;a href="https://datafusion.apache.org/python/"&gt;datafusion-python&lt;/a&gt;. This requires version 48.0.0 or later.&lt;/p&gt;
&lt;h2 id="other-use-cases"&gt;Other use cases&lt;a class="headerlink" href="#other-use-cases" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;The metadata attached to the fields can be used to store &lt;em&gt;any&lt;/em&gt; user data in key/value
pairs. Some of the other use cases that have been identified include:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;Creating output for downstream systems. One user of DataFusion produces
  &lt;a href="https://rerun.io/blog/column-chunks"&gt;data visualizations&lt;/a&gt; that are dependant upon metadata in record batch fields. By
  enabling metadata on output of user defined functions, we can now produce batches
  that are directly consumable by these systems.&lt;/li&gt;
&lt;li&gt;Describe the relationships between columns of data. You can store data about how
  one column of data relates to another and use these during function evaluation. For
  example, in robotics it is common to use &lt;a href="https://wiki.ros.org/tf2"&gt;transforms&lt;/a&gt; to describe how to convert
  from one coordinate system to another. It can be convenient to send the function
  all the columns that contain transform information and then allow the function
  to determine which columns to use based on the metadata. This allows for
  encapsulation of the transform logic within the user function.&lt;/li&gt;
&lt;li&gt;Storing logical types of the data model. &lt;a href="https://docs.influxdata.com/influxdb/v1/concepts/schema_and_data_layout/"&gt;InfluxDB&lt;/a&gt; uses field metadata to specify
  which columns are used for tags, times, and fields.&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;Based on the experience of the authors, we recommend caution when using metadata
for use cases other than type extension. One issue that can arises is that as columns
are used to compute new fields, some functions may pass through the metadata and the
semantic meaning may change. For example, suppose you decided to use metadata to
store some kind of statistics for the entire stream of record batches. Then you pass
that column through a filter that removes many rows of data. Your statistics
metadata may now be invalid, even though it was passed through the filter.&lt;/p&gt;
&lt;p&gt;Similarly, if you use metadata to form relations between one column and another and
the naming of the columns has changed at some point in your workflow, then the metadata
may indicate an incorrect column of data it is referring to. This can be mitigated by
not relying on column naming but rather adding additional metadata to all columns of
interest.&lt;/p&gt;
&lt;h2 id="acknowledgements"&gt;Acknowledgements&lt;a class="headerlink" href="#acknowledgements" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;We would like to thank &lt;a href="https://rerun.io"&gt;Rerun.io&lt;/a&gt; for sponsoring the development of this work. &lt;a href="https://rerun.io"&gt;Rerun.io&lt;/a&gt;
is building a data visualization system for Physical AI and uses metadata to specify 
context about columns in Arrow record batches.&lt;/p&gt;
&lt;h2 id="conclusion"&gt;Conclusion&lt;a class="headerlink" href="#conclusion" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;The enhanced metadata handling in [DataFusion 48.0.0] is a significant step
forward in the ability to handle more interesting types of data. Users can
validate the input data matches the intent of the data to be processed, enable
complex operations on binary data because we understand the encoding used, and 
use metadata to create new and interesting user defined data types.
We can't wait to see what you build with it!&lt;/p&gt;
&lt;h2 id="get-involved"&gt;Get Involved&lt;a class="headerlink" href="#get-involved" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;The DataFusion team is an active and engaging community and we would love to have you join
us and help the project.&lt;/p&gt;
&lt;p&gt;Here are some ways to get involved:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;Learn more by visiting the &lt;a href="https://datafusion.apache.org/index.html"&gt;DataFusion&lt;/a&gt; project page.&lt;/li&gt;
&lt;li&gt;Try out the project and provide feedback, file issues, and contribute code.&lt;/li&gt;
&lt;li&gt;Work on a &lt;a href="https://github.com/apache/datafusion/issues?q=is%3Aissue+is%3Aopen+label%3A%22good+first+issue%22"&gt;good first issue&lt;/a&gt;.&lt;/li&gt;
&lt;li&gt;Reach out to us via the &lt;a href="https://datafusion.apache.org/contributor-guide/communication.html"&gt;communication doc&lt;/a&gt;.&lt;/li&gt;
&lt;/ul&gt;</content><category term="blog"/></entry><entry><title>Apache DataFusion Comet 0.10.0 Release</title><link href="https://datafusion.apache.org/blog/2025/09/16/datafusion-comet-0.10.0" rel="alternate"/><published>2025-09-16T00:00:00+00:00</published><updated>2025-09-16T00:00:00+00:00</updated><author><name>pmc</name></author><id>tag:datafusion.apache.org,2025-09-16:/blog/2025/09/16/datafusion-comet-0.10.0</id><summary type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;p&gt;The Apache DataFusion PMC is pleased to announce version 0.10.0 of the &lt;a href="https://datafusion.apache.org/comet/"&gt;Comet&lt;/a&gt; subproject.&lt;/p&gt;
&lt;p&gt;Comet is an accelerator for Apache Spark that translates Spark physical plans to DataFusion physical plans for
improved performance and efficiency without requiring any code changes.&lt;/p&gt;
&lt;p&gt;This release covers approximately ten weeks of development …&lt;/p&gt;</summary><content type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;p&gt;The Apache DataFusion PMC is pleased to announce version 0.10.0 of the &lt;a href="https://datafusion.apache.org/comet/"&gt;Comet&lt;/a&gt; subproject.&lt;/p&gt;
&lt;p&gt;Comet is an accelerator for Apache Spark that translates Spark physical plans to DataFusion physical plans for
improved performance and efficiency without requiring any code changes.&lt;/p&gt;
&lt;p&gt;This release covers approximately ten weeks of development work and is the result of merging 183 PRs from 26
contributors. See the &lt;a href="https://github.com/apache/datafusion-comet/blob/main/dev/changelog/0.10.0.md"&gt;change log&lt;/a&gt; for more information.&lt;/p&gt;
&lt;h2 id="release-highlights"&gt;Release Highlights&lt;a class="headerlink" href="#release-highlights" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;h3 id="improved-support-for-apache-iceberg"&gt;Improved Support for Apache Iceberg&lt;a class="headerlink" href="#improved-support-for-apache-iceberg" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;It is now possible to use Comet with Apache Iceberg 1.8.1 to accelerate reads of Iceberg Parquet tables. Please refer to Comet's &lt;a href="https://datafusion.apache.org/comet/user-guide/latest/iceberg.html"&gt;Iceberg Guide&lt;/a&gt; for information on building Iceberg with Comet.&lt;/p&gt;
&lt;h3 id="improved-spark-400-support"&gt;Improved Spark 4.0.0 Support&lt;a class="headerlink" href="#improved-spark-400-support" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;Comet no longer falls back to Spark for all queries when ANSI mode is enabled (which is the default in Spark 4.0.0). 
Instead, Comet will now only fall back to Spark for arithmetic and aggregates expressions that support ANSI mode.&lt;/p&gt;
&lt;p&gt;Setting &lt;code&gt;spark.comet.ansi.ignore=true&lt;/code&gt; will override this behavior and force these expressions to continue to be 
accelerated by Comet. Full support for ANSI mode will be available in a future release.&lt;/p&gt;
&lt;p&gt;Comet will now use the &lt;code&gt;native_iceberg_compat&lt;/code&gt; scan for Spark 4.0.0 in most cases, which supports reading complex types.&lt;/p&gt;
&lt;h3 id="new-functionality"&gt;New Functionality&lt;a class="headerlink" href="#new-functionality" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;The following SQL functions are now supported:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;code&gt;array_min&lt;/code&gt;&lt;/li&gt;
&lt;li&gt;&lt;code&gt;map_entries&lt;/code&gt;&lt;/li&gt;
&lt;li&gt;&lt;code&gt;map_from_array&lt;/code&gt;&lt;/li&gt;
&lt;li&gt;&lt;code&gt;randn&lt;/code&gt;&lt;/li&gt;
&lt;li&gt;&lt;code&gt;from_unixtime&lt;/code&gt;&lt;/li&gt;
&lt;li&gt;&lt;code&gt;monotonically_increasing_id&lt;/code&gt;&lt;/li&gt;
&lt;li&gt;&lt;code&gt;spark_partition_id&lt;/code&gt;&lt;/li&gt;
&lt;li&gt;&lt;code&gt;try_add&lt;/code&gt;&lt;/li&gt;
&lt;li&gt;&lt;code&gt;try_divide&lt;/code&gt;&lt;/li&gt;
&lt;li&gt;&lt;code&gt;try_mod&lt;/code&gt;&lt;/li&gt;
&lt;li&gt;&lt;code&gt;try_multiply&lt;/code&gt;&lt;/li&gt;
&lt;li&gt;&lt;code&gt;try_subtract&lt;/code&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;Other new features include:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;Support for array literals&lt;/li&gt;
&lt;li&gt;Support for limit with offset&lt;/li&gt;
&lt;/ul&gt;
&lt;h3 id="ux-improvements"&gt;UX Improvements&lt;a class="headerlink" href="#ux-improvements" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;ul&gt;
&lt;li&gt;Improved reporting of reasons why Comet cannot accelerate some operators and expressions&lt;/li&gt;
&lt;li&gt;New &lt;code&gt;spark.comet.logFallbackReasons.enabled&lt;/code&gt; configuration setting for logging all fallback reasons&lt;/li&gt;
&lt;li&gt;CometScan nodes in the physical plan now show which scan implementation is being used (&lt;code&gt;native_comet&lt;/code&gt;, 
&lt;code&gt;native_datafusion&lt;/code&gt;, or &lt;code&gt;native_iceberg_compat&lt;/code&gt;)&lt;/li&gt;
&lt;/ul&gt;
&lt;h3 id="bug-fixes"&gt;Bug Fixes&lt;a class="headerlink" href="#bug-fixes" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;ul&gt;
&lt;li&gt;Improved memory safety for FFI transfers&lt;/li&gt;
&lt;li&gt;Fixed a double-free issue in the shuffle unified memory pool&lt;/li&gt;
&lt;li&gt;Fixed an FFI issue with non-zero offsets&lt;/li&gt;
&lt;li&gt;Fixed an issue with buffered reads from HDFS &lt;/li&gt;
&lt;/ul&gt;
&lt;h3 id="benchmarking"&gt;Benchmarking&lt;a class="headerlink" href="#benchmarking" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;Benchmarking scripts for benchmarks based on TPC-H and TPS-DS are now available in the repository under &lt;code&gt;dev/benchmarks&lt;/code&gt;.&lt;/p&gt;
&lt;h3 id="documentation-updates"&gt;Documentation Updates&lt;a class="headerlink" href="#documentation-updates" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;ul&gt;
&lt;li&gt;The documentation for supported &lt;a href="https://datafusion.apache.org/comet/user-guide/latest/operators.html"&gt;operators&lt;/a&gt; and &lt;a href="https://datafusion.apache.org/comet/user-guide/latest/expressions.html"&gt;expressions&lt;/a&gt; is now more complete, and Spark-compatibility status 
  per operator/expression is now documented.&lt;/li&gt;
&lt;li&gt;The documentation now contains a &lt;a href="https://datafusion.apache.org/comet/contributor-guide/roadmap.html"&gt;roadmap&lt;/a&gt; section.&lt;/li&gt;
&lt;li&gt;New &lt;a href="https://datafusion.apache.org/comet/gluten_comparison.html"&gt;guide&lt;/a&gt; comparing Comet with Apache Gluten (incubating) + Velox&lt;/li&gt;
&lt;li&gt;User guides are now available for multiple Comet versions&lt;/li&gt;
&lt;/ul&gt;
&lt;h3 id="spark-compatibility"&gt;Spark Compatibility&lt;a class="headerlink" href="#spark-compatibility" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;ul&gt;
&lt;li&gt;Spark 3.4.3 with JDK 11 &amp;amp; 17, Scala 2.12 &amp;amp; 2.13&lt;/li&gt;
&lt;li&gt;Spark 3.5.4 through 3.5.6 with JDK 11 &amp;amp; 17, Scala 2.12 &amp;amp; 2.13&lt;/li&gt;
&lt;li&gt;Experimental support for Spark 4.0.0 with JDK 17, Scala 2.13&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;We are looking for help from the community to fully support Spark 4.0.0. See &lt;a href="https://github.com/apache/datafusion-comet/issues/1637"&gt;EPIC: Support 4.0.0&lt;/a&gt; for more information.&lt;/p&gt;
&lt;h2 id="getting-involved"&gt;Getting Involved&lt;a class="headerlink" href="#getting-involved" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;The Comet project welcomes new contributors. We use the same &lt;a href="https://datafusion.apache.org/contributor-guide/communication.html#slack-and-discord"&gt;Slack and Discord&lt;/a&gt; channels as the main DataFusion
project and have a weekly &lt;a href="https://docs.google.com/document/d/1NBpkIAuU7O9h8Br5CbFksDhX-L9TyO9wmGLPMe0Plc8/edit?usp=sharing"&gt;DataFusion video call&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;The easiest way to get involved is to test Comet with your current Spark jobs and file issues for any bugs or
performance regressions that you find. See the &lt;a href="https://datafusion.apache.org/comet/user-guide/installation.html"&gt;Getting Started&lt;/a&gt; guide for instructions on downloading and installing
Comet.&lt;/p&gt;
&lt;p&gt;There are also many &lt;a href="https://github.com/apache/datafusion-comet/contribute"&gt;good first issues&lt;/a&gt; waiting for contributions.&lt;/p&gt;</content><category term="blog"/></entry><entry><title>Dynamic Filters: Passing Information Between Operators During Execution for 25x Faster Queries</title><link href="https://datafusion.apache.org/blog/2025/09/10/dynamic-filters" rel="alternate"/><published>2025-09-10T00:00:00+00:00</published><updated>2025-09-10T00:00:00+00:00</updated><author><name>Adrian Garcia Badaracco (Pydantic), Andrew Lamb (InfluxData)</name></author><id>tag:datafusion.apache.org,2025-09-10:/blog/2025/09/10/dynamic-filters</id><summary type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;!-- 
diagrams source: https://docs.google.com/presentation/d/1FFYy27ydZdeFZWWuMjZGnYKUx9QNJfzuVLAH8AE5wlc/edit?slide=id.g364a74cba3d_0_92#slide=id.g364a74cba3d_0_92
Intended Audience: Query engine / data systems developers who want to learn about topk optimization
Goal: Introduce TopK and dynamic filters as general optimization techniques for query engines, and how they were used to improve performance in DataFusion.
--&gt;
&lt;p&gt;This blog post introduces the query engine optimization techniques called TopK
and dynamic filters. We describe the motivating use case, how these
optimizations work, and how we implemented them with the &lt;a href="https://datafusion.apache.org/"&gt;Apache DataFusion&lt;/a&gt;
community to improve performance by an order of magnitude for some query
patterns.&lt;/p&gt;
&lt;h2 id="motivation-and-results"&gt;Motivation and Results&lt;a class="headerlink" href="#motivation-and-results" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;The …&lt;/p&gt;</summary><content type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;!-- 
diagrams source: https://docs.google.com/presentation/d/1FFYy27ydZdeFZWWuMjZGnYKUx9QNJfzuVLAH8AE5wlc/edit?slide=id.g364a74cba3d_0_92#slide=id.g364a74cba3d_0_92
Intended Audience: Query engine / data systems developers who want to learn about topk optimization
Goal: Introduce TopK and dynamic filters as general optimization techniques for query engines, and how they were used to improve performance in DataFusion.
--&gt;
&lt;p&gt;This blog post introduces the query engine optimization techniques called TopK
and dynamic filters. We describe the motivating use case, how these
optimizations work, and how we implemented them with the &lt;a href="https://datafusion.apache.org/"&gt;Apache DataFusion&lt;/a&gt;
community to improve performance by an order of magnitude for some query
patterns.&lt;/p&gt;
&lt;h2 id="motivation-and-results"&gt;Motivation and Results&lt;a class="headerlink" href="#motivation-and-results" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;The main commercial product at &lt;a href="https://pydantic.dev"&gt;Pydantic&lt;/a&gt;, &lt;a href="https://pydantic.dev/logfire"&gt;Logfire&lt;/a&gt;, is an observability
platform built on DataFusion. One of the most common workflows / queries is
"show me the last K traces" which translates to a query similar to:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-sql"&gt;SELECT * FROM records ORDER BY start_timestamp DESC LIMIT 1000;
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;We noticed this was &lt;em&gt;pretty slow&lt;/em&gt;, even though DataFusion has long had the
classic &lt;code&gt;TopK&lt;/code&gt; optimization (described below). After implementing the dynamic
filter techniques described in this blog, we saw performance improve &lt;em&gt;by over 10x&lt;/em&gt;
for this query pattern, and are applying the optimization to other queries and
operators as well.&lt;/p&gt;
&lt;p&gt;Let's look at some preliminary numbers, using &lt;a href="https://github.com/apache/datafusion/blob/main/benchmarks/queries/clickbench/queries/q23.sql"&gt;ClickBench&lt;/a&gt;, which has 
the same pattern as our motivating example:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-sql"&gt;SELECT * FROM hits WHERE "URL" LIKE '%google%' ORDER BY "EventTime" LIMIT 10;
&lt;/code&gt;&lt;/pre&gt;
&lt;div class="text-center"&gt;
&lt;img alt="Q23 Performance Improvement with Dynamic Filters and Late Materialization" class="img-fluid" src="/blog/images/dynamic-filters/execution-time.svg" width="80%"/&gt;
&lt;/div&gt;
&lt;p&gt;&lt;strong&gt;Figure 1&lt;/strong&gt;: Execution times for ClickBench Q23 with and without dynamic
filters (DF)&lt;sup id="fn1"&gt;&lt;a href="#footnote1"&gt;1&lt;/a&gt;&lt;/sup&gt;, and late materialization
(LM)&lt;sup id="fn2"&gt;&lt;a href="#footnote2"&gt;2&lt;/a&gt;&lt;/sup&gt; for different partitions / core usage.
Dynamic filters alone (yellow) and late materialization alone (red) show a large
improvement over the baseline (blue). When both optimizations are enabled (green)
performance improves by up to 22x. See the appendix for more measurement details.&lt;/p&gt;
&lt;h2 id="background-topk-and-dynamic-filters"&gt;Background: TopK and Dynamic Filters&lt;a class="headerlink" href="#background-topk-and-dynamic-filters" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;To explain how dynamic filters improve query performance, we first need to
explain the so-called "TopK" optimization. To do so, we will use a simplified
version of ClickBench Q23:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-sql"&gt;SELECT * 
FROM hits 
ORDER BY "EventTime"
LIMIT 10
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;A straightforward, though slow, plan to answer this query is shown in Figure 2.&lt;/p&gt;
&lt;div class="text-center"&gt;
&lt;img alt="Naive Query Plan" class="img-fluid" src="/blog/images/dynamic-filters/query-plan-naive.png" width="80%"/&gt;
&lt;/div&gt;
&lt;p&gt;&lt;strong&gt;Figure 2&lt;/strong&gt;: Simple Query Plan for ClickBench Q23. Data flows in plans from the
scan at the bottom to the limit at the top. This plan reads all 100M rows of the
&lt;code&gt;hits&lt;/code&gt; table, sorts them by &lt;code&gt;EventTime&lt;/code&gt;, and then discards everything except the top 10 rows.&lt;/p&gt;
&lt;p&gt;This naive plan requires substantial effort as all columns from all rows are
decoded and sorted, even though only 10 are returned. &lt;/p&gt;
&lt;p&gt;High-performance query engines typically avoid the expensive full sort with a
specialized operator that tracks the current top rows using a &lt;a href="https://en.wikipedia.org/wiki/Heap_(data_structure)"&gt;heap&lt;/a&gt;, rather
than sorting all the data. For example, this operator
is called &lt;a href="https://docs.rs/datafusion/latest/datafusion/physical_plan/struct.TopK.html"&gt;TopK in DataFusion&lt;/a&gt;, &lt;a href="https://docs.snowflake.com/en/user-guide/ui-snowsight-activity"&gt;SortWithLimit in Snowflake&lt;/a&gt;, and &lt;a href="https://duckdb.org/2024/10/25/topn.html#introduction-to-top-n"&gt;topn in
DuckDB&lt;/a&gt;. The plan for Q23 using this specialized operator is shown in Figure 3.&lt;/p&gt;
&lt;div class="text-center"&gt;
&lt;img alt="TopK Query Plan" class="img-fluid" src="/blog/images/dynamic-filters/query-plan-topk.png" width="80%"/&gt;
&lt;/div&gt;
&lt;p&gt;&lt;strong&gt;Figure 3&lt;/strong&gt;: Query plan for Q23 in DataFusion using the TopK operator. This
plan still reads all 100M rows of the &lt;code&gt;hits&lt;/code&gt; table, but instead of first sorting
them all by &lt;code&gt;EventTime&lt;/code&gt;, the TopK operator keeps track of the current top 10
rows using a min/max heap. Credit to &lt;a href="https://visualgo.net/en"&gt;Visualgo&lt;/a&gt; for the
heap icon&lt;/p&gt;
&lt;p&gt;Figure 3 is better, but it still reads and decodes all 100M rows of the &lt;code&gt;hits&lt;/code&gt; table,
which is often unnecessary once we have found the top 10 rows. For example,
while running the query, if the current top 10 rows all have &lt;code&gt;EventTime&lt;/code&gt; in
2025, then any subsequent rows with &lt;code&gt;EventTime&lt;/code&gt; in 2024 or earlier can be
skipped entirely without reading or decoding them. This technique is especially
effective at skipping entire files or row groups if the top 10 values are in the
first few files read, which is very common when the
data insert order is approximately the same as the timestamp order.&lt;/p&gt;
&lt;p&gt;Leveraging this insight is the key idea behind dynamic filters, which introduce
a runtime mechanism for the TopK operator to provide the current top values to
the scan operator, allowing it to skip unnecessary rows, entire files, or portions
of files. The plan for Q23 with dynamic filters is shown in Figure 4.&lt;/p&gt;
&lt;div class="text-center"&gt;
&lt;img alt="TopK Query Plan with Dynamic Filters" class="img-fluid" src="/blog/images/dynamic-filters/query-plan-topk-dynamic-filters.png" width="100%"/&gt;
&lt;/div&gt;
&lt;p&gt;&lt;strong&gt;Figure 4&lt;/strong&gt;: Query plan for Q23 in DataFusion with specialized TopK operator
and dynamic filters. The TopK operator provides the minimum &lt;code&gt;EventTime&lt;/code&gt; of the
current top 10 rows to the scan operator, allowing it to skip rows with
&lt;code&gt;EventTime&lt;/code&gt; later than that value. The scan operator uses this dynamic filter
to skip unnecessary files and rows, reducing the amount of data that needs to
be read and processed.&lt;/p&gt;
&lt;h2 id="worked-example"&gt;Worked Example&lt;a class="headerlink" href="#worked-example" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;To make dynamic filters more concrete, here is a fully worked example. Imagine
we have a table &lt;code&gt;records&lt;/code&gt; with a column &lt;code&gt;start_timestamp&lt;/code&gt; and we are running the
motivating query:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-sql"&gt;SELECT * 
FROM records 
ORDER BY start_timestamp 
DESC LIMIT 3;
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;In this example, at some point during execution, the heap in the &lt;code&gt;TopK&lt;/code&gt; operator
will contain the actual 3 most recent values, which might be:&lt;/p&gt;
&lt;table class="table"&gt;
&lt;thead&gt;
&lt;tr&gt;
&lt;th&gt;start_timestamp&lt;/th&gt;
&lt;/tr&gt;
&lt;/thead&gt;
&lt;tbody&gt;
&lt;tr&gt;
&lt;td&gt;2025-08-16T20:35:15.00Z&lt;/td&gt;
&lt;/tr&gt;
&lt;tr&gt;
&lt;td&gt;2025-08-16T20:35:14.00Z&lt;/td&gt;
&lt;/tr&gt;
&lt;tr&gt;
&lt;td&gt;2025-08-16T20:35:13.00Z&lt;/td&gt;
&lt;/tr&gt;
&lt;/tbody&gt;
&lt;/table&gt;
&lt;p&gt;Since &lt;code&gt;2025-08-16T20:35:13.00Z&lt;/code&gt; is the smallest of these values, we know that
any subsequent rows with &lt;code&gt;start_timestamp&lt;/code&gt; less than or equal to this value
cannot possibly be in the top 3, and can be skipped entirely.
This knowledge is encoded in a filter of the form &lt;code&gt;start_timestamp &amp;gt;
'2025-08-16T20:35:13.00Z'&lt;/code&gt;. If we knew the correct timestamp value before
starting the plan, we could simply write:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-sql"&gt;SELECT *
FROM records
WHERE start_timestamp &amp;gt; '2025-08-16T20:35:13.00Z'  -- Filter to skip rows
ORDER BY start_timestamp DESC
LIMIT 3;
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;And DataFusion's existing hierarchical pruning (described in &lt;a href="https://datafusion.apache.org/blog/2025/08/15/external-parquet-indexes/"&gt;this blog&lt;/a&gt;) would
skip reading unnecessary files and row groups, and only decode
the necessary rows.&lt;/p&gt;
&lt;p&gt;However, obviously when we start running the query we don't have the value
&lt;code&gt;'2025-08-16T20:35:13.00Z'&lt;/code&gt;, so what DataFusion now does is put a dynamic filter
into the plan instead, which you can think of as a function call like
&lt;code&gt;dynamic_filter()&lt;/code&gt;, something like this:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-sql"&gt;SELECT *
FROM records
WHERE dynamic_filter() -- Updated during execution as we know more
ORDER BY start_timestamp DESC
LIMIT 3;
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;In this case, &lt;code&gt;dynamic_filter()&lt;/code&gt; initially has the value &lt;code&gt;true&lt;/code&gt; (passes all
rows) but will be progressively updated by the TopK operator as the query
progresses to filter more and more rows. Note that while we are using SQL for
illustrative purposes in this example, these optimizations are done at the
physical plan (&lt;a href="https://docs.rs/datafusion/latest/datafusion/physical_plan/trait.ExecutionPlan.html"&gt;ExecutionPlan&lt;/a&gt;) level — and they apply equally to SQL, DataFrame
APIs, and custom query languages built with DataFusion.&lt;/p&gt;
&lt;h2 id="topk-dynamic-filters"&gt;TopK + Dynamic Filters&lt;a class="headerlink" href="#topk-dynamic-filters" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;As mentioned above, DataFusion has a specialized sort operator named &lt;a href="https://docs.rs/datafusion/latest/datafusion/physical_plan/struct.TopK.html"&gt;TopK&lt;/a&gt; that
only keeps &lt;code&gt;K&lt;/code&gt; rows in memory. For a &lt;code&gt;DESC&lt;/code&gt; sort order, each new input batch is
compared against the current &lt;code&gt;K&lt;/code&gt; largest values, and then the current &lt;code&gt;K&lt;/code&gt; rows
possibly get replaced with any new input rows that are larger. The &lt;a href="https://github.com/apache/datafusion/blob/b4a8b5ae54d939353b7cbd5ab8aee7d3bedecb66/datafusion/physical-plan/src/topk/mod.rs"&gt;code is
here&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;Prior to dynamic filters, DataFusion had no early termination: it would read the
&lt;em&gt;entire&lt;/em&gt; &lt;code&gt;records&lt;/code&gt; table even if it already had the top &lt;code&gt;K&lt;/code&gt; rows because it
still had to check that there were no rows that had larger &lt;code&gt;start_timestamp&lt;/code&gt;.
You can see how this is a problem if you have 2 years' worth of time-series data
and the largest &lt;code&gt;1000&lt;/code&gt; values of &lt;code&gt;start_timestamp&lt;/code&gt; are likely within the first
few files read. Even once the &lt;code&gt;TopK&lt;/code&gt; operator has seen 1000 timestamps (e.g. on
August 16th, 2025), DataFusion would still read all remaining files (e.g. even
those that contain data only from 2024) just to make sure.&lt;/p&gt;
&lt;p&gt;InfluxData &lt;a href="https://www.influxdata.com/blog/making-recent-value-queries-hundreds-times-faster/"&gt;optimized a similar query pattern in InfluxDB IOx&lt;/a&gt; using another
operator called &lt;code&gt;ProgressiveEvalExec&lt;/code&gt;. However, &lt;code&gt;ProgressiveEvalExec&lt;/code&gt; requires that the data
is already sorted and a careful analysis of ordering to prove that it can be
used and still produce correct results. That is not the case for Logfire data (and many other datasets):
data tends to be &lt;em&gt;roughly&lt;/em&gt; sorted (e.g. if you append to files as you receive
it) but that does not guarantee that it is fully sorted, either within or between
files. &lt;/p&gt;
&lt;p&gt;We &lt;a href="https://github.com/apache/datafusion/issues/15037"&gt;discussed possible solutions&lt;/a&gt; with the community, and ultimately decided to
implement generic "dynamic filters", which are general enough to be used in
joins as well (see next section). Our implementation appears very similar to
recently announced optimizations in closed-source, commercial systems such as
&lt;a href="https://program.berlinbuzzwords.de/bbuzz24/talk/3DTQJB/"&gt;Accelerating TopK Queries in Snowflake&lt;/a&gt;, or &lt;a href="https://www.alibabacloud.com/blog/about-database-kernel-%7C-learn-about-polardb-imci-optimization-techniques_600274"&gt;self-sharpening runtime filters in
Alibaba Cloud's PolarDB&lt;/a&gt;, and we are excited that we can offer similar features
in an open source query engine like DataFusion.&lt;/p&gt;
&lt;p&gt;At the query plan level, Q23 looks like this before it is executed:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-text"&gt;┌───────────────────────────┐
│       SortExec(TopK)      │
│    --------------------   │
│ EventTime@4 ASC NULLS LAST│
│                           │
│         limit: 10         │
└─────────────┬─────────────┘
┌─────────────┴─────────────┐
│       DataSourceExec      │
│    --------------------   │
│         files: 100        │
│      format: parquet      │
│                           │
│         predicate:        │
│ CAST(URL AS Utf8View) LIKE│
│      %google% AND true    │
└───────────────────────────┘
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;&lt;strong&gt;Figure 5&lt;/strong&gt;: Physical plan for ClickBench Q23 prior to execution. The dynamic
filter is shown as &lt;code&gt;true&lt;/code&gt; in the &lt;code&gt;predicate&lt;/code&gt; field of the &lt;code&gt;DataSourceExec&lt;/code&gt;
operator.&lt;/p&gt;
&lt;p&gt;The dynamic filter is updated by the &lt;code&gt;SortExec(TopK)&lt;/code&gt; operator during execution
as shown in Figure 6.&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-text"&gt;┌───────────────────────────┐
│       SortExec(TopK)      │
│    --------------------   │
│ EventTime@4 ASC NULLS LAST│
│                           │
│         limit: 10         │
└─────────────┬─────────────┘
┌─────────────┴─────────────┐
│       DataSourceExec      │
│    --------------------   │
│         files: 100        │
│      format: parquet      │
│                           │
│         predicate:        │
│ CAST(URL AS Utf8View) LIKE│
│      %google% AND         │
│ EventTime &amp;lt; 1372713773.0  │
└───────────────────────────┘
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;&lt;strong&gt;Figure 6&lt;/strong&gt;: Physical plan for ClickBench Q23 after execution. The dynamic filter has been
updated to &lt;code&gt;EventTime &amp;lt; 1372713773.0&lt;/code&gt;, which allows the &lt;code&gt;DataSourceExec&lt;/code&gt; operator to skip
files and rows that do not match the filter.&lt;/p&gt;
&lt;h2 id="hash-join-dynamic-filters"&gt;Hash Join + Dynamic Filters&lt;a class="headerlink" href="#hash-join-dynamic-filters" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;We spent significant effort to make dynamic filters a general-purpose
optimization (see the Extensibility section below for more details). Instead of
a one-off optimization for TopK queries, we created a general mechanism for
passing information between operators during execution that can be used in multiple contexts. 
We have already used the dynamic filter infrastructure to
improve hash joins by implementing a technique called &lt;a href="https://15721.courses.cs.cmu.edu/spring2020/papers/13-execution/shrinivas-icde2013.pdf"&gt;sideways information
passing&lt;/a&gt;, which is similar to &lt;a href="https://issues.apache.org/jira/browse/SPARK-32268"&gt;Bloom filter joins&lt;/a&gt; in Apache Spark. See 
&lt;a href="https://github.com/apache/datafusion/issues/7955"&gt;issue #7955&lt;/a&gt; for more details.&lt;/p&gt;
&lt;p&gt;In a Hash Join, the query engine picks one input of the join to be the "build"
input and the other input to be the "probe" side.&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;
&lt;p&gt;First, the &lt;strong&gt;build side&lt;/strong&gt; is loaded into memory, and turned into a hash table.&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;
&lt;p&gt;Then, the &lt;strong&gt;probe side&lt;/strong&gt; is scanned, and matching rows are found by looking 
  in the hash table. Non-matching rows are discarded and thus joins often act as
  filters.&lt;/p&gt;
&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;Many hash joins act as selective filters for rows from the probe side (when only
a small number of rows are matched), so it is natural to use the same dynamic
filter technique. DataFusion 50.0.0 pushes down knowledge of what keys exist on
the build side into the scan of the probe side with a dynamic filter based on
min/max join key values. For example, if the build side only has keys in the
range &lt;code&gt;[100, 200]&lt;/code&gt;, then DataFusion will filter out all probe rows with keys
outside that range during the scan.&lt;/p&gt;
&lt;p&gt;This simple approach is fast to evaluate and the filter improves performance
significantly when combined with statistics pruning, late materialization, and
other optimizations as shown in Figure 7.&lt;/p&gt;
&lt;div class="text-center"&gt;
&lt;img alt="Join Performance Improvements with Dynamic Filters" class="img-fluid" src="/blog/images/dynamic-filters/join-performance.svg" width="80%"/&gt;
&lt;/div&gt;
&lt;p&gt;&lt;strong&gt;Figure 7&lt;/strong&gt;: Join performance with and without dynamic filters. In DataFusion
49.0.2 the join takes 2.5s, even with late materialization (LM) enabled. In
DataFusion 50.0.0 with dynamic filters enabled (the default), the join takes
only 0.7s, a 5x improvement. With both dynamic filters and late materialization,
DataFusion 50.0.0 takes 0.1s, a 25x improvement. See this &lt;a href="https://github.com/apache/datafusion-site/pull/103#issuecomment-3262612288"&gt;discussion&lt;/a&gt; for more
details.&lt;/p&gt;
&lt;p&gt;You can see dynamic join filters in action with the following example. &lt;/p&gt;
&lt;pre&gt;&lt;code class="language-sql"&gt;-- create two tables: small_table with 1K rows and large_table with 100K rows
COPY (SELECT i as k, i as v FROM generate_series(1, 1000) t(i)) TO 'small_table.parquet';
CREATE EXTERNAL TABLE small_table STORED AS PARQUET LOCATION 'small_table.parquet';
COPY (SELECT i as k FROM generate_series(1, 100000) t(i)) TO 'large_table.parquet';
CREATE EXTERNAL TABLE large_table STORED AS PARQUET LOCATION 'large_table.parquet';

-- Join the two tables, with a filter on small_table
EXPLAIN 
SELECT * 
FROM small_table JOIN large_table ON small_table.k = large_table.k 
WHERE small_table.v &amp;gt;= 50;
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;Note there are no filters on the &lt;code&gt;large_table&lt;/code&gt; in the initial query, but a
dynamic filter is introduced by DataFusion on the &lt;code&gt;large_table&lt;/code&gt; scan. As the
&lt;code&gt;small_table&lt;/code&gt; is read and the hash table is built, the dynamic filter is updated 
to become more and more effective. Before execution, the plan
looks like this:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-text"&gt;+---------------+------------------------------------------------------------+
| plan_type     | plan                                                       |
+---------------+------------------------------------------------------------+
| physical_plan | ┌───────────────────────────┐                              |
|               | │    CoalesceBatchesExec    │                              |
|               | │    --------------------   │                              |
|               | │     target_batch_size:    │                              |
|               | │            8192           │                              |
|               | └─────────────┬─────────────┘                              |
|               | ┌─────────────┴─────────────┐                              |
|               | │        HashJoinExec       │                              |
|               | │    --------------------   ├──────────────┐               |
|               | │        on: (k = k)        │              │               |
|               | └─────────────┬─────────────┘              │               |
|               | ┌─────────────┴─────────────┐┌─────────────┴─────────────┐ |
|               | │   CoalescePartitionsExec  ││      RepartitionExec      │ |
|               | │                           ││    --------------------   │ |
|               | │                           ││ partition_count(in-&amp;gt;out): │ |
|               | │                           ││          1 -&amp;gt; 16          │ |
|               | │                           ││                           │ |
|               | │                           ││    partitioning_scheme:   │ |
|               | │                           ││    RoundRobinBatch(16)    │ |
|               | └─────────────┬─────────────┘└─────────────┬─────────────┘ |
|               | ┌─────────────┴─────────────┐┌─────────────┴─────────────┐ |
|               | │    CoalesceBatchesExec    ││       DataSourceExec      │ |
|               | │    --------------------   ││    --------------------   │ |
|               | │     target_batch_size:    ││          files: 1         │ |
|               | │            8192           ││      format: parquet      │ |
|               | │                           ││      predicate: true      │ |
|               | └─────────────┬─────────────┘└───────────────────────────┘ |
|               | ┌─────────────┴─────────────┐                              |
|               | │         FilterExec        │                              |
|               | │    --------------------   │                              |
|               | │     predicate: v &amp;gt;= 50    │                              |
|               | └─────────────┬─────────────┘                              |
|               | ┌─────────────┴─────────────┐                              |
|               | │      RepartitionExec      │                              |
|               | │    --------------------   │                              |
|               | │ partition_count(in-&amp;gt;out): │                              |
|               | │          1 -&amp;gt; 16          │                              |
|               | │                           │                              |
|               | │    partitioning_scheme:   │                              |
|               | │    RoundRobinBatch(16)    │                              |
|               | └─────────────┬─────────────┘                              |
|               | ┌─────────────┴─────────────┐                              |
|               | │       DataSourceExec      │                              |
|               | │    --------------------   │                              |
|               | │          files: 1         │                              |
|               | │      format: parquet      │                              |
|               | │     predicate: v &amp;gt;= 50    │                              |
|               | └───────────────────────────┘                              |
|               |                                                            |
+---------------+------------------------------------------------------------+
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;&lt;strong&gt;Figure 8&lt;/strong&gt;: Physical plan for the join query before execution. The left input
to the join is the build side, which scans &lt;code&gt;small_table&lt;/code&gt; and applies the filter
&lt;code&gt;v &amp;gt;= 50&lt;/code&gt;. The right input to the join is the probe side, which scans &lt;code&gt;large_table&lt;/code&gt;
and has the dynamic filter (shown here as the placeholder &lt;code&gt;true&lt;/code&gt;).&lt;/p&gt;
&lt;h2 id="dynamic-filter-extensibility-custom-executionplan-operators"&gt;Dynamic Filter Extensibility: Custom &lt;code&gt;ExecutionPlan&lt;/code&gt; Operators&lt;a class="headerlink" href="#dynamic-filter-extensibility-custom-executionplan-operators" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;We went to great efforts to ensure that dynamic filters are not a hardcoded
black box that only works for internal operators. This is important not only for
software maintainability, but also because DataFusion is used in many different
contexts including advanced custom operators specialized for specific use cases.&lt;/p&gt;
&lt;p&gt;Dynamic filter creation and pushdown are implemented as methods on the
&lt;a href="https://docs.rs/datafusion/latest/datafusion/physical_plan/trait.ExecutionPlan.html"&gt;ExecutionPlan trait&lt;/a&gt;. Thus, it is possible for user-defined, custom
&lt;code&gt;ExecutionPlan&lt;/code&gt;s to work with dynamic filters with little to no modification. We
also provide an extensive library of helper structs and functions, so it often
takes only 1-2 lines of code to implement filter pushdown support or a source of
dynamic filters for custom operators.&lt;/p&gt;
&lt;p&gt;This approach has already paid off, and we know of community members who have
implemented support for dynamic filter pushdown using preview releases of
DataFusion 50.0.0.&lt;/p&gt;
&lt;!-- AAL Who else has done this? --&gt;
&lt;h3 id="design-of-scan-operator-integration"&gt;Design of Scan Operator Integration&lt;a class="headerlink" href="#design-of-scan-operator-integration" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;A core design decision is to represent dynamic filters as &lt;code&gt;Arc&amp;lt;dyn
PhysicalExpr&amp;gt;&lt;/code&gt;,  the same interface as all other expressions in DataFusion. This
means that &lt;code&gt;DataSourceExec&lt;/code&gt; and other scan operators do not require special
logic to handle dynamic filters, and existing filter pushdown logic works
without modification. We did add some new functionality to &lt;code&gt;PhysicalExpr&lt;/code&gt; to
make working with dynamic filters more performant for specific use cases:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;
&lt;p&gt;&lt;code&gt;PhysicalExpr::generation() -&amp;gt; u64&lt;/code&gt;: to track if a tree of filters has
  changed (e.g. it has a dynamic filter that has been updated). For
  example, if a predicate changes from &lt;code&gt;c1 = 'a' AND DynamicFilter [ c2 &amp;gt; 1]&lt;/code&gt; to &lt;code&gt;c1 = 'a' AND
  DynamicFilter [ c2 &amp;gt; 2]&lt;/code&gt; the generation value will also change so operators know if they
  should re-evaluate the filter against static data like file or row group
  level statistics. This is used in the ListingTable provider to do early termination of reading a file if the
  filter is updated mid scan to skip the entire file, without
  needlessly re-evaluating file level statistics on each batch.&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;
&lt;p&gt;&lt;code&gt;PhysicalExpr::snapshot() -&amp;gt; Arc&amp;lt;dyn PhysicalExpr&amp;gt;&lt;/code&gt;: to create a snapshot
  of the filter at a given point in time. Dynamic filters use this to return the
  current value of their inner static filter. This can be used to serialize the
  filter across the network for distributed engines or pass to systems that
  support specific static filter patterns (e.g. stats pruning rewrites).&lt;/p&gt;
&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;This is all implemented in the &lt;code&gt;DynamicFilterPhysicalExpr&lt;/code&gt; struct.&lt;/p&gt;
&lt;p&gt;Another important design point was handling concurrency and information
flow. In early designs, the scan polled the source operators on every row /
batch, which had significant overhead. The final design is a "push" model where
the scan path has minimal locking and the write path (e.g. the TopK
operator) is responsible for updating the filter. You can think of
&lt;code&gt;DynamicFilterPhysicalExpr&lt;/code&gt; as an &lt;code&gt;Arc&amp;lt;RwLock&amp;lt;Arc&amp;lt;dyn PhysicalExpr&amp;gt;&amp;gt;&amp;gt;&lt;/code&gt;, which
allows the TopK operator to update the filter without blocking the scan
operator.&lt;/p&gt;
&lt;h2 id="future-work"&gt;Future Work&lt;a class="headerlink" href="#future-work" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;Although we've made great progress and DataFusion now has one of the most
advanced open-source dynamic filter / sideways information passing
implementations that we know of, we see many areas of future improvement such as:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;
&lt;p&gt;&lt;a href="https://github.com/apache/datafusion/issues/16973"&gt;Support for more types of joins&lt;/a&gt;: This optimization is only implemented for
  &lt;code&gt;INNER&lt;/code&gt; hash joins so far, but it could be implemented for other join algorithms
  (e.g. nested loop joins) and join types (e.g. &lt;code&gt;LEFT OUTER JOIN&lt;/code&gt;).&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;
&lt;p&gt;&lt;a href="https://github.com/apache/datafusion/issues/17171"&gt;Push down entire hash tables to the scan operator&lt;/a&gt;: Improve the representation
  of the dynamic filter beyond min/max values to improve performance for joins with many
  distinct matching keys that are not naturally ordered or have significant skew.&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;
&lt;p&gt;&lt;a href="https://github.com/apache/datafusion/issues/17348"&gt;Use file level statistics to order files&lt;/a&gt; to match the &lt;code&gt;ORDER BY&lt;/code&gt; clause as
  much as possible. This can help TopK dynamic filters be more effective at
  pruning by skipping more work earlier in the scan.&lt;/p&gt;
&lt;/li&gt;
&lt;/ul&gt;
&lt;h2 id="acknowledgements"&gt;Acknowledgements&lt;a class="headerlink" href="#acknowledgements" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;Thank you to &lt;a href="https://pydantic.dev"&gt;Pydantic&lt;/a&gt; and &lt;a href="https://www.influxdata.com/"&gt;InfluxData&lt;/a&gt; for supporting our work on DataFusion
and open source in general. Thank you to &lt;a href="https://github.com/zhuqi-lucas"&gt;zhuqi-lucas&lt;/a&gt;, &lt;a href="https://github.com/xudong963"&gt;xudong963&lt;/a&gt;,
&lt;a href="https://github.com/Dandandan"&gt;Dandandan&lt;/a&gt;, and &lt;a href="https://github.com/LiaCastaneda"&gt;LiaCastaneda&lt;/a&gt;, for helping with the dynamic join filter
implementation and testing. Thank you to &lt;a href="https://github.com/nuno-faria"&gt;nuno-faria&lt;/a&gt; for providing join performance
results and &lt;a href="https://github.com/djanderson"&gt;djanderson&lt;/a&gt; for their helpful review comments. &lt;/p&gt;
&lt;h2 id="about-the-authors"&gt;About the Authors&lt;a class="headerlink" href="#about-the-authors" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;&lt;a href="https://www.linkedin.com/in/adrian-garcia-badaracco/"&gt;Adrian Garcia Badaracco&lt;/a&gt; is a Founding Engineer at
&lt;a href="https://pydantic.dev/"&gt;Pydantic&lt;/a&gt;, and an &lt;a href="https://datafusion.apache.org/"&gt;Apache
DataFusion&lt;/a&gt; committer.&lt;/p&gt;
&lt;p&gt;&lt;a href="https://www.linkedin.com/in/andrewalamb/"&gt;Andrew Lamb&lt;/a&gt; is a Staff Engineer at
&lt;a href="https://www.influxdata.com/"&gt;InfluxData&lt;/a&gt;, and a member of the &lt;a href="https://datafusion.apache.org/"&gt;Apache
DataFusion&lt;/a&gt; and &lt;a href="https://arrow.apache.org/"&gt;Apache Arrow&lt;/a&gt; PMCs. He has been working on
databases and related systems for more than 20 years.&lt;/p&gt;
&lt;h2 id="about-datafusion"&gt;About DataFusion&lt;a class="headerlink" href="#about-datafusion" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;&lt;a href="https://datafusion.apache.org/"&gt;Apache DataFusion&lt;/a&gt; is an extensible query engine toolkit, written
in Rust, that uses &lt;a href="https://arrow.apache.org/"&gt;Apache Arrow&lt;/a&gt; as its in-memory format. DataFusion and
similar technology are part of the next generation “Deconstructed Database”
architectures, where new systems are built on a foundation of fast, modular
components, rather than as a single tightly integrated system.&lt;/p&gt;
&lt;p&gt;The &lt;a href="https://datafusion.apache.org/contributor-guide/communication.html"&gt;DataFusion community&lt;/a&gt; is always looking for new contributors to help
improve the project. If you are interested in learning more about how query
execution works, help document or improve the DataFusion codebase, or just try
it out, we would love for you to join us.&lt;/p&gt;
&lt;h2 id="footnotes"&gt;Footnotes&lt;a class="headerlink" href="#footnotes" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;&lt;a id="footnote1"&gt;&lt;/a&gt;&lt;sup&gt;&lt;a href="#fn1"&gt;1&lt;/a&gt;&lt;/sup&gt; &lt;em&gt;Dynamic Filters (DF)&lt;/em&gt; refers to the
optimization described in this blog post. The TopK operator will generate a
filter that is applied to the scan operators, which will first be used to skip
rows and then as we open new files (if there are more to open) it will be used
to skip entire files that do not match the filter.&lt;/p&gt;
&lt;p&gt;&lt;a id="footnote2"&gt;&lt;/a&gt;&lt;sup&gt;&lt;a href="#fn2"&gt;2&lt;/a&gt;&lt;/sup&gt; &lt;em&gt;Late Materialization (LM)&lt;/em&gt; refers to
the optimization described in &lt;a href="https://datafusion.apache.org/blog/2025/03/21/parquet-pushdown/"&gt;this blog post&lt;/a&gt;. Late Materialization is
particularly effective when combined with dynamic filters as it can apply
filters during a scan. Without late materialization, dynamic filters can only be
used to prune row groups or entire files, which will be less effective if the
files themselves are large or the top values are not in the first few files read.&lt;/p&gt;
&lt;h2 id="appendix"&gt;Appendix&lt;a class="headerlink" href="#appendix" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;h3 id="queries-and-data"&gt;Queries and Data&lt;a class="headerlink" href="#queries-and-data" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;h4 id="figure-1-clickbench-q23"&gt;Figure 1: ClickBench Q23&lt;a class="headerlink" href="#figure-1-clickbench-q23" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h4&gt;
&lt;pre&gt;&lt;code class="language-sql"&gt;-- Data was downloaded using apache/datafusion -&amp;gt; benchmarks/bench.sh -&amp;gt; ./benchmarks/bench.sh data clickbench_partitioned
create external table hits stored as parquet location 'benchmarks/data/hits_partitioned';

-- Must set for ClickBench hits_partitioned dataset. See https://github.com/apache/datafusion/issues/16591
set datafusion.execution.parquet.binary_as_string = true;
-- Only matters if pushdown_filters is enabled but they don't get enabled together sadly
set datafusion.execution.parquet.reorder_filters = true;

set datafusion.execution.target_partitions = 1;  -- or set to 12 to use multiple cores
set datafusion.optimizer.enable_dynamic_filter_pushdown = false;
set datafusion.execution.parquet.pushdown_filters = false;

explain analyze
SELECT *
FROM hits
WHERE "URL" LIKE '%google%'
ORDER BY "EventTime"
LIMIT 10;
&lt;/code&gt;&lt;/pre&gt;
&lt;table class="table"&gt;
&lt;thead&gt;
&lt;tr&gt;
&lt;th style="text-align: left;"&gt;dynamic filters&lt;/th&gt;
&lt;th style="text-align: left;"&gt;late materialization&lt;/th&gt;
&lt;th style="text-align: right;"&gt;cores&lt;/th&gt;
&lt;th style="text-align: right;"&gt;time (s)&lt;/th&gt;
&lt;/tr&gt;
&lt;/thead&gt;
&lt;tbody&gt;
&lt;tr&gt;
&lt;td style="text-align: left;"&gt;False&lt;/td&gt;
&lt;td style="text-align: left;"&gt;False&lt;/td&gt;
&lt;td style="text-align: right;"&gt;1&lt;/td&gt;
&lt;td style="text-align: right;"&gt;32.039&lt;/td&gt;
&lt;/tr&gt;
&lt;tr&gt;
&lt;td style="text-align: left;"&gt;False&lt;/td&gt;
&lt;td style="text-align: left;"&gt;True&lt;/td&gt;
&lt;td style="text-align: right;"&gt;1&lt;/td&gt;
&lt;td style="text-align: right;"&gt;16.903&lt;/td&gt;
&lt;/tr&gt;
&lt;tr&gt;
&lt;td style="text-align: left;"&gt;True&lt;/td&gt;
&lt;td style="text-align: left;"&gt;False&lt;/td&gt;
&lt;td style="text-align: right;"&gt;1&lt;/td&gt;
&lt;td style="text-align: right;"&gt;18.195&lt;/td&gt;
&lt;/tr&gt;
&lt;tr&gt;
&lt;td style="text-align: left;"&gt;True&lt;/td&gt;
&lt;td style="text-align: left;"&gt;True&lt;/td&gt;
&lt;td style="text-align: right;"&gt;1&lt;/td&gt;
&lt;td style="text-align: right;"&gt;1.42&lt;/td&gt;
&lt;/tr&gt;
&lt;tr&gt;
&lt;td style="text-align: left;"&gt;False&lt;/td&gt;
&lt;td style="text-align: left;"&gt;False&lt;/td&gt;
&lt;td style="text-align: right;"&gt;12&lt;/td&gt;
&lt;td style="text-align: right;"&gt;5.04&lt;/td&gt;
&lt;/tr&gt;
&lt;tr&gt;
&lt;td style="text-align: left;"&gt;False&lt;/td&gt;
&lt;td style="text-align: left;"&gt;True&lt;/td&gt;
&lt;td style="text-align: right;"&gt;12&lt;/td&gt;
&lt;td style="text-align: right;"&gt;2.37&lt;/td&gt;
&lt;/tr&gt;
&lt;tr&gt;
&lt;td style="text-align: left;"&gt;True&lt;/td&gt;
&lt;td style="text-align: left;"&gt;False&lt;/td&gt;
&lt;td style="text-align: right;"&gt;12&lt;/td&gt;
&lt;td style="text-align: right;"&gt;5.055&lt;/td&gt;
&lt;/tr&gt;
&lt;tr&gt;
&lt;td style="text-align: left;"&gt;True&lt;/td&gt;
&lt;td style="text-align: left;"&gt;True&lt;/td&gt;
&lt;td style="text-align: right;"&gt;12&lt;/td&gt;
&lt;td style="text-align: right;"&gt;0.602&lt;/td&gt;
&lt;/tr&gt;
&lt;/tbody&gt;
&lt;/table&gt;</content><category term="blog"/></entry><entry><title>Using External Indexes, Metadata Stores, Catalogs and Caches to Accelerate Queries on Apache Parquet</title><link href="https://datafusion.apache.org/blog/2025/08/15/external-parquet-indexes" rel="alternate"/><published>2025-08-15T00:00:00+00:00</published><updated>2025-08-15T00:00:00+00:00</updated><author><name>Andrew Lamb (InfluxData)</name></author><id>tag:datafusion.apache.org,2025-08-15:/blog/2025/08/15/external-parquet-indexes</id><summary type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;
&lt;!-- diagrams source https://docs.google.com/presentation/d/1e_Z_F8nt2rcvlNvhU11khF5lzJJVqNtqtyJ-G3mp4-Q --&gt;

&lt;p&gt;It is a common misconception that &lt;a href="https://parquet.apache.org/"&gt;Apache Parquet&lt;/a&gt; requires (slow) reparsing of
metadata and is limited to indexing structures provided by the format. In fact,
caching parsed metadata and using custom external indexes along with
Parquet's hierarchical data organization can significantly speed up query
processing.&lt;/p&gt;
&lt;p&gt;In this blog, I describe …&lt;/p&gt;</summary><content type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;
&lt;!-- diagrams source https://docs.google.com/presentation/d/1e_Z_F8nt2rcvlNvhU11khF5lzJJVqNtqtyJ-G3mp4-Q --&gt;

&lt;p&gt;It is a common misconception that &lt;a href="https://parquet.apache.org/"&gt;Apache Parquet&lt;/a&gt; requires (slow) reparsing of
metadata and is limited to indexing structures provided by the format. In fact,
caching parsed metadata and using custom external indexes along with
Parquet's hierarchical data organization can significantly speed up query
processing.&lt;/p&gt;
&lt;p&gt;In this blog, I describe the role of external indexes, caches, and metadata
stores in high performance systems, and demonstrate how to apply these concepts
to Parquet processing using &lt;a href="https://datafusion.apache.org/"&gt;Apache DataFusion&lt;/a&gt;. &lt;em&gt;Note this is an expanded
version of the &lt;a href="https://www.youtube.com/watch?v=74YsJT1-Rdk"&gt;companion video&lt;/a&gt; and &lt;a href="https://docs.google.com/presentation/d/1e_Z_F8nt2rcvlNvhU11khF5lzJJVqNtqtyJ-G3mp4-Q/edit"&gt;presentation&lt;/a&gt;.&lt;/em&gt;&lt;/p&gt;
&lt;h2 id="motivation"&gt;Motivation&lt;a class="headerlink" href="#motivation" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;System designers choose between a pre-configured data system or the often
daunting task of building their own custom data platform from scratch.
For many users and use cases, one of the existing data systems will
likely be good enough. However, traditional systems such as &lt;a href="https://spark.apache.org/"&gt;Apache Spark&lt;/a&gt;, &lt;a href="https://duckdb.org/"&gt;DuckDB&lt;/a&gt;,
&lt;a href="https://clickhouse.com/"&gt;ClickHouse&lt;/a&gt;, &lt;a href="https://hive.apache.org/"&gt;Hive&lt;/a&gt;, or &lt;a href="https://www.snowflake.com/"&gt;Snowflake&lt;/a&gt; are each optimized for a certain set of
tradeoffs between performance, cost, availability, interoperability, deployment
target, cloud / on-premises, operational ease and many other factors.&lt;/p&gt;
&lt;p&gt;For new, or especially demanding use cases, where no existing system makes your
optimal tradeoffs, you can build your own custom data platform. Previously this
was a long and expensive endeavor, but today, in the era of &lt;a href="https://www.vldb.org/pvldb/vol16/p2679-pedreira.pdf"&gt;Composable Data
Systems&lt;/a&gt;, it is increasingly feasible. High quality, open source building blocks
such as &lt;a href="https://parquet.apache.org/"&gt;Apache Parquet&lt;/a&gt; for storage, &lt;a href="https://arrow.apache.org/"&gt;Apache Arrow&lt;/a&gt; for in-memory processing,
and &lt;a href="https://datafusion.apache.org/"&gt;Apache DataFusion&lt;/a&gt; for query execution make it possible to quickly build
custom data platforms optimized for your specific
needs&lt;sup&gt;&lt;a href="#footnote1"&gt;1&lt;/a&gt;&lt;/sup&gt;.&lt;/p&gt;
&lt;h2 id="introduction-to-external-indexes-catalogs-metadata-stores-caches"&gt;Introduction to External Indexes / Catalogs / Metadata Stores / Caches&lt;a class="headerlink" href="#introduction-to-external-indexes-catalogs-metadata-stores-caches" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;div class="text-center"&gt;
&lt;img alt="Using External Indexes to Accelerate Queries" class="img-fluid" src="/blog/images/external-parquet-indexes/external-index-overview.png" width="80%"/&gt;
&lt;/div&gt;
&lt;p&gt;&lt;strong&gt;Figure 1&lt;/strong&gt;: Using external indexes to speed up queries in an analytic system.
Given a user's query (Step 1), the system uses an external index (one that is not
stored inline in the data files) to quickly find files that may contain
relevant data (Step 2). Then, for each file, the system uses the external index
to further narrow the required data to only those &lt;strong&gt;parts&lt;/strong&gt; of each file
(e.g. data pages) that are relevant (Step 3). Finally, the system reads only those
parts of the file and returns the results to the user (Step 4).&lt;/p&gt;
&lt;p&gt;In this blog, I use the term &lt;strong&gt;"index"&lt;/strong&gt; to mean any structure that helps
locate relevant data during processing, and a high level overview of how
external indexes are used to speed up queries is shown in Figure 1.&lt;/p&gt;
&lt;p&gt;All data systems typically store both the data itself and additional information
(metadata) to more quickly find data relevant to a query. Metadata is often
stored in structures with names like "index," "catalog" and "cache" and the
terminology varies widely across systems. &lt;/p&gt;
&lt;p&gt;There are many different types of indexes, types of content stored in indexes,
strategies to keep indexes up to date, and ways to apply indexes during query
processing. These differences each have their own set of tradeoffs, and thus
different systems understandably make different choices depending on their use
case. There is no one-size-fits-all solution for indexing. For example, Hive
uses the &lt;a href="https://cwiki.apache.org/confluence/display/Hive/Design#Design-Metastore"&gt;Hive Metastore&lt;/a&gt;, &lt;a href="https://www.vertica.com/"&gt;Vertica&lt;/a&gt; uses a purpose-built &lt;a href="https://www.vertica.com/docs/latest/HTML/Content/Authoring/AdministratorsGuide/Managing/Metadata/CatalogOverview.htm"&gt;Catalog&lt;/a&gt;, and open
data lake systems typically use a table format such as &lt;a href="https://iceberg.apache.org/"&gt;Apache Iceberg&lt;/a&gt; or &lt;a href="https://delta.io/"&gt;Delta
Lake&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;External Indexes&lt;/strong&gt; store information separately ("external") to the data
itself. External indexes are flexible and widely used, but require additional
operational overhead to keep in sync with the data files. For example, if you
add a new Parquet file to your data lake, you must also update the relevant
external index to include information about the new file. Note, you can
avoid the operational overhead of external indexes by using only the data files
themselves, including &lt;a href="https://datafusion.apache.org/blog/2025/07/14/user-defined-parquet-indexes/"&gt;Embedding User-Defined Indexes in Apache Parquet
Files&lt;/a&gt;. However, this approach comes with its own set of tradeoffs such as 
increased file sizes and the need to update the data files to update the index.&lt;/p&gt;
&lt;p&gt;Examples of information commonly stored in external indexes include:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;Min/Max statistics&lt;/li&gt;
&lt;li&gt;Bloom filters&lt;/li&gt;
&lt;li&gt;Inverted indexes / Full Text indexes &lt;/li&gt;
&lt;li&gt;Information needed to read the remote file (e.g the schema, or Parquet footer metadata)&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;Examples of locations where external indexes can be stored include:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;strong&gt;Separate files&lt;/strong&gt; such as &lt;a href="https://www.json.org/"&gt;JSON&lt;/a&gt; or Parquet files.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Transactional databases&lt;/strong&gt; such as &lt;a href="https://www.postgresql.org/"&gt;PostgreSQL&lt;/a&gt; tables.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Distributed key-value stores&lt;/strong&gt; such as &lt;a href="https://redis.io/"&gt;Redis&lt;/a&gt; or &lt;a href="https://cassandra.apache.org/"&gt;Cassandra&lt;/a&gt;.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Local memory&lt;/strong&gt; such as an in-memory hash map.&lt;/li&gt;
&lt;/ul&gt;
&lt;h2 id="using-apache-parquet-for-storage"&gt;Using Apache Parquet for Storage&lt;a class="headerlink" href="#using-apache-parquet-for-storage" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;While the rest of this blog focuses on building custom external indexes using
Parquet and DataFusion, I first briefly discuss why Parquet is a good choice for
modern analytic systems. The research community frequently confuses limitations
of a particular &lt;a href="https://parquet.apache.org/docs/file-format/implementationstatus/"&gt;implementation of the Parquet format&lt;/a&gt; with the &lt;a href="https://parquet.apache.org/docs/file-format/"&gt;Parquet Format&lt;/a&gt;
itself, and this confusion often obscures capabilities that make Parquet a good
target for external indexes.&lt;/p&gt;
&lt;p&gt;Apache Parquet's combination of good compression, high-performance, high quality
open source libraries, and wide ecosystem interoperability make it a compelling
choice when building new systems. While there are some niche use cases that may
benefit from specialized formats, Parquet is typically the obvious choice.
While recent proprietary file formats differ in details, they all use the same
high level structure&lt;sup&gt;&lt;a href="#footnote2"&gt;2&lt;/a&gt;&lt;/sup&gt; as Parquet: &lt;/p&gt;
&lt;ol&gt;
&lt;li&gt;Metadata (typically at the end  of the file)&lt;/li&gt;
&lt;li&gt;Data divided into columns and then into horizontal slices (e.g. Parquet Row Groups and/or Data Pages). &lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;The structure is so widespread because it enables the hierarchical pruning
approach described in the next section. For example, the native &lt;a href="https://clickhouse.com/docs/engines/table-engines/mergetree-family/mergetree"&gt;Clickhouse
MergeTree&lt;/a&gt; format consists of &lt;em&gt;Parts&lt;/em&gt; (similar to Parquet files), and &lt;em&gt;Granules&lt;/em&gt;
(similar to Row Groups). The &lt;a href="https://clickhouse.com/docs/guides/best-practices/sparse-primary-indexes#clickhouse-index-design"&gt;Clickhouse indexing strategy&lt;/a&gt; follows a classic
hierarchical pruning approach that first locates the Parts and then the Granules
that may contain relevant data for the query. This is exactly the same pattern
as Parquet based systems, which first locate the relevant Parquet files and then
the Row Groups / Data Pages within those files.&lt;/p&gt;
&lt;p&gt;A common criticism of using Parquet is that it is not as performant as some new
proposal. These criticisms typically cherry-pick a few queries and/or datasets
and build a specialized index or data layout for that specific case. However,
as I explain in the &lt;a href="https://www.youtube.com/watch?v=74YsJT1-Rdk"&gt;companion video&lt;/a&gt; of this blog, even for
&lt;a href="https://clickbench.com/"&gt;ClickBench&lt;/a&gt;&lt;sup&gt;&lt;a href="#footnote6"&gt;6&lt;/a&gt;&lt;/sup&gt;, the current
benchmaxxing&lt;sup&gt;&lt;a href="#footnote3"&gt;3&lt;/a&gt;&lt;/sup&gt; target of analytics vendors, there is
less than a factor of two difference in performance between custom file formats
and Parquet. The difference becomes even lower when using Parquet files that
use the full range of existing Parquet features such Column and Offset
Indexes and Bloom Filters&lt;sup&gt;&lt;a href="#footnote7"&gt;7&lt;/a&gt;&lt;/sup&gt;. Compared to the low
interoperability and expensive transcoding/loading step of alternate file
formats, Parquet is hard to beat.&lt;/p&gt;
&lt;h2 id="hierarchical-pruning-overview"&gt;Hierarchical Pruning Overview&lt;a class="headerlink" href="#hierarchical-pruning-overview" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;The key technique for optimizing query processing systems is skipping as
much data as possible, as quickly as possible. Analytic systems typically use a hierarchical
approach to progressively narrow the set of data that needs to be processed. 
The standard approach is shown in Figure 2:&lt;/p&gt;
&lt;ol&gt;
&lt;li&gt;Entire files are ruled out&lt;/li&gt;
&lt;li&gt;Within each file, large sections (e.g. Row Groups) are ruled out&lt;/li&gt;
&lt;li&gt;(Optionally) smaller sections (e.g. Data Pages) are ruled out&lt;/li&gt;
&lt;li&gt;Finally, the system reads only the relevant data pages and applies the query
   predicate to the data&lt;/li&gt;
&lt;/ol&gt;
&lt;div class="text-center"&gt;
&lt;img alt="Standard Pruning Layers." class="img-fluid" src="/blog/images/external-parquet-indexes/processing-pipeline.png" width="80%"/&gt;
&lt;/div&gt;
&lt;p&gt;&lt;strong&gt;Figure 2&lt;/strong&gt;: Hierarchical Pruning: The system first rules out files, then
Row Groups, then Data Pages, and finally reads only the relevant data pages.&lt;/p&gt;
&lt;p&gt;The process is hierarchical because the per-row computation required at the
earlier stages (e.g. skipping an entire file) is lower than the computation
required at later stages (apply predicates to the data). 
As mentioned before, while the details of what metadata is used and how that
metadata is managed varies substantially across query systems, they almost all
use a hierarchical pruning strategy.&lt;/p&gt;
&lt;h2 id="apache-parquet-overview"&gt;Apache Parquet Overview&lt;a class="headerlink" href="#apache-parquet-overview" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;This section provides a brief background on the organization of Apache Parquet
files which is needed to fully understand the sections on implementing external indexes.
If you are already familiar with Parquet, you can skip this section.&lt;/p&gt;
&lt;p&gt;Logically, Parquet files are organized into  &lt;em&gt;Row Groups&lt;/em&gt; and &lt;em&gt;Column Chunks&lt;/em&gt; as
shown below.&lt;/p&gt;
&lt;div class="text-center"&gt;
&lt;img alt="Logical Parquet File layout: Row Groups and Column Chunks." class="img-fluid" src="/blog/images/external-parquet-indexes/parquet-layout.png" width="80%"/&gt;
&lt;/div&gt;
&lt;p&gt;&lt;strong&gt;Figure 3&lt;/strong&gt;: Logical Parquet File Layout: Data is first divided in horizontal slices
called Row Groups. The data is then stored column by column in &lt;em&gt;Column Chunks&lt;/em&gt;.
This arrangement allows efficient access to only the portions of columns needed
for a query.&lt;/p&gt;
&lt;p&gt;Physically, Parquet data is stored as a series of Data Pages along with metadata
stored at the end of the file (in the footer), as shown below.&lt;/p&gt;
&lt;div class="text-center"&gt;
&lt;img alt="Physical Parquet File layout: Metadata and Footer." class="img-fluid" src="/blog/images/external-parquet-indexes/parquet-metadata.png" width="80%"/&gt;
&lt;/div&gt;
&lt;p&gt;&lt;strong&gt;Figure 4&lt;/strong&gt;: Physical Parquet File Layout: A typical Parquet file is composed
of many data pages,  which contain the raw encoded data, and a footer that
stores metadata about the file, including the schema and the location of the
relevant data pages, and optional statistics such as min/max values for each
Column Chunk.&lt;/p&gt;
&lt;p&gt;Parquet files are organized to minimize IO and processing using two key mechanisms:&lt;/p&gt;
&lt;ol&gt;
&lt;li&gt;
&lt;p&gt;&lt;strong&gt;Projection Pushdown&lt;/strong&gt;: if a query needs only a subset of columns from a table, it
   only needs to read the pages for the relevant Column Chunks&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;
&lt;p&gt;&lt;strong&gt;Filter Pushdown&lt;/strong&gt;: Similarly, given a query with a filter predicate such as
   &lt;code&gt;WHERE C &amp;gt; 25&lt;/code&gt;, query engines can use statistics such as (but not limited to)
   the min/max values stored in the metadata to skip reading and decoding pages that
   cannot possibly match the predicate.&lt;/p&gt;
&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;The high level mechanics of Parquet predicate pushdown is shown below:&lt;/p&gt;
&lt;div class="text-center"&gt;
&lt;img alt="Parquet Filter Pushdown: use filter predicate to skip pages." class="img-fluid" src="/blog/images/external-parquet-indexes/parquet-filter-pushdown.png" width="80%"/&gt;
&lt;/div&gt;
&lt;p&gt;&lt;strong&gt;Figure 5&lt;/strong&gt;: Filter Pushdown in Parquet: query engines use the predicate,
&lt;code&gt;C &amp;gt; 25&lt;/code&gt;, from the query along with statistics from the metadata, to identify
pages that may match the predicate which are read for further processing. 
Please refer to the &lt;a href="https://datafusion.apache.org/blog/2025/03/21/parquet-pushdown"&gt;Efficient Filter Pushdown&lt;/a&gt; blog for more details.
&lt;strong&gt;NOTE the exact same pattern can be applied using information from external
indexes, as described in the next sections.&lt;/strong&gt;&lt;/p&gt;
&lt;h2 id="pruning-files-with-external-indexes"&gt;Pruning Files with External Indexes&lt;a class="headerlink" href="#pruning-files-with-external-indexes" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;The first step in hierarchical pruning is quickly ruling out files that cannot
match the query. For example, if a system expects to see queries that
apply to a time range, it might create an external index to store the minimum
and maximum &lt;code&gt;time&lt;/code&gt; values for each file. Then, during query processing, the
system can quickly rule out files that cannot possibly contain relevant data.&lt;/p&gt;
&lt;p&gt;For example, if the user issues a query that only matches the last 7 days of
data:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-sql"&gt;WHERE time &amp;gt; now() - interval '7 days'
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;then the index can quickly rule out files that only have data older than the
most recent 7 days.&lt;/p&gt;
&lt;div class="text-center"&gt;
&lt;img alt="Data Skipping: Pruning Files." class="img-fluid" src="/blog/images/external-parquet-indexes/prune-files.png" width="80%"/&gt;
&lt;/div&gt;
&lt;p&gt;&lt;strong&gt;Figure 6&lt;/strong&gt;: Step 1: File Pruning. Given a query predicate, systems use external
indexes to quickly rule out files that cannot match the query. In this case, by
consulting the index all but two files can be ruled out.&lt;/p&gt;
&lt;p&gt;External indexes offer much faster lookups and lower I/O overhead than Parquet's
built-in file-level indexes by skipping further processing for many data files.
Without an external index, systems typically fall back to reading each file's
footer to find files needed for further processing. Skipping per-file processing
is especially important when reading from remote object stores such as &lt;a href="https://aws.amazon.com/s3/"&gt;S3&lt;/a&gt;,
&lt;a href="https://cloud.google.com/storage"&gt;GCS&lt;/a&gt; or &lt;a href="https://azure.microsoft.com/en-us/services/storage/blobs/"&gt;Azure Blob Store&lt;/a&gt;, where each request adds &lt;a href="https://www.vldb.org/pvldb/vol16/p2769-durner.pdf"&gt;tens to hundreds of
milliseconds of latency&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;There are many different systems that use external indexes to find files such as 
&lt;a href="https://cwiki.apache.org/confluence/display/Hive/Design#Design-Metastore"&gt;Hive Metadata Store&lt;/a&gt;,
&lt;a href="https://iceberg.apache.org/"&gt;Iceberg&lt;/a&gt;, 
&lt;a href="https://delta.io/"&gt;Delta Lake&lt;/a&gt;,
&lt;a href="https://duckdb.org/2025/05/27/ducklake.html"&gt;DuckLake&lt;/a&gt;,
and &lt;a href="https://sparkbyexamples.com/apache-hive/hive-partitions-explained-with-examples/"&gt;Hive Style Partitioning&lt;/a&gt;&lt;sup&gt;&lt;a href="#footnote4"&gt;4&lt;/a&gt;&lt;/sup&gt;.
Of course, each of these systems works well for their intended use cases, but
if none meets your needs, or you want to experiment with
different strategies, you can easily build your own external index using
DataFusion.&lt;/p&gt;
&lt;h3 id="pruning-files-with-external-indexes-using-datafusion"&gt;Pruning Files with External Indexes Using DataFusion&lt;a class="headerlink" href="#pruning-files-with-external-indexes-using-datafusion" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;To implement file pruning in DataFusion, you implement a custom &lt;a href="https://docs.rs/datafusion/latest/datafusion/datasource/trait.TableProvider.html"&gt;TableProvider&lt;/a&gt;
with the &lt;a href="https://docs.rs/datafusion/latest/datafusion/datasource/trait.TableProvider.html#method.supports_filters_pushdown"&gt;supports_filter_pushdown&lt;/a&gt; and &lt;a href="https://docs.rs/datafusion/latest/datafusion/datasource/trait.TableProvider.html#tymethod.scan"&gt;scan&lt;/a&gt; methods. The
&lt;code&gt;supports_filter_pushdown&lt;/code&gt; method tells DataFusion which predicates can be used
and the &lt;code&gt;scan&lt;/code&gt; method uses those predicates with the
external index to find the files that may contain data that matches the query.&lt;/p&gt;
&lt;p&gt;The DataFusion repository contains a fully working and well-commented example,
&lt;a href="https://github.com/apache/datafusion/blob/main/datafusion-examples/examples/parquet_index.rs"&gt;parquet_index.rs&lt;/a&gt;, of this technique that you can use as a starting point. 
The example creates a simple index that stores the min/max values for a column
called &lt;code&gt;value&lt;/code&gt; along with the file name. Then it runs the following query:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-sql"&gt;SELECT file_name, value FROM index_table WHERE value = 150
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;The custom &lt;code&gt;IndexTableProvider&lt;/code&gt;'s &lt;code&gt;scan&lt;/code&gt; method uses the index to find files
that may contain data matching the predicate as shown below:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-rust"&gt;impl TableProvider for IndexTableProvider {
    async fn scan(
        &amp;amp;self,
        state: &amp;amp;dyn Session,
        projection: Option&amp;lt;&amp;amp;Vec&amp;lt;usize&amp;gt;&amp;gt;,
        filters: &amp;amp;[Expr],
        limit: Option&amp;lt;usize&amp;gt;,
    ) -&amp;gt; Result&amp;lt;Arc&amp;lt;dyn ExecutionPlan&amp;gt;&amp;gt; {
        let df_schema = DFSchema::try_from(self.schema())?;
        // Combine all the filters into a single ANDed predicate
        let predicate = conjunction(filters.to_vec());

        // Use the index to find the files that might have data that matches the
        // predicate. Any file that can not have data that matches the predicate
        // will not be returned.
        let files = self.index.get_files(predicate.clone())?;

        let object_store_url = ObjectStoreUrl::parse("file://")?;
        let source = Arc::new(ParquetSource::default().with_predicate(predicate));
        let mut file_scan_config_builder =
            FileScanConfigBuilder::new(object_store_url, self.schema(), source)
                .with_projection(projection.cloned())
                .with_limit(limit);

        // Add the files to the scan config
        for file in files {
            file_scan_config_builder = file_scan_config_builder.with_file(
                PartitionedFile::new(file.path(), file_size.size()),
            );
        }
        Ok(DataSourceExec::from_data_source(
            file_scan_config_builder.build(),
        ))
    }
    ...
}
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;DataFusion handles the details of pushing down the filters to the
&lt;code&gt;TableProvider&lt;/code&gt; and the mechanics of reading the Parquet files, so you can focus
on the system specific details such as building, storing, and applying the index.
While this example uses a standard min/max index, you can implement any indexing
strategy you need, such as bloom filters, a full text index, or a more complex
multidimensional index.&lt;/p&gt;
&lt;p&gt;DataFusion also includes several libraries to help with common filtering and
pruning tasks, such as:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;
&lt;p&gt;A full and well documented expression representation (&lt;a href="https://docs.rs/datafusion/latest/datafusion/logical_expr/enum.Expr.html"&gt;Expr&lt;/a&gt;) and &lt;a href="https://docs.rs/datafusion/latest/datafusion/logical_expr/enum.Expr.html#visiting-and-rewriting-exprs"&gt;APIs for
  building, visiting, and rewriting&lt;/a&gt; query predicates.&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;
&lt;p&gt;Range Based Pruning (&lt;a href="https://docs.rs/datafusion/latest/datafusion/physical_optimizer/pruning/struct.PruningPredicate.html"&gt;PruningPredicate&lt;/a&gt;) for cases where your index stores
  min/max values.&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;
&lt;p&gt;Expression simplification (&lt;a href="https://docs.rs/datafusion/latest/datafusion/optimizer/simplify_expressions/struct.ExprSimplifier.html#method.simplify"&gt;ExprSimplifier&lt;/a&gt;) for simplifying predicates before
  applying them to the index.&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;
&lt;p&gt;Range analysis for predicates (&lt;a href="https://docs.rs/datafusion/latest/datafusion/physical_expr/intervals/cp_solver/index.html"&gt;cp_solver&lt;/a&gt;) for interval-based range analysis
  (e.g. &lt;code&gt;col &amp;gt; 5 AND col &amp;lt; 10&lt;/code&gt;).&lt;/p&gt;
&lt;/li&gt;
&lt;/ul&gt;
&lt;h2 id="pruning-parts-of-parquet-files-with-external-indexes"&gt;Pruning Parts of Parquet Files with External Indexes&lt;a class="headerlink" href="#pruning-parts-of-parquet-files-with-external-indexes" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;Once the set of files to be scanned has been determined, the next step in the
hierarchical pruning process is to further narrow down the data within each file.
Similarly to the previous step, almost all advanced query processing systems use additional
metadata to prune unnecessary parts of the file, such as &lt;a href="https://clickhouse.com/docs/optimize/skipping-indexes"&gt;Data Skipping Indexes
in ClickHouse&lt;/a&gt;. &lt;/p&gt;
&lt;p&gt;For Parquet-based systems, the most common strategy is using the built-in metadata such
as &lt;a href="https://github.com/apache/parquet-format/blob/1dbc814b97c9307687a2e4bee55545ab6a2ef106/src/main/thrift/parquet.thrift#L267"&gt;min/max statistics&lt;/a&gt; and &lt;a href="https://parquet.apache.org/docs/file-format/bloomfilter/"&gt;Bloom Filters&lt;/a&gt;. However, it is also possible to use external
indexes for filtering &lt;em&gt;WITHIN&lt;/em&gt; Parquet files as shown below. &lt;/p&gt;
&lt;p&gt;&lt;img alt="Data Skipping: Pruning Row Groups and DataPages" class="img-fluid" src="/blog/images/external-parquet-indexes/prune-row-groups.png" width="80%"/&gt;&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Figure 7&lt;/strong&gt;: Step 2: Pruning Parquet Row Groups and Data Pages. Given a query predicate,
systems can use external indexes / metadata stores as well as Parquet's built-in
structures to quickly rule out Row Groups and Data Pages that cannot match the query.
In this case, the index has ruled out all but three data pages which must then be fetched
for more processing.&lt;/p&gt;
&lt;h2 id="pruning-parts-of-parquet-files-with-external-indexes-using-datafusion"&gt;Pruning Parts of Parquet Files with External Indexes using DataFusion&lt;a class="headerlink" href="#pruning-parts-of-parquet-files-with-external-indexes-using-datafusion" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;To implement pruning within Parquet files, you use the same [&lt;code&gt;TableProvider&lt;/code&gt;] APIs
as for pruning files. For each file your provider wants to scan, you provide 
an additional &lt;a href="https://docs.rs/datafusion/latest/datafusion/datasource/physical_plan/parquet/struct.ParquetAccessPlan.html"&gt;ParquetAccessPlan&lt;/a&gt; that tells DataFusion what parts of the file to read. This plan is
then &lt;a href="https://docs.rs/datafusion/latest/datafusion/datasource/physical_plan/parquet/source/struct.ParquetSource.html#implementing-external-indexes"&gt;further refined by the DataFusion Parquet reader&lt;/a&gt; using the built-in
Parquet metadata to potentially prune additional row groups and data pages
during query execution. You can find a full working example in
the &lt;a href="https://github.com/apache/datafusion/blob/main/datafusion-examples/examples/advanced_parquet_index.rs"&gt;advanced_parquet_index.rs&lt;/a&gt; example of the DataFusion repository.&lt;/p&gt;
&lt;p&gt;Here is how you build a &lt;code&gt;ParquetAccessPlan&lt;/code&gt; to scan only specific row groups
and rows within those row groups. &lt;/p&gt;
&lt;pre&gt;&lt;code class="language-rust"&gt;// Default to scan all (4) row groups
let mut access_plan = ParquetAccessPlan::new_all(4);
access_plan.skip(0); // skip row group 0
// Specify scanning rows 100-200 and 350-400
// in row group 1 that has 1000 rows
let row_selection = RowSelection::from(vec![
   RowSelector::skip(100),
   RowSelector::select(100),
   RowSelector::skip(150),
   RowSelector::select(50),
   RowSelector::skip(600),  // skip last 600 rows
]);
access_plan.scan_selection(1, row_selection);
access_plan.skip(2); // skip row group 2
// all of row group 3 is scanned by default
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;The rows that are selected by the resulting plan look like this:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-text"&gt;┌───────────────────┐
│                   │
│                   │  SKIP
│                   │
└───────────────────┘
     Row Group 0

┌───────────────────┐
│ ┌───────────────┐ │  SCAN ONLY ROWS
│ └───────────────┘ │  100-200
│ ┌───────────────┐ │  350-400
│ └───────────────┘ │
└───────────────────┘
     Row Group 1

┌───────────────────┐
│                   │
│                   │  SKIP
│                   │
└───────────────────┘
     Row Group 2

┌───────────────────┐
│                   │
│                   │  SCAN ALL ROWS
│                   │
└───────────────────┘
     Row Group 3
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;In the &lt;code&gt;scan&lt;/code&gt; method, you return an &lt;code&gt;ExecutionPlan&lt;/code&gt; that includes the
&lt;code&gt;ParquetAccessPlan&lt;/code&gt; for each file as shown below (again, slightly simplified for
clarity):&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-rust"&gt;impl TableProvider for IndexTableProvider {
    async fn scan(
        &amp;amp;self,
        state: &amp;amp;dyn Session,
        projection: Option&amp;lt;&amp;amp;Vec&amp;lt;usize&amp;gt;&amp;gt;,
        filters: &amp;amp;[Expr],
        limit: Option&amp;lt;usize&amp;gt;,
    ) -&amp;gt; Result&amp;lt;Arc&amp;lt;dyn ExecutionPlan&amp;gt;&amp;gt; {
        let indexed_file = &amp;amp;self.indexed_file;
        let predicate = self.filters_to_predicate(state, filters)?;

        // Use the external index to create a starting ParquetAccessPlan
        // that determines which row groups to scan based on the predicate
        let access_plan = self.create_plan(&amp;amp;predicate)?;

        let partitioned_file = indexed_file
            .partitioned_file()
            // provide the access plan to the DataSourceExec by
            // storing it as  "extensions" on PartitionedFile
            .with_extensions(Arc::new(access_plan) as _);

        let file_source = Arc::new(
            ParquetSource::default()
                // provide the predicate to the standard DataFusion source as well so
                // DataFusion's Parquet reader will apply row group pruning based on
                // the built-in Parquet metadata (min/max, bloom filters, etc) as well
                .with_predicate(predicate)
        );
        let file_scan_config =
            FileScanConfigBuilder::new(object_store_url, schema, file_source)
                .with_limit(limit)
                .with_projection(projection.cloned())
                .with_file(partitioned_file)
                .build();

        // Finally, put it all together into a DataSourceExec
        Ok(DataSourceExec::from_data_source(file_scan_config))
    }
    ...
}
&lt;/code&gt;&lt;/pre&gt;
&lt;h2 id="caching-parquet-metadata"&gt;Caching Parquet Metadata&lt;a class="headerlink" href="#caching-parquet-metadata" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;It is often said that Parquet is unsuitable for low latency query systems
because the footer must be read and parsed for each query. This is simply not
true, and &lt;strong&gt;many systems use Parquet for low latency analytics and cache the parsed
metadata in memory to avoid re-reading and re-parsing the footer for each query&lt;/strong&gt;.&lt;/p&gt;
&lt;h3 id="caching-parquet-metadata-using-datafusion"&gt;Caching Parquet Metadata using DataFusion&lt;a class="headerlink" href="#caching-parquet-metadata-using-datafusion" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;Reusing cached Parquet Metadata is also shown in the &lt;a href="https://github.com/apache/datafusion/blob/main/datafusion-examples/examples/advanced_parquet_index.rs"&gt;advanced_parquet_index.rs&lt;/a&gt;
example. The example reads and caches the metadata for each file when the index
is first built and then uses the cached metadata when reading the files during
query execution.&lt;/p&gt;
&lt;p&gt;(Note that thanks to &lt;a href="https://nuno-faria.github.io/"&gt;Nuno Faria&lt;/a&gt;, &lt;a href="https://github.com/jonathanc-n"&gt;Jonathan Chen&lt;/a&gt;, and &lt;a href="https://github.com/shehabgamin"&gt;Shehab Amin&lt;/a&gt; the built
in &lt;a href="https://docs.rs/datafusion/latest/datafusion/datasource/listing/struct.ListingTable.html"&gt;ListingTable&lt;/a&gt; &lt;code&gt;TableProvider&lt;/code&gt; included with DataFusion will cache Parquet
metadata in the next release of DataFusion (50.0.0). See the &lt;a href="https://github.com/apache/datafusion/issues/17000"&gt;mini epic&lt;/a&gt; for
details).&lt;/p&gt;
&lt;p&gt;To avoid reparsing the metadata, first implement a custom
&lt;a href="https://docs.rs/datafusion/latest/datafusion/datasource/physical_plan/trait.ParquetFileReaderFactory.html"&gt;ParquetFileReaderFactory&lt;/a&gt; as shown below, again slightly simplified for
clarity:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-rust"&gt;impl ParquetFileReaderFactory for CachedParquetFileReaderFactory {
    fn create_reader(
        &amp;amp;self,
        _partition_index: usize,
        file_meta: FileMeta,
        metadata_size_hint: Option&amp;lt;usize&amp;gt;,
        _metrics: &amp;amp;ExecutionPlanMetricsSet,
    ) -&amp;gt; Result&amp;lt;Box&amp;lt;dyn AsyncFileReader + Send&amp;gt;&amp;gt; {
        let filename = file_meta.location();

        // Pass along the information to access the underlying storage
        // (e.g. S3, GCS, local filesystem, etc)
        let object_store = Arc::clone(&amp;amp;self.object_store);
        let mut inner =
            ParquetObjectReader::new(object_store, file_meta.object_meta.location)
                .with_file_size(file_meta.object_meta.size);

        // retrieve the pre-parsed metadata from the cache
        // (which was built when the index was built and is kept in memory)
        let metadata = self
            .metadata
            .get(&amp;amp;filename)
            .expect("metadata for file not found: {filename}");

        // Return a ParquetReader that uses the cached metadata
        Ok(Box::new(ParquetReaderWithCache {
            filename,
            metadata: Arc::clone(metadata),
            inner,
        }))
    }
}
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;Then, in your &lt;a href="https://docs.rs/datafusion/latest/datafusion/datasource/trait.TableProvider.html"&gt;TableProvider&lt;/a&gt; use the factory to avoid re-reading the metadata
for each file:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-rust"&gt;impl TableProvider for IndexTableProvider {
    async fn scan(
        &amp;amp;self,
        state: &amp;amp;dyn Session,
        projection: Option&amp;lt;&amp;amp;Vec&amp;lt;usize&amp;gt;&amp;gt;,
        filters: &amp;amp;[Expr],
        limit: Option&amp;lt;usize&amp;gt;,
    ) -&amp;gt; Result&amp;lt;Arc&amp;lt;dyn ExecutionPlan&amp;gt;&amp;gt; {
        // Configure a factory interface to avoid re-reading the metadata for each file
        let reader_factory =
            CachedParquetFileReaderFactory::new(Arc::clone(&amp;amp;self.object_store))
                .with_file(indexed_file);

        // build the partitioned file (see example above for details)
        let partitioned_file = ...; 

        // Create the ParquetSource with the predicate and the factory
        let file_source = Arc::new(
            ParquetSource::default()
                // provide the factory to create Parquet reader without re-reading metadata
                .with_parquet_file_reader_factory(Arc::new(reader_factory)),
        );

        // Pass along the information needed to read the files
        let file_scan_config =
            FileScanConfigBuilder::new(object_store_url, schema, file_source)
                .with_limit(limit)
                .with_projection(projection.cloned())
                .with_file(partitioned_file)
                .build();

        // Finally, put it all together into a DataSourceExec
        Ok(DataSourceExec::from_data_source(file_scan_config))
    }
    ...
}
&lt;/code&gt;&lt;/pre&gt;
&lt;h2 id="conclusion"&gt;Conclusion&lt;a class="headerlink" href="#conclusion" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;Parquet has the right structure for high performance analytics via hierarchical
pruning, and it is straightforward to build external indexes to speed up queries
using DataFusion without changing the file format. If you need to build a custom
data platform, it has never been easier to build it with Parquet and DataFusion.&lt;/p&gt;
&lt;p&gt;I am a firm believer that data systems of the future will be built on a
foundation of modular, high quality, open source components such as Parquet,
Arrow, and DataFusion. We should focus our efforts as a community on
improving these components rather than building new file formats that are
optimized for narrow use cases.&lt;/p&gt;
&lt;p&gt;Come Join Us! 🎣 &lt;/p&gt;
&lt;p&gt;&lt;a href="https://datafusion.apache.org/contributor-guide/communication.html"&gt;
&lt;img alt="https://datafusion.apache.org/" class="img-fluid" src="/blog/images/logo_original4x.png" width="20%"/&gt;
&lt;/a&gt;&lt;/p&gt;
&lt;h2 id="about-the-author"&gt;About the Author&lt;a class="headerlink" href="#about-the-author" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;&lt;a href="https://www.linkedin.com/in/andrewalamb/"&gt;Andrew Lamb&lt;/a&gt; is a Staff Engineer at
&lt;a href="https://www.influxdata.com/"&gt;InfluxData&lt;/a&gt;, and a member of the &lt;a href="https://datafusion.apache.org/"&gt;Apache
DataFusion&lt;/a&gt; and &lt;a href="https://arrow.apache.org/"&gt;Apache Arrow&lt;/a&gt; PMCs. He has been working on
Databases and related systems more than 20 years.&lt;/p&gt;
&lt;h2 id="about-datafusion"&gt;About DataFusion&lt;a class="headerlink" href="#about-datafusion" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;&lt;a href="https://datafusion.apache.org/"&gt;Apache DataFusion&lt;/a&gt; is an extensible query engine toolkit, written
in Rust, that uses &lt;a href="https://arrow.apache.org/"&gt;Apache Arrow&lt;/a&gt; as its in-memory format. DataFusion and
similar technology are part of the next generation “Deconstructed Database”
architectures, where new systems are built on a foundation of fast, modular
components, rather than as a single tightly integrated system.&lt;/p&gt;
&lt;p&gt;The &lt;a href="https://datafusion.apache.org/contributor-guide/communication.html"&gt;DataFusion community&lt;/a&gt; is always looking for new contributors to help
improve the project. If you are interested in learning more about how query
execution works, help document or improve the DataFusion codebase, or just try
it out, we would love for you to join us.&lt;/p&gt;
&lt;h3 id="acknowledgements"&gt;Acknowledgements&lt;a class="headerlink" href="#acknowledgements" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;Thank you to &lt;a href="https://github.com/zhuqi-lucas"&gt;Qi Zhu&lt;/a&gt;, &lt;a href="https://github.com/adamreeve"&gt;Adam Reeve&lt;/a&gt;, &lt;a href="https://github.com/JigaoLuo"&gt;Jigao Luo&lt;/a&gt;, &lt;a href="https://github.com/comphead"&gt;Oleks V&lt;/a&gt;, &lt;a href="https://github.com/shehabgamin"&gt;Shehab Amin&lt;/a&gt;, &lt;a href="https://nuno-faria.github.io/"&gt;Nuno Faria&lt;/a&gt;
and &lt;a href="https://github.com/Omega359"&gt;Bruce Ritchie&lt;/a&gt; for their insightful feedback on this blog post.&lt;/p&gt;
&lt;h3 id="footnotes"&gt;Footnotes&lt;a class="headerlink" href="#footnotes" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;&lt;a id="footnote1"&gt;&lt;/a&gt;&lt;code&gt;1&lt;/code&gt;: This trend is described in more detail in the &lt;a href="https://www.influxdata.com/blog/flight-datafusion-arrow-parquet-fdap-architecture-influxdb/"&gt;FDAP Stack&lt;/a&gt; blog&lt;/p&gt;
&lt;p&gt;&lt;a id="footnote2"&gt;&lt;/a&gt;&lt;code&gt;2&lt;/code&gt;: This layout is referred to as &lt;a href="https://www.vldb.org/conf/2001/P169.pdf"&gt;PAX in the
database literature&lt;/a&gt; after the first research paper to describe the technique.&lt;/p&gt;
&lt;p&gt;&lt;a id="footnote3"&gt;&lt;/a&gt;&lt;code&gt;3&lt;/code&gt;: Benchmaxxing (verb): to add specific optimizations that only
impact benchmark results and are not widely applicable to real world use cases.&lt;/p&gt;
&lt;p&gt;&lt;a id="footnote4"&gt;&lt;/a&gt;&lt;code&gt;4&lt;/code&gt;: Hive Style Partitioning is a simple and widely used form of indexing based on directory paths, where the directory structure is used to
store information about the data in the files. For example, a directory structure like &lt;code&gt;year=2025/month=08/day=15/&lt;/code&gt; can be used to store data for a specific day
and the system can quickly rule out directories that do not match the query predicate.&lt;/p&gt;
&lt;p&gt;&lt;a id="footnote5"&gt;&lt;/a&gt;&lt;code&gt;5&lt;/code&gt;: I am also convinced that we can speed up the process of parsing Parquet footer
with additional engineering effort (see &lt;a href="https://xiangpeng.systems/"&gt;Xiangpeng Hao&lt;/a&gt;'s &lt;a href="https://www.influxdata.com/blog/how-good-parquet-wide-tables/"&gt;previous blog on the
topic&lt;/a&gt;). &lt;a href="https://github.com/etseidl"&gt;Ed Seidl&lt;/a&gt; is beginning this effort. See the &lt;a href="https://github.com/apache/arrow-rs/issues/5854"&gt;ticket&lt;/a&gt; for details.&lt;/p&gt;
&lt;p&gt;&lt;a id="footnote6"&gt;&lt;/a&gt;&lt;code&gt;6&lt;/code&gt;: ClickBench includes a wide variety of query patterns
such as point lookups, filters of different selectivity, and aggregations.&lt;/p&gt;
&lt;p&gt;&lt;a id="footnote7"&gt;&lt;/a&gt;&lt;code&gt;7&lt;/code&gt;: For example, &lt;a href="https://github.com/zhuqi-lucas"&gt;Qi Zhu&lt;/a&gt; was able to speed up reads by over 2x 
simply by rewriting the Parquet files with Offset Indexes and no compression (see &lt;a href="https://github.com/apache/datafusion/issues/16149#issuecomment-2918761743"&gt;issue #16149 comment&lt;/a&gt; for details).
There is likely significant additional performance available by using Bloom Filters and resorting the data
to be clustered in a more optimal way for the queries.&lt;/p&gt;</content><category term="blog"/></entry><entry><title>Apache DataFusion 49.0.0 Released</title><link href="https://datafusion.apache.org/blog/2025/07/28/datafusion-49.0.0" rel="alternate"/><published>2025-07-28T00:00:00+00:00</published><updated>2025-07-28T00:00:00+00:00</updated><author><name>pmc</name></author><id>tag:datafusion.apache.org,2025-07-28:/blog/2025/07/28/datafusion-49.0.0</id><summary type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;!-- see https://github.com/apache/datafusion/issues/16347 for details --&gt;
&lt;h2 id="introduction"&gt;Introduction&lt;a class="headerlink" href="#introduction" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;We are proud to announce the release of &lt;a href="https://crates.io/crates/datafusion/49.0.0"&gt;DataFusion 49.0.0&lt;/a&gt;. This blog post highlights some of
the major improvements since the release of &lt;a href="https://datafusion.apache.org/blog/2025/07/18/datafusion-48.0.0/"&gt;DataFusion 48.0.0&lt;/a&gt;. The complete list of changes is available in the &lt;a href="https://github.com/apache/datafusion/blob/branch-49/dev/changelog/49.0.0.md"&gt;changelog&lt;/a&gt;.&lt;/p&gt;
&lt;h2 id="performance-improvements"&gt;Performance Improvements 🚀&lt;a class="headerlink" href="#performance-improvements" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;DataFusion continues to focus on enhancing performance, as …&lt;/p&gt;</summary><content type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;!-- see https://github.com/apache/datafusion/issues/16347 for details --&gt;
&lt;h2 id="introduction"&gt;Introduction&lt;a class="headerlink" href="#introduction" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;We are proud to announce the release of &lt;a href="https://crates.io/crates/datafusion/49.0.0"&gt;DataFusion 49.0.0&lt;/a&gt;. This blog post highlights some of
the major improvements since the release of &lt;a href="https://datafusion.apache.org/blog/2025/07/18/datafusion-48.0.0/"&gt;DataFusion 48.0.0&lt;/a&gt;. The complete list of changes is available in the &lt;a href="https://github.com/apache/datafusion/blob/branch-49/dev/changelog/49.0.0.md"&gt;changelog&lt;/a&gt;.&lt;/p&gt;
&lt;h2 id="performance-improvements"&gt;Performance Improvements 🚀&lt;a class="headerlink" href="#performance-improvements" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;DataFusion continues to focus on enhancing performance, as shown in the ClickBench and other results. &lt;/p&gt;
&lt;p&gt;&lt;img alt="ClickBench performance results over time for DataFusion" class="img-fluid" src="/blog/images/datafusion-49.0.0/performance_over_time_clickbench.png" width="100%"/&gt;&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Figure 1&lt;/strong&gt;: ClickBench performance improvements over time
Average and median normalized query execution times for ClickBench queries for each git revision. 
Query times are normalized using the ClickBench definition. Data and definitions on the 
&lt;a href="https://alamb.github.io/datafusion-benchmarking/"&gt;DataFusion Benchmarking Page&lt;/a&gt;. &lt;/p&gt;
&lt;!--
NOTE: Andrew is working on gathering these numbers

&lt;img
src="/blog/images/datafusion-49.0.0/performance_over_time_planning.png"
width="80%"
class="img-fluid"
alt="Planning benchmark performance results over time for DataFusion"
/&gt;

**Figure 2**: Planning benchmark performance improved XXX between DataFusion 48.0.1 and DataFusion 49.0.0. Chart source: TODO
--&gt;
&lt;p&gt;Here are some noteworthy optimizations added since DataFusion 48:&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Equivalence system upgrade:&lt;/strong&gt; The lower levels of the equivalence system, which is used to implement the
  optimizations described in &lt;a href="https://datafusion.apache.org/blog/2025/03/11/ordering-analysis"&gt;Using Ordering for Better Plans&lt;/a&gt;, were rewritten, leading to
  much faster planning times, especially for queries with a &lt;a href="https://github.com/apache/datafusion/pull/16217#pullrequestreview-2891941229"&gt;large number of columns&lt;/a&gt;. This change also prepares
  the way for more sophisticated sort-based optimizations in the future. (PR &lt;a href="https://github.com/apache/datafusion/pull/16217"&gt;#16217&lt;/a&gt; by &lt;a href="https://github.com/ozankabak"&gt;ozankabak&lt;/a&gt;).&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Dynamic Filters and TopK pushdown&lt;/strong&gt;&lt;/p&gt;
&lt;p&gt;DataFusion now supports dynamic filters, which are improved during query execution,
and physical filter pushdown. Together, these features improve the performance of
queries that use &lt;code&gt;LIMIT&lt;/code&gt; and &lt;code&gt;ORDER BY&lt;/code&gt; clauses, such as the following:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-sql"&gt;SELECT *
FROM data
ORDER BY timestamp DESC
LIMIT 10
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;While the query above is simple, without dynamic filtering or knowing that the data
is already sorted by &lt;code&gt;timestamp&lt;/code&gt;, a query engine must decode &lt;em&gt;all&lt;/em&gt; of the data to
find the top 10 values. With the dynamic filters system, DataFusion applies an
increasingly selective filter during query execution. It checks the &lt;strong&gt;current&lt;/strong&gt;
top 10 values of the &lt;code&gt;timestamp&lt;/code&gt; column &lt;strong&gt;before&lt;/strong&gt; opening files or reading
Parquet Row Groups and Data Pages, which can skip older data very quickly.&lt;/p&gt;
&lt;p&gt;Dynamic predicates are a common feature of advanced engines such as &lt;a href="https://docs.starburst.io/latest/admin/dynamic-filtering.html"&gt;Dynamic
Filters in Starburst&lt;/a&gt; and &lt;a href="https://www.snowflake.com/en/engineering-blog/optimizing-top-k-aggregation-snowflake/"&gt;Top-K Aggregation Optimization at Snowflake&lt;/a&gt;. The
technique drastically improves query performance (we've seen over a 1.5x
improvement for some TPC-H-style queries), especially in combination with late
materialization and columnar file formats such as Parquet. We &lt;a href="https://github.com/apache/datafusion/issues/15513"&gt;plan to write a
blog post&lt;/a&gt; explaining the details of this optimization in the future, and we expect to
use the same mechanism to implement additional optimizations such as &lt;a href="https://github.com/apache/datafusion/issues/7955"&gt;Sideways
Information Passing for joins&lt;/a&gt; (Issue
&lt;a href="https://github.com/apache/datafusion/issues/15037"&gt;#15037&lt;/a&gt; PR
&lt;a href="https://github.com/apache/datafusion/pull/15770"&gt;#15770&lt;/a&gt; by
&lt;a href="https://github.com/adriangb"&gt;adriangb&lt;/a&gt;).&lt;/p&gt;
&lt;h2 id="community-growth"&gt;Community Growth  📈&lt;a class="headerlink" href="#community-growth" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;The last few months, between &lt;code&gt;46.0.0&lt;/code&gt; and &lt;code&gt;49.0.0&lt;/code&gt;, have seen our community grow:&lt;/p&gt;
&lt;ol&gt;
&lt;li&gt;New PMC members and committers: &lt;a href="https://github.com/berkaysynnada"&gt;berkay&lt;/a&gt;, &lt;a href="https://github.com/xudong963"&gt;xudong963&lt;/a&gt; and &lt;a href="https://github.com/timsaucer"&gt;timsaucer&lt;/a&gt; joined the PMC.
   &lt;a href="https://github.com/blaginin"&gt;blaginin&lt;/a&gt;, &lt;a href="https://github.com/milenkovicm"&gt;milenkovicm&lt;/a&gt;, &lt;a href="https://github.com/adriangb"&gt;adriangb&lt;/a&gt; and &lt;a href="https://github.com/kosiew"&gt;kosiew&lt;/a&gt; joined as committers. See the &lt;a href="https://lists.apache.org/list.html?dev@datafusion.apache.org"&gt;mailing list&lt;/a&gt; for more details.&lt;/li&gt;
&lt;li&gt;In the &lt;a href="https://github.com/apache/arrow-datafusion"&gt;core DataFusion repo&lt;/a&gt; alone, we reviewed and accepted over 850 PRs from 172 different
   committers, created over 669 issues, and closed 379 of them 🚀. All changes are listed in the detailed
   &lt;a href="https://github.com/apache/datafusion/tree/main/dev/changelog"&gt;changelogs&lt;/a&gt;.&lt;/li&gt;
&lt;li&gt;DataFusion published a number of blog posts, including &lt;a href="https://datafusion.apache.org/blog/2025/04/19/user-defined-window-functions"&gt;User defined Window Functions&lt;/a&gt;, &lt;a href="https://datafusion.apache.org/blog/2025/06/15/optimizing-sql-dataframes-part-one"&gt;Optimizing SQL (and DataFrames)
   in DataFusion part 1&lt;/a&gt;, &lt;a href="https://datafusion.apache.org/blog/2025/06/15/optimizing-sql-dataframes-part-two"&gt;part 2&lt;/a&gt;, &lt;a href="https://datafusion.apache.org/blog/2025/06/30/cancellation"&gt;Using Rust async for Query Execution and Cancelling Long-Running Queries&lt;/a&gt;, and
   &lt;a href="https://datafusion.apache.org/blog/2025/07/14/user-defined-parquet-indexes/"&gt;Embedding User-Defined Indexes in Apache Parquet Files&lt;/a&gt;.&lt;/li&gt;
&lt;/ol&gt;
&lt;!--
# Unique committers
$ git shortlog -sn 46.0.0..49.0.0-rc1  .| wc -l
     172
# commits
$ git log --pretty=oneline 46.0.0..49.0.0-rc1 . | wc -l
     884


https://crates.io/crates/datafusion/49.0.0
DataFusion 49 released July 25, 2025

https://crates.io/crates/datafusion/46.0.0
DataFusion 46 released March 7, 2025

Issues created in this time: 290 open, 379 closed = 669 total
https://github.com/apache/datafusion/issues?q=is%3Aissue+created%3A2025-03-07..2025-07-25

Issues closed: 508
https://github.com/apache/arrow-datafusion/issues?q=is%3Aissue+closed%3A2025-03-07..2025-07-25

PRs merged in this time 874
https://github.com/apache/arrow-datafusion/pulls?q=is%3Apr+merged%3A2025-03-07..2025-07-25

--&gt;
&lt;h2 id="new-features"&gt;New Features ✨&lt;a class="headerlink" href="#new-features" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;h3 id="async-user-defined-functions"&gt;Async User-Defined Functions&lt;a class="headerlink" href="#async-user-defined-functions" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;It is now possible to write &lt;code&gt;async&lt;/code&gt; User-Defined Functions
(UDFs) in DataFusion that perform asynchronous
operations, such as network requests or database queries, without blocking the
execution of the query. This enables new use cases, such as
integrating with large language models (LLMs) or other external services, and we can't
wait to see what the community builds with it.&lt;/p&gt;
&lt;p&gt;See the &lt;a href="https://datafusion.apache.org/library-user-guide/functions/adding-udfs.html"&gt;documentation&lt;/a&gt; for more details and the &lt;a href="https://github.com/apache/datafusion/blob/main/datafusion-examples/examples/async_udf.rs"&gt;async UDF example&lt;/a&gt; for
working code. &lt;/p&gt;
&lt;p&gt;You could, for example, implement a function &lt;code&gt;ask_llm&lt;/code&gt; that asks a large language model
(LLM) service a question based on the content of two columns.&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-sql"&gt;SELECT * 
FROM animal a
WHERE ask_llm(a.name, 'Is this animal furry?')")
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;The implementation of an async UDF is almost identical to a normal
UDF, except that it must implement the &lt;code&gt;AsyncScalarUDFImpl&lt;/code&gt; trait in addition to &lt;code&gt;ScalarUDFImpl&lt;/code&gt; and
provide an &lt;code&gt;async&lt;/code&gt; implementation via &lt;code&gt;invoke_async_with_args&lt;/code&gt;:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-rust"&gt;#[derive(Debug)]
struct AskLLM {
    signature: Signature,
}

#[async_trait]
impl AsyncScalarUDFImpl for AskLLM {
    /// The `invoke_async_with_args` method is similar to `invoke_with_args`,
    /// but it returns a `Future` that resolves to the result.
    ///
    /// Since this signature is `async`, it can do any `async` operations, such
    /// as network requests.
    async fn invoke_async_with_args(
        &amp;amp;self,
        args: ScalarFunctionArgs,
        options: &amp;amp;ConfigOptions,
    ) -&amp;gt; Result&amp;lt;ArrayRef&amp;gt; {
        // Converts the arguments to arrays for simplicity.
        let args = ColumnarValue::values_to_arrays(&amp;amp;args.args)?;
        let [column_of_interest, question] = take_function_args(self.name(), args)?;
        let client = Client::new();

        // Make a network request to a hypothetical LLM service
        let res = client
            .post(URI)
            .headers(get_llm_headers(options))
            .json(&amp;amp;req)
            .send()
            .await?
            .json::&amp;lt;LLMResponse&amp;gt;()
            .await?;

        let results = extract_results_from_llm_response(&amp;amp;res);

        Ok(Arc::new(results))
    }
}
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;(Issue &lt;a href="https://github.com/apache/datafusion/issues/6518"&gt;#6518&lt;/a&gt;,
&lt;a href="https://github.com/apache/datafusion/pull/14837"&gt;PR #14837&lt;/a&gt; from
&lt;a href="https://github.com/goldmedal"&gt;goldmedal&lt;/a&gt; 🏆)&lt;/p&gt;
&lt;h3 id="better-cancellation-for-certain-long-running-queries"&gt;Better Cancellation for Certain Long-Running Queries&lt;a class="headerlink" href="#better-cancellation-for-certain-long-running-queries" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;In rare cases, it was previously not possible to cancel long-running queries,
leading to unresponsiveness. Other projects would likely have fixed this issue
by treating the symptom, but &lt;a href="https://github.com/pepijnve"&gt;pepijnve&lt;/a&gt; and the DataFusion community worked together to
treat the root cause. The general solution required a deep understanding of the
DataFusion execution engine, Rust &lt;code&gt;Streams&lt;/code&gt;, and the tokio cooperative
scheduling model. The &lt;a href="https://github.com/apache/datafusion/pull/16398"&gt;resulting PR&lt;/a&gt; is a model of careful
community engineering and a great example of using Rust's &lt;code&gt;async&lt;/code&gt; ecosystem
to implement complex functionality. It even resulted in a &lt;a href="https://github.com/tokio-rs/tokio/pull/7405"&gt;contribution upstream to tokio&lt;/a&gt;
(since accepted). See the &lt;a href="https://datafusion.apache.org/blog/2025/06/30/cancellation"&gt;blog post&lt;/a&gt; for more details.&lt;/p&gt;
&lt;h3 id="metadata-for-user-defined-types-such-as-variant-and-geometry"&gt;Metadata for User Defined Types such as &lt;code&gt;Variant&lt;/code&gt; and &lt;code&gt;Geometry&lt;/code&gt;&lt;a class="headerlink" href="#metadata-for-user-defined-types-such-as-variant-and-geometry" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;User-defined types have been &lt;a href="https://github.com/apache/datafusion/issues/12644"&gt;a long-requested feature&lt;/a&gt;, and this release provides
the low-level APIs to support them efficiently.&lt;/p&gt;
&lt;ol&gt;
&lt;li&gt;Metadata handling in PRs &lt;a href="https://github.com/apache/datafusion/pull/15646"&gt;#15646&lt;/a&gt; and &lt;a href="https://github.com/apache/datafusion/pull/16170"&gt;#16170&lt;/a&gt; from &lt;a href="https://github.com/timsaucer"&gt;timsaucer&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;Pushdown of filters and expressions (see "Dynamic Filters and TopK pushdown" section above)&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;We still have some work to do to fully support user-defined types, specifically
in documentation and testing, and we would
love your help in this area. If you are interested in contributing,
please see &lt;a href="https://github.com/apache/datafusion/issues/12644"&gt;issue #12644&lt;/a&gt;.&lt;/p&gt;
&lt;h3 id="parquet-modular-encryption"&gt;Parquet Modular Encryption&lt;a class="headerlink" href="#parquet-modular-encryption" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;DataFusion now supports reading and writing encrypted &lt;a href="https://parquet.apache.org/"&gt;Apache Parquet&lt;/a&gt; files with &lt;a href="https://parquet.apache.org/docs/file-format/data-pages/encryption/"&gt;modular
encryption&lt;/a&gt;. This allows users to encrypt specific columns in a Parquet file
using different keys, while still being able to read data without needing to
decrypt the entire file.&lt;/p&gt;
&lt;p&gt;Here is an example of how to configure DataFusion to read an encrypted Parquet
table with two columns, &lt;code&gt;double_field&lt;/code&gt; and &lt;code&gt;float_field&lt;/code&gt;, using modular
encryption:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-sql"&gt;CREATE EXTERNAL TABLE encrypted_parquet_table
(
double_field double,
float_field float
)
STORED AS PARQUET LOCATION 'pq/' OPTIONS (
    -- encryption
    'format.crypto.file_encryption.encrypt_footer' 'true',
    'format.crypto.file_encryption.footer_key_as_hex' '30313233343536373839303132333435',  -- b"0123456789012345"
    'format.crypto.file_encryption.column_key_as_hex::double_field' '31323334353637383930313233343530', -- b"1234567890123450"
    'format.crypto.file_encryption.column_key_as_hex::float_field' '31323334353637383930313233343531', -- b"1234567890123451"
    -- decryption
    'format.crypto.file_decryption.footer_key_as_hex' '30313233343536373839303132333435', -- b"0123456789012345"
    'format.crypto.file_decryption.column_key_as_hex::double_field' '31323334353637383930313233343530', -- b"1234567890123450"
    'format.crypto.file_decryption.column_key_as_hex::float_field' '31323334353637383930313233343531', -- b"1234567890123451"
);
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;(&lt;a href="https://github.com/apache/datafusion/issues/15216"&gt;Issue #15216&lt;/a&gt;,
&lt;a href="https://github.com/apache/datafusion/pull/16351"&gt;PR #16351&lt;/a&gt;
from &lt;a href="https://github.com/corwinjoy"&gt;corwinjoy&lt;/a&gt; and &lt;a href="https://github.com/adamreeve"&gt;adamreeve&lt;/a&gt;)&lt;/p&gt;
&lt;h3 id="support-for-within-group-for-ordered-set-aggregate-functions"&gt;Support for &lt;code&gt;WITHIN GROUP&lt;/code&gt; for Ordered-Set Aggregate Functions&lt;a class="headerlink" href="#support-for-within-group-for-ordered-set-aggregate-functions" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;DataFusion now supports the &lt;code&gt;WITHIN GROUP&lt;/code&gt; clause for &lt;a href="https://www.postgresql.org/docs/9.4/functions-aggregate.html#FUNCTIONS-ORDEREDSET-TABLE"&gt;ordered-set aggregate
functions&lt;/a&gt; such as &lt;code&gt;approx_percentile_cont&lt;/code&gt;, &lt;code&gt;percentile_cont&lt;/code&gt;, and
&lt;code&gt;percentile_disc&lt;/code&gt;, which allows users to specify the precise order.&lt;/p&gt;
&lt;p&gt;For example, the following query computes the 50th percentile for the &lt;code&gt;temperature&lt;/code&gt; column
in the &lt;code&gt;city_data&lt;/code&gt; table, ordered by &lt;code&gt;date&lt;/code&gt;:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-sql"&gt;SELECT
    percentile_disc(0.5) WITHIN GROUP (ORDER BY date) AS median_temperature
FROM city_data;
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;(Issue &lt;a href="https://github.com/apache/datafusion/issues/11732"&gt;#11732&lt;/a&gt;, 
PR &lt;a href="https://github.com/apache/datafusion/pull/13511"&gt;#13511&lt;/a&gt;,
by &lt;a href="https://github.com/Garamda"&gt;Garamda&lt;/a&gt;)&lt;/p&gt;
&lt;h3 id="compressed-spill-files"&gt;Compressed Spill Files&lt;a class="headerlink" href="#compressed-spill-files" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;DataFusion now supports compressing the files written to disk when spilling
larger-than-memory datasets while sorting and grouping. Using compression
can significantly reduce the
size of the intermediate files and improve performance when reading them back into memory.&lt;/p&gt;
&lt;p&gt;(Issue &lt;a href="https://github.com/apache/datafusion/issues/16130"&gt;#16130&lt;/a&gt;,
PR &lt;a href="https://github.com/apache/datafusion/pull/16268"&gt;#16268&lt;/a&gt;
by &lt;a href="https://github.com/ding-young"&gt;ding-young&lt;/a&gt;)&lt;/p&gt;
&lt;h3 id="support-for-regex_instr-function"&gt;Support for &lt;code&gt;REGEX_INSTR&lt;/code&gt; function&lt;a class="headerlink" href="#support-for-regex_instr-function" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;DataFusion now supports the [&lt;code&gt;REGEXP_INSTR&lt;/code&gt; function], which returns the position of a
regular expression match within a string.&lt;/p&gt;
&lt;p&gt;For example, to find the position of the first match of the regular expression
&lt;code&gt;C(.)(..)&lt;/code&gt; in the string &lt;code&gt;ABCDEF&lt;/code&gt;, you can use:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-sql"&gt;&amp;gt; SELECT regexp_instr('ABCDEF', 'C(.)(..)');
+---------------------------------------------------------------+
| regexp_instr(Utf8("ABCDEF"),Utf8("C(.)(..)"))                 |
+---------------------------------------------------------------+
| 3                                                             |
+---------------------------------------------------------------+
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;(&lt;a href="https://github.com/apache/datafusion/issues/13009"&gt;Issue #13009&lt;/a&gt;,
&lt;a href="https://github.com/apache/datafusion/pull/15928"&gt;PR #15928&lt;/a&gt;
by &lt;a href="https://github.com/nirnayroy"&gt;nirnayroy&lt;/a&gt;)&lt;/p&gt;
&lt;h2 id="upgrade-guide-and-changelog"&gt;Upgrade Guide and Changelog&lt;a class="headerlink" href="#upgrade-guide-and-changelog" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;Upgrading to 49.0.0 should be straightforward for most users. Please review the
&lt;a href="https://datafusion.apache.org/library-user-guide/upgrading.html"&gt;Upgrade Guide&lt;/a&gt;
for details on breaking changes and code snippets to help with the transition.
Recently, some users have reported success automatically upgrading DataFusion by
pairing AI tools with the upgrade guide. For a comprehensive list of all changes,
please refer to the &lt;a href="https://github.com/apache/datafusion/blob/branch-49/dev/changelog/49.0.0.md"&gt;changelog&lt;/a&gt;.&lt;/p&gt;
&lt;h2 id="about-datafusion"&gt;About DataFusion&lt;a class="headerlink" href="#about-datafusion" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;&lt;a href="https://datafusion.apache.org/"&gt;Apache DataFusion&lt;/a&gt; is an extensible query engine, written in &lt;a href="https://www.rust-lang.org/"&gt;Rust&lt;/a&gt;, that
uses &lt;a href="https://arrow.apache.org"&gt;Apache Arrow&lt;/a&gt; as its in-memory format. DataFusion is used by developers to
create new, fast, data-centric systems such as databases, dataframe libraries,
and machine learning and streaming applications. While &lt;a href="https://datafusion.apache.org/user-guide/introduction.html#project-goals"&gt;DataFusion’s primary design
goal&lt;/a&gt; is to accelerate the creation of other data-centric systems, it provides a
reasonable experience directly out of the box as a &lt;a href="https://datafusion.apache.org/user-guide/dataframe.html"&gt;dataframe library&lt;/a&gt;,
&lt;a href="https://datafusion.apache.org/python/"&gt;python library&lt;/a&gt;, and [command-line SQL tool].&lt;/p&gt;
&lt;p&gt;DataFusion's core thesis is that as a community, together we can build much more
advanced technology than any of us as individuals or companies could do alone.
Without DataFusion, highly performant vectorized query engines would remain
the domain of a few large companies and world-class research institutions.
With DataFusion, we can all build on top of a shared foundation and focus on
what makes our projects unique.&lt;/p&gt;
&lt;h2 id="how-to-get-involved"&gt;How to Get Involved&lt;a class="headerlink" href="#how-to-get-involved" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;DataFusion is not a project built or driven by a single person, company, or
foundation. Rather, our community of users and contributors works together to
build a shared technology that none of us could have built alone.&lt;/p&gt;
&lt;p&gt;If you are interested in joining us, we would love to have you. You can try out
DataFusion on some of your own data and projects and let us know how it goes,
contribute suggestions, documentation, bug reports, or a PR with documentation,
tests, or code. A list of open issues suitable for beginners is &lt;a href="https://github.com/apache/arrow-datafusion/issues?q=is%3Aissue+is%3Aopen+label%3A%22good+first+issue%22"&gt;here&lt;/a&gt;, and you
can find out how to reach us on the &lt;a href="https://datafusion.apache.org/contributor-guide/communication.html"&gt;communication doc&lt;/a&gt;.&lt;/p&gt;</content><category term="blog"/></entry><entry><title>Apache DataFusion 48.0.0 Released</title><link href="https://datafusion.apache.org/blog/2025/07/16/datafusion-48.0.0" rel="alternate"/><published>2025-07-16T00:00:00+00:00</published><updated>2025-07-16T00:00:00+00:00</updated><author><name>PMC</name></author><id>tag:datafusion.apache.org,2025-07-16:/blog/2025/07/16/datafusion-48.0.0</id><summary type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;!-- see https://github.com/apache/datafusion/issues/16347 for details --&gt;
&lt;p&gt;We’re excited to announce the release of &lt;strong&gt;Apache DataFusion 48.0.0&lt;/strong&gt;! As always, this version packs in a wide range of 
improvements and fixes. You can find the complete details in the full 
&lt;a href="https://github.com/apache/datafusion/blob/branch-48/dev/changelog/48.0.0.md"&gt;changelog&lt;/a&gt;. We’ll highlight the most
important changes below and guide you through upgrading.&lt;/p&gt;
&lt;h2 id="breaking-changes"&gt;Breaking …&lt;/h2&gt;</summary><content type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;!-- see https://github.com/apache/datafusion/issues/16347 for details --&gt;
&lt;p&gt;We’re excited to announce the release of &lt;strong&gt;Apache DataFusion 48.0.0&lt;/strong&gt;! As always, this version packs in a wide range of 
improvements and fixes. You can find the complete details in the full 
&lt;a href="https://github.com/apache/datafusion/blob/branch-48/dev/changelog/48.0.0.md"&gt;changelog&lt;/a&gt;. We’ll highlight the most
important changes below and guide you through upgrading.&lt;/p&gt;
&lt;h2 id="breaking-changes"&gt;Breaking Changes&lt;a class="headerlink" href="#breaking-changes" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;DataFusion 48.0.0 brings a few &lt;strong&gt;breaking changes&lt;/strong&gt; that may require adjustments to your code as described in
the &lt;a href="https://datafusion.apache.org/library-user-guide/upgrading.html#datafusion-48-0-0"&gt;Upgrade Guide&lt;/a&gt;. Here are the most notable ones:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;
&lt;p&gt;&lt;code&gt;datafusion.execution.collect_statistics&lt;/code&gt; defaults to &lt;code&gt;true&lt;/code&gt;: In DataFusion 48.0.0, the default value of this &lt;a href="https://datafusion.apache.org/user-guide/configs.html"&gt;configuration setting&lt;/a&gt; is now true, and DataFusion will collect and store statistics when a table is first created via &lt;code&gt;CREATE EXTERNAL TABLE&lt;/code&gt; or one of the &lt;code&gt;DataFrame::register_*&lt;/code&gt; APIs.&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;
&lt;p&gt;&lt;code&gt;Expr::Literal&lt;/code&gt; has optional metadata: The &lt;code&gt;Expr::Literal&lt;/code&gt; variant now includes optional metadata, which allows 
  for carrying through Arrow field metadata to support extension types and other uses. This means code such as&lt;/p&gt;
&lt;/li&gt;
&lt;/ul&gt;
&lt;pre&gt;&lt;code class="language-rust"&gt;match expr {
...
  Expr::Literal(scalar) =&amp;gt; ...
...
}
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;Should be updated to:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-rust"&gt;match expr {
...
  Expr::Literal(scalar, _metadata) =&amp;gt; ...
...
}
&lt;/code&gt;&lt;/pre&gt;
&lt;ul&gt;
&lt;li&gt;
&lt;p&gt;&lt;code&gt;Expr::WindowFunction&lt;/code&gt; is now Boxed: &lt;code&gt;Expr::WindowFunction&lt;/code&gt; is now a &lt;code&gt;Box&amp;lt;WindowFunction&amp;gt;&lt;/code&gt; instead of a &lt;code&gt;WindowFunction&lt;/code&gt; 
  directly. This change was made to reduce the size of &lt;code&gt;Expr&lt;/code&gt; and improve performance when planning queries 
  (see details on &lt;a href="https://github.com/apache/datafusion/pull/16207"&gt;#16207&lt;/a&gt;).&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;
&lt;p&gt;UDFs changed to use &lt;code&gt;FieldRef&lt;/code&gt; instead of &lt;code&gt;DataType&lt;/code&gt;: To support metadata handling and 
  prepare for extension types, UDF traits now use &lt;a href="https://docs.rs/arrow/latest/arrow/datatypes/type.FieldRef.html"&gt;FieldRef&lt;/a&gt; rather than a &lt;code&gt;DataType&lt;/code&gt;
  and nullability. &lt;code&gt;FieldRef&lt;/code&gt; contains the type and nullability, and additionally allows access to 
  metadata fields, which can be used for extension types.&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;
&lt;p&gt;Physical Expression return &lt;code&gt;Field&lt;/code&gt;: Similarly to UDFs, in order to prepare for extension type support the 
  &lt;a href="https://docs.rs/datafusion/latest/datafusion/physical_expr/trait.PhysicalExpr.html"&gt;PhysicalExpr&lt;/a&gt; trait has been changed to return &lt;a href="https://docs.rs/arrow/latest/arrow/datatypes/struct.Field.html"&gt;Field&lt;/a&gt; rather than &lt;code&gt;DataType&lt;/code&gt;. To upgrade structs which 
  implement &lt;code&gt;PhysicalExpr&lt;/code&gt; you need to implement the &lt;code&gt;return_field&lt;/code&gt; function. &lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;
&lt;p&gt;&lt;code&gt;FileFormat::supports_filters_pushdown&lt;/code&gt; was replaced with &lt;code&gt;FileSource::try_pushdown_filters&lt;/code&gt; to support upcoming work to push down dynamic filters and physical filter pushdown. &lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;
&lt;p&gt;&lt;code&gt;ParquetExec&lt;/code&gt;, &lt;code&gt;AvroExec&lt;/code&gt;, &lt;code&gt;CsvExec&lt;/code&gt;, &lt;code&gt;JsonExec&lt;/code&gt; removed: &lt;code&gt;ParquetExec&lt;/code&gt;, &lt;code&gt;AvroExec&lt;/code&gt;, &lt;code&gt;CsvExec&lt;/code&gt;, and &lt;code&gt;JsonExec&lt;/code&gt;
  were deprecated in DataFusion 46 and are removed in DataFusion 48.&lt;/p&gt;
&lt;/li&gt;
&lt;/ul&gt;
&lt;h2 id="performance-improvements"&gt;Performance Improvements&lt;a class="headerlink" href="#performance-improvements" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;DataFusion 48.0.0 comes with some noteworthy performance enhancements:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;
&lt;p&gt;&lt;strong&gt;Fewer unnecessary projections:&lt;/strong&gt; DataFusion now removes additional unnecessary &lt;code&gt;Projection&lt;/code&gt;s in queries. (PRs &lt;a href="https://github.com/apache/datafusion/pull/15787"&gt;#15787&lt;/a&gt;, &lt;a href="https://github.com/apache/datafusion/pull/15761"&gt;#15761&lt;/a&gt;,
  and &lt;a href="https://github.com/apache/datafusion/pull/15746"&gt;#15746&lt;/a&gt; by &lt;a href="https://github.com/xudong963"&gt;xudong963&lt;/a&gt;).&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;
&lt;p&gt;&lt;strong&gt;Accelerated string functions&lt;/strong&gt;: The &lt;code&gt;ascii&lt;/code&gt; function was optimized to significantly improve its performance
  (PR &lt;a href="https://github.com/apache/datafusion/pull/16087"&gt;#16087&lt;/a&gt; by &lt;a href="https://github.com/tlm365"&gt;tlm365&lt;/a&gt;). The &lt;code&gt;character_length&lt;/code&gt; function was optimized resulting in 
  &lt;a href="https://github.com/apache/datafusion/pull/15931#issuecomment-2848561984"&gt;up to 3x&lt;/a&gt; performance improvement (PR &lt;a href="https://github.com/apache/datafusion/pull/15931"&gt;#15931&lt;/a&gt; by &lt;a href="https://github.com/Dandandan"&gt;Dandandan&lt;/a&gt;)&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;
&lt;p&gt;&lt;strong&gt;Constant aggregate window expressions:&lt;/strong&gt; For unbounded aggregate window functions the result is the 
  same for all rows within a partition. DataFusion 48.0.0 avoids unnecessary computation for such queries, resulting in &lt;a href="https://github.com/apache/datafusion/pull/16234#issuecomment-2935960865"&gt;improved performance by 5.6x&lt;/a&gt;
  (PR &lt;a href="https://github.com/apache/datafusion/pull/16234"&gt;#16234&lt;/a&gt; by &lt;a href="https://github.com/suibianwanwank"&gt;suibianwanwank&lt;/a&gt;)&lt;/p&gt;
&lt;/li&gt;
&lt;/ul&gt;
&lt;h2 id="highlighted-new-features"&gt;Highlighted New Features&lt;a class="headerlink" href="#highlighted-new-features" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;h3 id="new-datafusion-spark-crate"&gt;New &lt;code&gt;datafusion-spark&lt;/code&gt; crate&lt;a class="headerlink" href="#new-datafusion-spark-crate" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;The DataFusion community has requested &lt;a href="https://spark.apache.org"&gt;Apache Spark&lt;/a&gt;-compatible functions for many years, but the current builtin function library is most similar to Postgresql, which leads to friction. Unfortunately, there are even functions with the same name but different signatures and/or return types in the two systems.&lt;/p&gt;
&lt;p&gt;One of the many uses of DataFusion is to enhance (e.g. &lt;a href="https://github.com/apache/datafusion-comet"&gt;Apache DataFusion Comet&lt;/a&gt;) 
or replace (e.g. &lt;a href="https://github.com/lakehq/sail"&gt;Sail&lt;/a&gt;) &lt;a href="https://spark.apache.org/"&gt;Apache Spark&lt;/a&gt;. To 
support the community requests and the use cases mentioned above, we have introduced a new
&lt;a href="https://crates.io/crates/datafusion-spark"&gt;datafusion-spark&lt;/a&gt; crate for DataFusion with spark-compatible functions so the 
community can collaborate to build this shared resource. There are several hundred functions to implement, and we are looking for help to &lt;a href="https://github.com/apache/datafusion/issues/15914"&gt;complete datafusion-spark Spark Compatible Functions&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;To register all functions in &lt;code&gt;datafusion-spark&lt;/code&gt; you can use:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-Rust"&gt;    // Create a new session context
    let mut ctx = SessionContext::new();
    // register all spark functions with the context
    datafusion_spark::register_all(&amp;amp;mut ctx)?;
    // run a query. Note the `sha2` function is now available which
    // has Spark semantics
    let df = ctx.sql("SELECT sha2('The input String', 256)").await?;
    ...
}
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;Or, to use an individual function, you can do:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-Rust"&gt;use datafusion_expr::{col, lit};
use datafusion_spark::expr_fn::sha2;
// Create the expression `sha2(my_data, 256)`
let expr = sha2(col("my_data"), lit(256));
...
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;Thanks to &lt;a href="https://github.com/shehabgamin"&gt;shehabgamin&lt;/a&gt; for the initial PR &lt;a href="https://github.com/apache/datafusion/pull/15168"&gt;#15168&lt;/a&gt; 
and many others for their help adding additional functions. Please consider 
helping &lt;a href="https://github.com/apache/datafusion/issues/15914"&gt;complete datafusion-spark Spark Compatible Functions&lt;/a&gt;. &lt;/p&gt;
&lt;h3 id="order-by-all-sql-support"&gt;&lt;code&gt;ORDER BY ALL sql&lt;/code&gt; support&lt;a class="headerlink" href="#order-by-all-sql-support" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;Inspired by &lt;a href="https://duckdb.org/docs/stable/sql/query_syntax/orderby.html#order-by-all-examples"&gt;DuckDB&lt;/a&gt;, DataFusion 48.0.0 adds support for &lt;code&gt;ORDER BY ALL&lt;/code&gt;. This allows for easy ordering of all columns in a query:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-sql"&gt;&amp;gt; set datafusion.sql_parser.dialect = 'DuckDB';
0 row(s) fetched.
&amp;gt; CREATE OR REPLACE TABLE addresses AS
    SELECT '123 Quack Blvd' AS address, 'DuckTown' AS city, '11111' AS zip
    UNION ALL
    SELECT '111 Duck Duck Goose Ln', 'DuckTown', '11111'
    UNION ALL
    SELECT '111 Duck Duck Goose Ln', 'Duck Town', '11111'
    UNION ALL
    SELECT '111 Duck Duck Goose Ln', 'Duck Town', '11111-0001';
0 row(s) fetched.
&amp;gt; SELECT * FROM addresses ORDER BY ALL;
+------------------------+-----------+------------+
| address                | city      | zip        |
+------------------------+-----------+------------+
| 111 Duck Duck Goose Ln | Duck Town | 11111      |
| 111 Duck Duck Goose Ln | Duck Town | 11111-0001 |
| 111 Duck Duck Goose Ln | DuckTown  | 11111      |
| 123 Quack Blvd         | DuckTown  | 11111      |
+------------------------+-----------+------------+
4 row(s) fetched.
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;Thanks to &lt;a href="https://github.com/PokIsemaine"&gt;PokIsemaine&lt;/a&gt; for PR &lt;a href="https://github.com/apache/datafusion/pull/15772"&gt;#15772&lt;/a&gt;&lt;/p&gt;
&lt;h3 id="ffi-support-for-aggregateudf-and-windowudf"&gt;FFI Support for &lt;code&gt;AggregateUDF&lt;/code&gt; and &lt;code&gt;WindowUDF&lt;/code&gt;&lt;a class="headerlink" href="#ffi-support-for-aggregateudf-and-windowudf" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;This improvement allows for using user defined aggregate and user defined window functions across FFI boundaries, which enables shared libraries to pass functions back and forth. This feature unlocks:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;
&lt;p&gt;Modules to provide DataFusion based FFI aggregates that can be reused in projects such as &lt;a href="https://github.com/apache/datafusion-python"&gt;datafusion-python&lt;/a&gt;&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;
&lt;p&gt;Using the same aggregate and window functions without recompiling with different DataFusion versions.&lt;/p&gt;
&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;This completes the work to add support for all UDF types to DataFusion's FFI bindings. Thanks to &lt;a href="https://github.com/timsaucer"&gt;timsaucer&lt;/a&gt;
for PRs &lt;a href="https://github.com/apache/datafusion/pull/16261"&gt;#16261&lt;/a&gt; and &lt;a href="https://github.com/apache/datafusion/pull/14775"&gt;#14775&lt;/a&gt;.&lt;/p&gt;
&lt;h3 id="reduced-size-of-expr-struct"&gt;Reduced size of &lt;code&gt;Expr&lt;/code&gt; struct&lt;a class="headerlink" href="#reduced-size-of-expr-struct" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;The &lt;a href="https://docs.rs/datafusion/latest/datafusion/logical_expr/enum.Expr.html"&gt;Expr&lt;/a&gt; struct is widely used across the DataFusion and downstream codebases. By &lt;code&gt;Box&lt;/code&gt;ing &lt;code&gt;WindowFunction&lt;/code&gt;s,  we reduced the size of &lt;code&gt;Expr&lt;/code&gt; by almost 50%, from &lt;code&gt;272&lt;/code&gt; to &lt;code&gt;144&lt;/code&gt; bytes. This reduction improved planning times between 10% and 20% and reduced memory usage. Thanks to &lt;a href="https://github.com/hendrikmakait"&gt;hendrikmakait&lt;/a&gt; for 
PR &lt;a href="https://github.com/apache/datafusion/pull/16207"&gt;#16207&lt;/a&gt;&lt;/p&gt;
&lt;h2 id="upgrade-guide-and-changelog"&gt;Upgrade Guide and Changelog&lt;a class="headerlink" href="#upgrade-guide-and-changelog" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;Upgrading to 48.0.0 should be straightforward for most users, but do review
the &lt;a href="https://datafusion.apache.org/library-user-guide/upgrading.html#datafusion-48-0-0"&gt;Upgrade Guide for DataFusion 48.0.0&lt;/a&gt; for detailed
steps and code changes. The upgrade guide covers the breaking changes mentioned above and provides code snippets to help with the
transition. For a comprehensive list of all changes, please refer to the &lt;a href="https://github.com/apache/datafusion/blob/branch-48/dev/changelog/48.0.0.md"&gt;changelog&lt;/a&gt; 
for the 48.0.0 release. The changelog enumerates every merged PR in this release, including many smaller fixes and improvements 
that we couldn’t cover in this post.&lt;/p&gt;
&lt;h2 id="get-involved"&gt;Get Involved&lt;a class="headerlink" href="#get-involved" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;Apache DataFusion is an open-source project, and we welcome involvement from anyone interested. Now is a great time to
take 48.0.0 for a spin: try it out on your workloads, and let us know if you encounter any issues or have suggestions.
You can report bugs or request features on our GitHub issue tracker, or better yet, submit a pull request. Join our
community discussions – whether you have questions, want to share how you’re using DataFusion, or are looking to
contribute, we’d love to hear from you. A list of open issues suitable for beginners
is &lt;a href="https://github.com/apache/arrow-datafusion/issues?q=is%3Aissue+is%3Aopen+label%3A%22good+first+issue%22"&gt;here&lt;/a&gt; and you
can find how to reach us on the &lt;a href="https://datafusion.apache.org/contributor-guide/communication.html"&gt;communication doc&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;Happy querying!&lt;/p&gt;</content><category term="blog"/></entry><entry><title>Embedding User-Defined Indexes in Apache Parquet Files</title><link href="https://datafusion.apache.org/blog/2025/07/14/user-defined-parquet-indexes" rel="alternate"/><published>2025-07-14T00:00:00+00:00</published><updated>2025-07-14T00:00:00+00:00</updated><author><name>Qi Zhu (Cloudera), Jigao Luo (Systems Group at TU Darmstadt), and Andrew Lamb (InfluxData)</name></author><id>tag:datafusion.apache.org,2025-07-14:/blog/2025/07/14/user-defined-parquet-indexes</id><summary type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;p&gt;It’s a common misconception that &lt;a href="https://parquet.apache.org/"&gt;Apache Parquet&lt;/a&gt; files are limited to basic Min/Max/Null Count statistics and Bloom filters, and that adding more advanced indexes requires changing the specification or creating a new file format. In fact, footer metadata and offset-based addressing already provide everything needed to embed …&lt;/p&gt;</summary><content type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;p&gt;It’s a common misconception that &lt;a href="https://parquet.apache.org/"&gt;Apache Parquet&lt;/a&gt; files are limited to basic Min/Max/Null Count statistics and Bloom filters, and that adding more advanced indexes requires changing the specification or creating a new file format. In fact, footer metadata and offset-based addressing already provide everything needed to embed user-defined index structures within Parquet files without breaking compatibility with other Parquet readers.&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Motivating Example:&lt;/strong&gt; Imagine your data has a &lt;code&gt;Nation&lt;/code&gt; column with dozens of distinct values across thousands of Parquet files. You execute:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-sql"&gt;  SELECT AVG(sales_amount)
  FROM sales
  WHERE nation = 'Singapore'
  GROUP BY year;
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;Relying on the min/max statistics from the Parquet format will be ineffective at pruning files when &lt;code&gt;Nation&lt;/code&gt; spans "Argentina" through "Zimbabwe". Instead of relying on a Bloom Filter, you may want to store a list of every distinct &lt;code&gt;Nation&lt;/code&gt; value in the file near the end. At query time, your engine will read that tiny list and skip any file that does not contain 'Singapore'. This special distinct value index can yield dramatically better file‑pruning performance for your engine, all while preserving full compatibility with standard Parquet readers.&lt;/p&gt;
&lt;p&gt;In this post, we review how indexes are stored in the Apache Parquet format, explain the mechanism for storing user-defined indexes, and finally show how to read and write a user-defined index using &lt;a href="https://datafusion.apache.org/"&gt;Apache DataFusion&lt;/a&gt;.&lt;/p&gt;
&lt;h2 id="introduction"&gt;Introduction&lt;a class="headerlink" href="#introduction" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;hr/&gt;
&lt;p&gt;Apache Parquet is a popular columnar file format with well understood and &lt;a href="https://arrow.apache.org/blog/2022/12/26/querying-parquet-with-millisecond-latency/"&gt;production grade libraries for high‑performance analytics&lt;/a&gt;. Features like efficient encodings, column pruning, and predicate pushdown work well for many common query patterns. Apache DataFusion includes a &lt;a href="https://datafusion.apache.org/blog/2025/03/20/parquet-pruning/"&gt;highly optimized Parquet implementation&lt;/a&gt; and has excellent performance in general. However, some production query patterns require more than the statistics included in the Parquet format itself&lt;sup&gt;&lt;a href="#footnote1"&gt;1&lt;/a&gt;&lt;/sup&gt;.&lt;/p&gt;
&lt;p&gt;Many systems improve query performance using &lt;em&gt;external&lt;/em&gt; indexes or other metadata in addition to Parquet. For example, Apache Iceberg's &lt;a href="https://iceberg.apache.org/docs/latest/performance/#scan-planning"&gt;Scan Planning&lt;/a&gt; uses metadata stored in separate files or an in memory cache, and the &lt;a href="https://github.com/apache/datafusion/blob/main/datafusion-examples/examples/parquet_index.rs"&gt;parquet_index.rs&lt;/a&gt; and &lt;a href="https://github.com/apache/datafusion/blob/main/datafusion-examples/examples/advanced_parquet_index.rs"&gt;advanced_parquet_index.rs&lt;/a&gt; examples in the DataFusion repository use external files for Parquet pruning (skipping).&lt;/p&gt;
&lt;p&gt;External indexes are powerful and widespread, but they have some drawbacks:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;strong&gt;Increased Cost and Operational Complexity:&lt;/strong&gt; You need additional files and systems as well as the original Parquet. &lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Synchronization Risks:&lt;/strong&gt; The external index may become out of sync with the Parquet data if you do not manage it carefully.&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;Proponents have even cited these drawbacks as justification for new file formats, such as Microsoft's &lt;a href="https://github.com/microsoft/amudai/blob/main/docs/spec/src/what_about_parquet.md"&gt;Amudai&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;However, Parquet is extensible with user-defined indexes&lt;/strong&gt;: Parquet tolerates unknown bytes within the file body and permits arbitrary key/value pairs in its footer metadata. These two features enable &lt;strong&gt;embedding&lt;/strong&gt; user-defined indexes directly in the file—no extra files, no format forks, and no compatibility breakage. &lt;/p&gt;
&lt;h2 id="parquet-file-anatomy-standard-index-structures"&gt;Parquet File Anatomy &amp;amp; Standard Index Structures&lt;a class="headerlink" href="#parquet-file-anatomy-standard-index-structures" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;hr/&gt;
&lt;p&gt;Logically, Parquet files contain row groups, each with column chunks, which in turn contain data pages. Physically, a Parquet file is a sequence of bytes with a Thrift-encoded footer metadata containing metadata about the file structure. The footer metadata includes the schema, row groups, column chunks, and other metadata required to read the file.&lt;/p&gt;
&lt;p&gt;The Parquet format includes three main types&lt;sup&gt;&lt;a href="#footnote2"&gt;2&lt;/a&gt;&lt;/sup&gt; of optional index structures:&lt;/p&gt;
&lt;ol&gt;
&lt;li&gt;
&lt;p&gt;&lt;strong&gt;&lt;a href="https://github.com/apache/parquet-format/blob/819adce0ec6aa848e56c56f20b9347f4ab50857f/src/main/thrift/parquet.thrift#L263-L266"&gt;Min/Max/Null Count Statistics&lt;/a&gt;&lt;/strong&gt; for each chunk in a row group. Engines use these to quickly skip row groups that do not match a query predicate. &lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;
&lt;p&gt;&lt;strong&gt;&lt;a href="https://parquet.apache.org/docs/file-format/pageindex/"&gt;Page Index&lt;/a&gt;&lt;/strong&gt;: Offsets, sizes, and statistics for each data page. Engines use these to quickly locate data pages without scanning all pages for a column chunk.&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;
&lt;p&gt;&lt;strong&gt;&lt;a href="https://parquet.apache.org/docs/file-format/bloomfilter/"&gt;Bloom Filters&lt;/a&gt;&lt;/strong&gt;: Data structure to quickly determine if a value is present in a column chunk without scanning any data pages. Particularly useful for equality and &lt;code&gt;IN&lt;/code&gt; predicates.&lt;/p&gt;
&lt;/li&gt;
&lt;/ol&gt;
&lt;!-- Source: https://docs.google.com/presentation/d/1aFjTLEDJyDqzFZHgcmRxecCvLKKXV2OvyEpTQFCNZPw --&gt;
&lt;p&gt;&lt;img alt="Parquet File layout with standard index structures." class="img-fluid" src="/blog/images/user-defined-parquet-indexes/standard_index_structures.png" width="80%"/&gt;&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Figure 1&lt;/strong&gt;: Parquet file layout with standard index structures (as written by arrow-rs).&lt;/p&gt;
&lt;p&gt;Only the Min/Max/Null Count Statistics are stored inline in the Parquet footer metadata. The Page Index and Bloom Filters are typically stored in the file body before the Thrift-encoded footer metadata. The locations of these index structures are recorded in the footer metadata, as shown in Figure 1. Parquet readers that do not understand these structures simply ignore them.&lt;/p&gt;
&lt;p&gt;Modern Parquet writers create these indexes automatically and provide APIs to control their generation and placement. For example, the &lt;a href="https://docs.rs/parquet/latest/parquet/"&gt;Rust Parquet Library&lt;/a&gt; provides &lt;a href="https://docs.rs/parquet/latest/parquet/file/properties/struct.WriterProperties.html"&gt;Parquet WriterProperties&lt;/a&gt;, &lt;a href="https://docs.rs/parquet/latest/parquet/file/properties/enum.EnabledStatistics.html"&gt;EnabledStatistics&lt;/a&gt;, and &lt;a href="https://docs.rs/parquet/latest/parquet/file/properties/enum.BloomFilterPosition.html"&gt;BloomFilterPosition&lt;/a&gt;.&lt;/p&gt;
&lt;h2 id="embedding-user-defined-indexes-in-parquet-files"&gt;Embedding User Defined Indexes in Parquet Files&lt;a class="headerlink" href="#embedding-user-defined-indexes-in-parquet-files" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;hr/&gt;
&lt;p&gt;Embedding user-defined indexes in Parquet files is straightforward and follows the same principles as standard index structures&lt;sup&gt;&lt;a href="#footnote6"&gt;6&lt;/a&gt;&lt;/sup&gt;:&lt;/p&gt;
&lt;ol&gt;
&lt;li&gt;
&lt;p&gt;Serialize the index into a binary format and write it into the file body before the Thrift-encoded footer metadata.&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;
&lt;p&gt;Record the index location in the footer metadata as a key/value pair, such as &lt;code&gt;"my_index_offset" -&amp;gt; "&amp;lt;byte-offset&amp;gt;"&lt;/code&gt;.&lt;/p&gt;
&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;Figure 2 shows the resulting file layout.&lt;/p&gt;
&lt;!-- Source: https://docs.google.com/presentation/d/1aFjTLEDJyDqzFZHgcmRxecCvLKKXV2OvyEpTQFCNZPw --&gt;
&lt;p&gt;&lt;img alt="Parquet File layout with custom index structures." class="img-fluid" src="/blog/images/user-defined-parquet-indexes/custom_index_structures.png" width="80%"/&gt;&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Figure 2&lt;/strong&gt;: Parquet file layout with user-defined indexes.&lt;/p&gt;
&lt;p&gt;Like standard index structures, user-defined indexes can be stored anywhere in the file body, such as after row group data or before the footer. There is no limit to the number of user-defined indexes, nor any restriction on their granularity: they can operate at the file, row group, page, or even row level. This flexibility enables a wide range of use cases, including:&lt;/p&gt;
&lt;ol&gt;
&lt;li&gt;
&lt;p&gt;Row group or page-level distinct sets: a finer-grained version of the file-level example in this blog.&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;
&lt;p&gt;&lt;a href="https://en.wikipedia.org/wiki/HyperLogLog"&gt;HyperLogLog&lt;/a&gt; sketches for distinct value estimation, addressing a common criticism&lt;sup&gt;3&lt;/sup&gt; of Parquet’s lack of cardinality estimation.&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;
&lt;p&gt;Additional zone maps (&lt;a href="https://www.vldb.org/conf/1998/p476.pdf"&gt;small materialized aggregates&lt;/a&gt;) such as precomputed &lt;code&gt;sum&lt;/code&gt;s at the column chunk or data page level for faster query execution.&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;
&lt;p&gt;Histograms or samples at the row group or column chunk level for predicate selectivity estimates.&lt;/p&gt;
&lt;/li&gt;
&lt;/ol&gt;
&lt;h2 id="example-embedding-a-user-defined-distinct-value-index-in-parquet-files"&gt;Example: Embedding a User Defined Distinct Value Index in Parquet Files&lt;a class="headerlink" href="#example-embedding-a-user-defined-distinct-value-index-in-parquet-files" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;hr/&gt;
&lt;p&gt;This section demonstrates how to embed a simple distinct value index in Parquet files and use it for file-level pruning (skipping) in DataFusion. The full example is available in the DataFusion repository at &lt;a href="https://github.com/apache/datafusion/blob/main/datafusion-examples/examples/parquet_embedded_index.rs"&gt;parquet_embedded_index.rs&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;Note that the example requires &lt;strong&gt;&lt;a href="https://crates.io/crates/parquet/55.2.0"&gt;arrow‑rs v55.2.0&lt;/a&gt;&lt;/strong&gt; or later, which includes the new “buffered write” API (&lt;a href="https://github.com/apache/arrow-rs/pull/7714"&gt;apache/arrow-rs#7714&lt;/a&gt;) to keep the internal byte count in sync after appending index bytes immediately after data pages.&lt;/p&gt;
&lt;p&gt;This example is intentionally simple for clarity, but you can adapt the same approach for any index type or data types. The high-level design is:&lt;/p&gt;
&lt;ol&gt;
&lt;li&gt;
&lt;p&gt;&lt;strong&gt;Define your index payload&lt;/strong&gt; (e.g., bitmap, Bloom filter, sketch, distinct values list, etc.).&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;
&lt;p&gt;&lt;strong&gt;Serialize your index to bytes&lt;/strong&gt; and append them into the Parquet file body before writing the footer.&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;
&lt;p&gt;&lt;strong&gt;Record the index location&lt;/strong&gt; by adding a key/value entry (e.g., &lt;code&gt;"my_index_offset" -&amp;gt; "&amp;lt;byte‑offset&amp;gt;"&lt;/code&gt;) in the Parquet footer metadata.&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;
&lt;p&gt;&lt;strong&gt;Extend DataFusion&lt;/strong&gt; with a custom &lt;code&gt;TableProvider&lt;/code&gt; (or wrap the existing Parquet provider) to use the index.&lt;/p&gt;
&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;The &lt;code&gt;TableProvider&lt;/code&gt; simply reads the footer metadata to discover the index offset, seeks to that offset and deserializes the index, and then uses the index to speed up processing (e.g., skip files, row groups, data pages, etc.).&lt;/p&gt;
&lt;p&gt;The resulting Parquet files remain fully compatible with other tools such as DuckDB and Spark, which simply ignore the unknown index bytes and key/value metadata.&lt;/p&gt;
&lt;h3 id="introduction-to-distinct-value-indexes"&gt;Introduction to Distinct Value Indexes&lt;a class="headerlink" href="#introduction-to-distinct-value-indexes" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;hr/&gt;
&lt;p&gt;A &lt;strong&gt;distinct value index&lt;/strong&gt; stores the unique values of a specific column. This type of index is effective for columns with a small number of distinct values and can be used to quickly skip files that do not match the query. These indexes are popular in several engines, such as the &lt;a href="https://clickhouse.com/docs/optimize/skipping-indexes#set"&gt;"set" Skip Index in ClickHouse&lt;/a&gt; and the &lt;a href="https://docs.influxdata.com/influxdb3/enterprise/admin/distinct-value-cache/"&gt;Distinct Value Cache&lt;/a&gt; in InfluxDB 3.0.&lt;/p&gt;
&lt;p&gt;For example, if the files contain a column named &lt;code&gt;Category&lt;/code&gt; like this:&lt;/p&gt;
&lt;table class="table"&gt;
&lt;tr&gt;
&lt;td&gt;&lt;b&gt;&lt;code&gt;Category&lt;/code&gt;&lt;/b&gt;&lt;/td&gt;
&lt;/tr&gt;
&lt;tr&gt;
&lt;td&gt;&lt;code&gt;foo&lt;/code&gt;&lt;/td&gt;
&lt;/tr&gt;
&lt;tr&gt;
&lt;td&gt;&lt;code&gt;bar&lt;/code&gt;&lt;/td&gt;
&lt;/tr&gt;
&lt;tr&gt;
&lt;td&gt;&lt;code&gt;...&lt;/code&gt;&lt;/td&gt;
&lt;/tr&gt;
&lt;tr&gt;
&lt;td&gt;&lt;code&gt;baz&lt;/code&gt;&lt;/td&gt;
&lt;/tr&gt;
&lt;tr&gt;
&lt;td&gt;&lt;code&gt;foo&lt;/code&gt;&lt;/td&gt;
&lt;/tr&gt;
&lt;/table&gt;
&lt;p&gt;The distinct value index will contain the values &lt;code&gt;foo&lt;/code&gt;, &lt;code&gt;bar&lt;/code&gt;, and &lt;code&gt;baz&lt;/code&gt;. In contrast, traditional min/max statistics would store only the minimum (&lt;code&gt;bar&lt;/code&gt;) and maximum (&lt;code&gt;foo&lt;/code&gt;) values, so a query like&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-sql"&gt;SELECT * FROM t WHERE Category = 'bas'
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;cannot skip the file using min/max values because &lt;code&gt;bas&lt;/code&gt; falls between &lt;code&gt;bar&lt;/code&gt; and &lt;code&gt;foo&lt;/code&gt; in lexicographic order, even though &lt;code&gt;bas&lt;/code&gt; does not appear in the column.&lt;/p&gt;
&lt;p&gt;This is a key benefit of a distinct value index: accurate filtering without requiring the column to be sorted, unlike min/max-based pruning which is most effective when data is ordered.&lt;/p&gt;
&lt;p&gt;While not a traditional index structure like a B-tree, the distinct value set acts as a lightweight, embedded index that enables fast pruning and is especially effective for columns with low cardinality.&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Supported Filters&lt;/strong&gt;&lt;/p&gt;
&lt;p&gt;Distinct value indexes are most effective for &lt;strong&gt;equality filters&lt;/strong&gt;, such as:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-sql"&gt;WHERE category = 'foo'
WHERE category IN ('foo', 'bar')
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;They can also help with NOT IN and anti-joins, as long as the engine can evaluate them using the list of known distinct values.&lt;/p&gt;
&lt;p&gt;However, these indexes are not suitable for range predicates (e.g., category &amp;gt; 'foo'), as they do not preserve any ordering information. For such cases, other structures such as min/max statistics or sorted data layouts may be more effective.&lt;/p&gt;
&lt;p&gt;We represent a distinct value index in Rust for our example as a simple &lt;code&gt;HashSet&amp;lt;String&amp;gt;&lt;/code&gt;:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-rust"&gt;/// An index of distinct values for a single column
#[derive(Debug, Clone)]
struct DistinctIndex {
   inner: HashSet&amp;lt;String&amp;gt;,
}
&lt;/code&gt;&lt;/pre&gt;
&lt;h3 id="file-layout-with-distinct-value-index"&gt;File Layout with Distinct Value Index&lt;a class="headerlink" href="#file-layout-with-distinct-value-index" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;hr/&gt;
&lt;p&gt;In this example, we write a distinct value index for the &lt;code&gt;Category&lt;/code&gt; column into the Parquet file body after all the data pages, and record the index location in the footer metadata. The resulting file layout looks like this:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-text"&gt;                  ┌──────────────────────┐                           
                  │┌───────────────────┐ │                           
                  ││     DataPage      │ │                           
                  │└───────────────────┘ │                           
 Standard Parquet │┌───────────────────┐ │                           
 Data Pages       ││     DataPage      │ │                           
                  │└───────────────────┘ │                           
                  │        ...           │                           
                  │┌───────────────────┐ │                           
                  ││     DataPage      │ │                           
                  │└───────────────────┘ │                           
                  │┏━━━━━━━━━━━━━━━━━━━┓ │                           
Non standard      │┃                   ┃ │                           
index (ignored by │┃Custom Binary Index┃ │                           
other Parquet     │┃ (Distinct Values) ┃◀│─ ─ ─                      
readers)          │┃                   ┃ │     │                     
                  │┗━━━━━━━━━━━━━━━━━━━┛ │                           
Standard Parquet  │┏━━━━━━━━━━━━━━━━━━━┓ │     │  key/value metadata
Page Index        │┃    Page Index     ┃ │        contains location  
                  │┗━━━━━━━━━━━━━━━━━━━┛ │     │  of special index   
                  │╔═══════════════════╗ │                           
                  │║ Parquet Footer w/ ║ │     │                     
                  │║     Metadata      ║ ┼ ─ ─                       
                  │║ (Thrift Encoded)  ║ │                           
                  │╚═══════════════════╝ │                           
                  └──────────────────────┘                           

&lt;/code&gt;&lt;/pre&gt;
&lt;h3 id="serializing-the-distinctvalue-index"&gt;Serializing the Distinct‑Value Index&lt;a class="headerlink" href="#serializing-the-distinctvalue-index" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;hr/&gt;
&lt;p&gt;The example uses a simple newline‑separated UTF‑8 format as the binary format. The code to serialize the distinct index is shown below:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-rust"&gt;/// Magic bytes to identify our custom index format
const INDEX_MAGIC: &amp;amp;[u8] = b"IDX1";

/// Serialize the distinct index to a writer as bytes
fn serialize&amp;lt;W: Write + Send&amp;gt;(
   &amp;amp;self,
   arrow_writer: &amp;amp;mut ArrowWriter&amp;lt;W&amp;gt;,
) -&amp;gt; Result&amp;lt;()&amp;gt; {
   let serialized = self
           .inner
           .iter()
           .map(|s| s.as_str())
           .collect::&amp;lt;Vec&amp;lt;_&amp;gt;&amp;gt;()
           .join("\n");
   let index_bytes = serialized.into_bytes();

   // Set the offset for the index
   let offset = arrow_writer.bytes_written();
   let index_len = index_bytes.len() as u64;

   // Write the index magic and length to the file
   arrow_writer.write_all(INDEX_MAGIC)?;
   arrow_writer.write_all(&amp;amp;index_len.to_le_bytes())?;

   // Write the index bytes
   arrow_writer.write_all(&amp;amp;index_bytes)?;

   // Append metadata about the index to the Parquet file footer metadata
   arrow_writer.append_key_value_metadata(KeyValue::new(
      "distinct_index_offset".to_string(),
      offset.to_string(),
   ));
   Ok(())
}
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;This code does the following:&lt;/p&gt;
&lt;ol&gt;
&lt;li&gt;
&lt;p&gt;Creates a newline‑separated UTF‑8 string from the distinct values.&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;
&lt;p&gt;Writes a magic header (&lt;code&gt;IDX1&lt;/code&gt;) and the length of the index.&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;
&lt;p&gt;Writes the index bytes to the file using the &lt;a href="https://docs.rs/parquet/latest/parquet/arrow/arrow_writer/struct.ArrowWriter.html"&gt;ArrowWriter&lt;/a&gt; API.&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;
&lt;p&gt;Records the index location by adding a key/value entry (&lt;code&gt;"distinct_index_offset" -&amp;gt; &amp;lt;offset&amp;gt;&lt;/code&gt;) in the Parquet footer metadata.&lt;/p&gt;
&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;Note: Use the &lt;a href="https://docs.rs/parquet/latest/parquet/arrow/arrow_writer/struct.ArrowWriter.html#method.write_all"&gt;ArrowWriter::write_all&lt;/a&gt; API to ensure the offsets in the footer metadata are correctly tracked. &lt;/p&gt;
&lt;h3 id="reading-the-index"&gt;Reading the Index&lt;a class="headerlink" href="#reading-the-index" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;hr/&gt;
&lt;p&gt;This code reads the distinct index from a Parquet file:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-rust"&gt;/// Read a `DistinctIndex` from a Parquet file
fn read_distinct_index(path: &amp;amp;Path) -&amp;gt; Result&amp;lt;DistinctIndex&amp;gt; {
    let file = File::open(path)?;

    let file_size = file.metadata()?.len();
    println!("Reading index from {} (size: {file_size})", path.display(), );

    let reader = SerializedFileReader::new(file.try_clone()?)?;
    let meta = reader.metadata().file_metadata();

    let offset = get_key_value(meta, "distinct_index_offset")
        .ok_or_else(|| ParquetError::General("Missing index offset".into()))?
        .parse::&amp;lt;u64&amp;gt;()
        .map_err(|e| ParquetError::General(e.to_string()))?;

    println!("Reading index at offset: {offset}, length");
    DistinctIndex::new_from_reader(file, offset)
}
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;This function:&lt;/p&gt;
&lt;ol&gt;
&lt;li&gt;
&lt;p&gt;Opens the Parquet footer metadata and extracts &lt;code&gt;distinct_index_offset&lt;/code&gt; from the metadata.&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;
&lt;p&gt;Calls &lt;code&gt;DistinctIndex::new_from_reader&lt;/code&gt; to read the index from the file at that offset.&lt;/p&gt;
&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;&lt;code&gt;DistinctIndex::new_from_reader&lt;/code&gt; actually reads the index as shown below:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-rust"&gt; /// Read the distinct values index from a reader at the given offset and length
 pub fn new_from_reader&amp;lt;R: Read + Seek&amp;gt;(mut reader: R, offset: u64) -&amp;gt; Result&amp;lt;DistinctIndex&amp;gt; {
     reader.seek(SeekFrom::Start(offset))?;

     let mut magic_buf = [0u8; 4];
     reader.read_exact(&amp;amp;mut magic_buf)?;
     if magic_buf != INDEX_MAGIC {
         return exec_err!("Invalid index magic number at offset {offset}");
     }

     let mut len_buf = [0u8; 8];
     reader.read_exact(&amp;amp;mut len_buf)?;
     let stored_len = u64::from_le_bytes(len_buf) as usize;

     let mut index_buf = vec![0u8; stored_len];
     reader.read_exact(&amp;amp;mut index_buf)?;

     let Ok(s) = String::from_utf8(index_buf) else {
         return exec_err!("Invalid UTF-8 in index data");
     };

     Ok(Self {
         inner: s.lines().map(|s| s.to_string()).collect(),
     })
 }
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;This code:&lt;/p&gt;
&lt;ol&gt;
&lt;li&gt;
&lt;p&gt;Seeks to the offset of the index in the file.&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;
&lt;p&gt;Reads the magic bytes and checks they match &lt;code&gt;IDX1&lt;/code&gt;.&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;
&lt;p&gt;Reads the length of the index and allocates a buffer.&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;
&lt;p&gt;Reads the index bytes, converts them to a &lt;code&gt;String&lt;/code&gt;, and splits into lines to populate the &lt;code&gt;HashSet&amp;lt;String&amp;gt;&lt;/code&gt;.&lt;/p&gt;
&lt;/li&gt;
&lt;/ol&gt;
&lt;h3 id="extending-datafusions-tableprovider"&gt;Extending DataFusion’s &lt;code&gt;TableProvider&lt;/code&gt;&lt;a class="headerlink" href="#extending-datafusions-tableprovider" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;hr/&gt;
&lt;p&gt;To use the distinct index for file-level pruning, extend DataFusion's &lt;code&gt;TableProvider&lt;/code&gt; to read the index and apply it during query execution:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-rust"&gt;impl TableProvider for DistinctIndexTable {
    /* ... */

    /// Prune files before reading: only keep files whose distinct set
    /// contains the filter value
    async fn scan(
        &amp;amp;self,
        _ctx: &amp;amp;dyn Session,
        _proj: Option&amp;lt;&amp;amp;Vec&amp;lt;usize&amp;gt;&amp;gt;,
        filters: &amp;amp;[Expr],
        _limit: Option&amp;lt;usize&amp;gt;,
    ) -&amp;gt; Result&amp;lt;Arc&amp;lt;dyn ExecutionPlan&amp;gt;&amp;gt; {
        // This example only handles filters of the form
        // `category = 'X'` where X is a string literal
        //
        // You can use `PruningPredicate` for much more general range and
        // equality analysis or write your own custom logic.
        let mut target: Option&amp;lt;&amp;amp;str&amp;gt; = None;

        if filters.len() == 1 {
            if let Expr::BinaryExpr(expr) = &amp;amp;filters[0] {
                if expr.op == Operator::Eq {
                    if let (
                        Expr::Column(c),
                        Expr::Literal(ScalarValue::Utf8(Some(v)), _),
                    ) = (&amp;amp;*expr.left, &amp;amp;*expr.right)
                    {
                        if c.name == "category" {
                            println!("Filtering for category: {v}");
                            target = Some(v);
                        }
                    }
                }
            }
        }
        // Determine which files to scan
        // files_and_index is a Vec&amp;lt;(String, DistinctIndex)&amp;gt;,
        // See the full example for how this is populated.
        let files_to_scan: Vec&amp;lt;_&amp;gt; = self
            .files_and_index
            .iter()
            .filter_map(|(f, distinct_index)| {
                // keep file if no target or target is in the distinct set
                if target.is_none() || distinct_index.contains(target?) {
                    Some(f)
                } else {
                    None
                }
            })
            .collect();

        // Build ParquetSource to actually read the files
        let url = ObjectStoreUrl::parse("file://")?;
        let source = Arc::new(ParquetSource::default().with_enable_page_index(true));
        let mut builder = FileScanConfigBuilder::new(url, self.schema.clone(), source);
        for file in files_to_scan {
            let path = self.dir.join(file);
            let len = std::fs::metadata(&amp;amp;path)?.len();
           // If the index contained information about row groups or pages,
           // you could also pass that information here to further prune
           // the data read from the file.
           let partitioned_file =
                   PartitionedFile::new(path.to_str().unwrap().to_string(), len);
           builder = builder.with_file(partitioned_file);
        }
        Ok(DataSourceExec::from_data_source(builder.build()))
    }

    /// Tell DataFusion that we can handle filters on the "category" column
    fn supports_filters_pushdown(
        &amp;amp;self,
        fs: &amp;amp;[&amp;amp;Expr],
    ) -&amp;gt; Result&amp;lt;Vec&amp;lt;TableProviderFilterPushDown&amp;gt;&amp;gt; {
        // Mark as inexact since pruning is file‑granular
        Ok(vec![TableProviderFilterPushDown::Inexact; fs.len()])
    }
}

&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;This code does the following:&lt;/p&gt;
&lt;ol&gt;
&lt;li&gt;
&lt;p&gt;Implements the &lt;code&gt;scan&lt;/code&gt; method to filter files based on the distinct index.&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;
&lt;p&gt;Checks if the filter is an equality predicate on the &lt;code&gt;category&lt;/code&gt; column.&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;
&lt;p&gt;If the target value is specified, checks if the distinct index contains that value.&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;
&lt;p&gt;Builds a &lt;code&gt;FileScanConfig&lt;/code&gt; with only the files that match the filter.&lt;/p&gt;
&lt;/li&gt;
&lt;/ol&gt;
&lt;h3 id="putting-it-all-together"&gt;Putting It All Together&lt;a class="headerlink" href="#putting-it-all-together" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;To use the distinct index in a DataFusion query, write sample Parquet files with the embedded index, register the &lt;code&gt;DistinctIndexTable&lt;/code&gt; provider, and run a query with a predicate that can be optimized by the index as shown below.&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-rust"&gt;// Write sample files with embedded indexes
tmp_dir.iter().for_each(|(name, vals)| {
    write_file_with_index(&amp;amp;dir.join(name), vals).unwrap();
});

// Register provider and query
let provider = Arc::new(DistinctIndexTable::try_new(dir, schema.clone())?);
ctx.register_table("t", provider)?;

// Only files containing 'foo' will be scanned
let df = ctx.sql("SELECT * FROM t WHERE category = 'foo'").await?;
df.show().await?;
&lt;/code&gt;&lt;/pre&gt;
&lt;h3 id="verifying-compatibility-with-duckdb"&gt;Verifying Compatibility with DuckDB&lt;a class="headerlink" href="#verifying-compatibility-with-duckdb" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;hr/&gt;
&lt;p&gt;Even with extra bytes and unknown metadata keys, standard Parquet readers ignore the index. You can verify this using another system such as DuckDB to read the Parquet created in the example. DuckDB will read the files without any issues, ignoring the custom index and unknown footer metadata.&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-sql"&gt;SELECT * FROM read_parquet('/tmp/parquet_index_data/*');
┌──────────┐
│ category │
│ varchar  │
├──────────┤
│ foo      │
│ bar      │
│ foo      │
│ baz      │
│ qux      │
│ foo      │
│ quux     │
│ quux     │
└──────────┘
&lt;/code&gt;&lt;/pre&gt;
&lt;h2 id="conclusion"&gt;Conclusion&lt;a class="headerlink" href="#conclusion" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;In this post, we explained how index structures are stored in Apache Parquet, how to embed user-defined indexes without changing the format, and how to use user-defined indexes to speed up query processing.&lt;/p&gt;
&lt;p&gt;Parquet-based systems can achieve significant performance improvements for almost any query pattern while still retaining broad compatibility, using user-defined embedded indexes, external indexes&lt;sup&gt;&lt;a href="#footnote4"&gt;4&lt;/a&gt;&lt;/sup&gt; and rewriting files optimized for specific queries&lt;sup&gt;&lt;a href="#footnote5"&gt;5&lt;/a&gt;&lt;/sup&gt;. System designers can choose among the available options to make the appropriate trade-offs between operational complexity, performance, file size, and cost for their specific use cases.&lt;/p&gt;
&lt;p&gt;We hope this post inspires you to explore custom indexes in Parquet files, rather than proposing new file formats and reimplementing existing features. The DataFusion community is excited to see how you use this feature in your projects!&lt;/p&gt;
&lt;h2 id="about-the-authors"&gt;About the Authors&lt;a class="headerlink" href="#about-the-authors" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;&lt;a href="https://www.linkedin.com/in/qi-zhu-862330119/"&gt;Qi Zhu&lt;/a&gt; is a Senior Engineer at &lt;a href="https://www.cloudera.com/"&gt;Cloudera&lt;/a&gt;, an active contributor to &lt;a href="https://datafusion.apache.org/"&gt;Apache DataFusion&lt;/a&gt; and &lt;a href="https://arrow.apache.org/"&gt;Apache Arrow&lt;/a&gt;, a committer on &lt;a href="https://hadoop.apache.org/"&gt;Apache Hadoop&lt;/a&gt; and &lt;a href="https://yunikorn.apache.org/"&gt;Apache YuniKorn&lt;/a&gt;. He has extensive experience in distributed systems, scheduling, and large-scale computing.&lt;/p&gt;
&lt;p&gt;&lt;a href="https://www.linkedin.com/in/jigao-luo/"&gt;Jigao Luo&lt;/a&gt; is a 1.5-year PhD student at
&lt;a href="https://tuda.systems"&gt;Systems Group @ TU Darmstadt&lt;/a&gt;. Regarding Parquet, he is an external 
contributor to &lt;a href="https://github.com/rapidsai/cudf"&gt;NVIDIA RAPIDS cuDF&lt;/a&gt;, focusing on the GPU Parquet reader.&lt;/p&gt;
&lt;p&gt;&lt;a href="https://www.linkedin.com/in/andrewalamb/"&gt;Andrew Lamb&lt;/a&gt; is a Staff Engineer at
&lt;a href="https://www.influxdata.com/"&gt;InfluxData&lt;/a&gt;, and a member of the &lt;a href="https://datafusion.apache.org/"&gt;Apache
DataFusion&lt;/a&gt; and &lt;a href="https://arrow.apache.org/"&gt;Apache Arrow&lt;/a&gt; PMCs. He has been working on
Databases and related systems more than 20 years.&lt;/p&gt;
&lt;h2 id="about-datafusion"&gt;About DataFusion&lt;a class="headerlink" href="#about-datafusion" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;&lt;a href="https://datafusion.apache.org/"&gt;Apache DataFusion&lt;/a&gt; is an extensible query engine toolkit, written
in Rust, that uses &lt;a href="https://arrow.apache.org/"&gt;Apache Arrow&lt;/a&gt; as its in-memory format. DataFusion and
similar technology are part of the next generation “Deconstructed Database”
architectures, where new systems are built on a foundation of fast, modular
components, rather than as a single tightly integrated system.&lt;/p&gt;
&lt;p&gt;The &lt;a href="https://datafusion.apache.org/contributor-guide/communication.html"&gt;DataFusion community&lt;/a&gt; is always looking for new contributors to help
improve the project. If you are interested in learning more about how query
execution works, help document or improve the DataFusion codebase, or just try
it out, we would love for you to join us.&lt;/p&gt;
&lt;h3 id="footnotes"&gt;Footnotes&lt;a class="headerlink" href="#footnotes" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;&lt;a id="footnote1"&gt;&lt;/a&gt;&lt;code&gt;1&lt;/code&gt;: A commonly cited example is highly selective predicates (e.g. &lt;code&gt;category = 'foo'&lt;/code&gt;) but for which the built in BloomFilters are not sufficient.&lt;/p&gt;
&lt;p&gt;&lt;a id="footnote2"&gt;&lt;/a&gt;&lt;code&gt;2&lt;/code&gt;: There are other index structures, but they are either 1) not widely supported (such as statistics in the page headers) or 2) not yet widely used in practice at the time of this writing (such as &lt;a href="https://github.com/apache/parquet-format/blob/819adce0ec6aa848e56c56f20b9347f4ab50857f/src/main/thrift/parquet.thrift#L256"&gt;GeospatialStatistics&lt;/a&gt; and &lt;a href="https://github.com/apache/parquet-format/blob/819adce0ec6aa848e56c56f20b9347f4ab50857f/src/main/thrift/parquet.thrift#L194-L202"&gt;SizeStatistics&lt;/a&gt;).&lt;/p&gt;
&lt;p&gt;&lt;a id="footnote3"&gt;&lt;/a&gt;&lt;code&gt;3&lt;/code&gt;: &lt;a href="https://dl.gi.de/items/2a8571f8-0ef2-481c-8ee9-05f82ee258c8"&gt;Seamless Integration of Parquet Files into Data Processing. / Rey, Alice; Freitag, Michael; Neumann, Thomas. / BTW 2023&lt;/a&gt;&lt;/p&gt;
&lt;p&gt;&lt;a id="footnote4"&gt;&lt;/a&gt;&lt;code&gt;4&lt;/code&gt;: For more information about external indexes, see &lt;a href="https://www.youtube.com/watch?v=74YsJT1-Rdk"&gt;this talk&lt;/a&gt; and the &lt;a href="https://github.com/apache/datafusion/blob/main/datafusion-examples/examples/parquet_index.rs"&gt;parquet_index.rs&lt;/a&gt; and &lt;a href="https://github.com/apache/datafusion/blob/main/datafusion-examples/examples/advanced_parquet_index.rs"&gt;advanced_parquet_index.rs&lt;/a&gt; examples in the DataFusion repository.&lt;/p&gt;
&lt;p&gt;&lt;a id="footnote5"&gt;&lt;/a&gt;&lt;code&gt;5&lt;/code&gt;: For information about rewriting files to optimize for specific queries, such as resorting, repartitioning, and tuning data page and row group sizes, see &lt;a href="https://github.com/XiangpengHao/liquid-cache/issues/227"&gt;XiangpengHao/liquid‑cache#227&lt;/a&gt; and the conversation between &lt;a href="https://github.com/JigaoLuo"&gt;JigaoLuo&lt;/a&gt; and &lt;a href="https://github.com/XiangpengHao"&gt;XiangpengHao&lt;/a&gt; for details. We hope to make a future post about this topic.&lt;/p&gt;
&lt;p&gt;&lt;a id="footnote6"&gt;&lt;/a&gt;&lt;code&gt;6&lt;/code&gt;: An index can also be stored inline in the key-value metadata. This approach is simple to implement and ensures the index is available once the footer is read, without additional I/O. However, it requires the index to be serialized as a UTF-8 string, which may be less efficient and increases the size of the footer metadata, impacting all Parquet readers, even those that ignore the index.&lt;/p&gt;</content><category term="blog"/></entry><entry><title>Apache DataFusion 47.0.0 Released</title><link href="https://datafusion.apache.org/blog/2025/07/11/datafusion-47.0.0" rel="alternate"/><published>2025-07-11T00:00:00+00:00</published><updated>2025-07-11T00:00:00+00:00</updated><author><name>PMC</name></author><id>tag:datafusion.apache.org,2025-07-11:/blog/2025/07/11/datafusion-47.0.0</id><summary type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;!-- see https://github.com/apache/datafusion/issues/16347 for details --&gt;
&lt;p&gt;We’re excited to announce the release of &lt;strong&gt;Apache DataFusion 47.0.0&lt;/strong&gt;! This new version represents a significant
milestone for the project, packing in a wide range of improvements and fixes. You can find the complete details in the
full &lt;a href="https://github.com/apache/datafusion/blob/branch-47/dev/changelog/47.0.0.md"&gt;changelog&lt;/a&gt;. We’ll highlight the most
important changes below …&lt;/p&gt;</summary><content type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;!-- see https://github.com/apache/datafusion/issues/16347 for details --&gt;
&lt;p&gt;We’re excited to announce the release of &lt;strong&gt;Apache DataFusion 47.0.0&lt;/strong&gt;! This new version represents a significant
milestone for the project, packing in a wide range of improvements and fixes. You can find the complete details in the
full &lt;a href="https://github.com/apache/datafusion/blob/branch-47/dev/changelog/47.0.0.md"&gt;changelog&lt;/a&gt;. We’ll highlight the most
important changes below and guide you through upgrading.&lt;/p&gt;
&lt;p&gt;Note that DataFusion 47.0.0 was released in April 2025, but we are only now publishing the blog post due to 
limited bandwidth in the DataFusion community. We apologize for the delay and encourage you to come help us
accelerate the next release and announcements 
by &lt;a href="https://datafusion.apache.org/contributor-guide/communication.html"&gt;joining the community&lt;/a&gt;  🎣.&lt;/p&gt;
&lt;h2 id="breaking-changes"&gt;Breaking Changes&lt;a class="headerlink" href="#breaking-changes" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;DataFusion 47.0.0 brings a few &lt;strong&gt;breaking changes&lt;/strong&gt; that may require adjustments to your code as described in
the &lt;a href="https://datafusion.apache.org/library-user-guide/upgrading.html#datafusion-47-0-0"&gt;Upgrade Guide&lt;/a&gt;. Here are some notable ones:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;a href="https://github.com/apache/datafusion/pull/15466"&gt;Upgrades to arrow-rs and arrow-parquet 55.0.0 and object_store 0.12.0&lt;/a&gt;:
  Several APIs changed in the underlying &lt;code&gt;arrow&lt;/code&gt;, &lt;code&gt;parquet&lt;/code&gt; and &lt;code&gt;object_store&lt;/code&gt; libraries to use a &lt;code&gt;u64&lt;/code&gt; instead of usize to better support
  WASM. This requires converting from &lt;code&gt;usize&lt;/code&gt; to &lt;code&gt;u64&lt;/code&gt; occasionally as well as changes to ObjectStore implementations such as&lt;/li&gt;
&lt;/ul&gt;
&lt;pre&gt;&lt;code class="language-Rust"&gt;impl ObjectStore {
    ...

    // The range is now a u64 instead of usize
    async fn get_range(&amp;amp;self, location: &amp;amp;Path, range: Range&amp;lt;u64&amp;gt;) -&amp;gt; ObjectStoreResult&amp;lt;Bytes&amp;gt; {
        self.inner.get_range(location, range).await
    }

    ...

    // the lifetime is now 'static instead of '_ (meaning the captured closure can't contain references)
    // (this also applies to list_with_offset)
    fn list(&amp;amp;self, prefix: Option&amp;lt;&amp;amp;Path&amp;gt;) -&amp;gt; BoxStream&amp;lt;'static, ObjectStoreResult&amp;lt;ObjectMeta&amp;gt;&amp;gt; {
        self.inner.list(prefix)
    }
}
&lt;/code&gt;&lt;/pre&gt;
&lt;ul&gt;
&lt;li&gt;&lt;a href="https://github.com/apache/datafusion/issues/14914"&gt;DisplayFormatType::TreeRender&lt;/a&gt;:
  Implementations of &lt;code&gt;ExecutionPlan&lt;/code&gt; must also provide a description in the &lt;code&gt;DisplayFormatType::TreeRender&lt;/code&gt; format to
  provide support for the new &lt;a href="https://datafusion.apache.org/user-guide/sql/explain.html#tree-format-default"&gt;tree style explains&lt;/a&gt;.
  This can be the same as the existing &lt;code&gt;DisplayFormatType::Default&lt;/code&gt;.&lt;/li&gt;
&lt;/ul&gt;
&lt;h2 id="performance-improvements"&gt;Performance Improvements&lt;a class="headerlink" href="#performance-improvements" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;DataFusion 47.0.0 comes with numerous performance enhancements across the board. Here are some of the noteworthy
optimizations in this release:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;
&lt;p&gt;&lt;strong&gt;&lt;code&gt;FIRST_VALUE&lt;/code&gt; and &lt;code&gt;LAST_VALUE&lt;/code&gt;:&lt;/strong&gt; &lt;code&gt;FIRST_VALUE&lt;/code&gt; and &lt;code&gt;LAST_VALUE&lt;/code&gt; functions execute much faster for data with high cardinality such as those with many groups or partitions. DataFusion 47.0.0 executes the following in &lt;strong&gt;7 seconds&lt;/strong&gt; compared to &lt;strong&gt;36 seconds&lt;/strong&gt; in DataFusion 46.0.0: &lt;code&gt;select id2, id4, first_value(v1 order by id2, id4) as r2 from '~/h2o_100m.parquet' group by id2, id4&lt;/code&gt; (h2o.ai dataset). (PR's &lt;a href="https://github.com/apache/datafusion/pull/15266"&gt;#15266&lt;/a&gt;
  and &lt;a href="https://github.com/apache/datafusion/pull/15542"&gt;#15542&lt;/a&gt; by &lt;a href="https://github.com/UBarney"&gt;UBarney&lt;/a&gt;).&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;
&lt;p&gt;&lt;strong&gt;&lt;code&gt;MIN&lt;/code&gt;, &lt;code&gt;MAX&lt;/code&gt; and &lt;code&gt;AVG&lt;/code&gt; for Durations:&lt;/strong&gt;  DataFusion executes aggregate queries up to 2.5x faster when they include &lt;code&gt;MIN&lt;/code&gt;, &lt;code&gt;MAX&lt;/code&gt; and &lt;code&gt;AVG&lt;/code&gt; on &lt;code&gt;Duration&lt;/code&gt; columns. 
  (PRs &lt;a href="https://github.com/apache/datafusion/pull/15322"&gt;#15322&lt;/a&gt; and &lt;a href="https://github.com/apache/datafusion/pull/15748"&gt;#15748&lt;/a&gt;
  by &lt;a href="https://github.com/shruti2522"&gt;shruti2522&lt;/a&gt;).&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;
&lt;p&gt;&lt;strong&gt;Short circuit evaluation for &lt;code&gt;AND&lt;/code&gt; and &lt;code&gt;OR&lt;/code&gt;:&lt;/strong&gt; DataFusion now eagerly skips the evaluation of
  the right operand if the left is known to be false (&lt;code&gt;AND&lt;/code&gt;) or true (&lt;code&gt;OR&lt;/code&gt;) in certain cases. For complex predicates, such as those with many &lt;code&gt;LIKE&lt;/code&gt; or &lt;code&gt;CASE&lt;/code&gt; expressions, this optimization results in
  &lt;a href="https://github.com/apache/datafusion/issues/11212#issuecomment-2753584617"&gt;significant performance improvements&lt;/a&gt; (up to 100x in extreme cases).
  (PRs &lt;a href="https://github.com/apache/datafusion/pull/15462"&gt;#15462&lt;/a&gt; and &lt;a href="https://github.com/apache/datafusion/pull/15694"&gt;#15694&lt;/a&gt;
  by &lt;a href="https://github.com/acking-you"&gt;acking-you&lt;/a&gt;).&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;
&lt;p&gt;&lt;strong&gt;TopK optimization for partially sorted input:&lt;/strong&gt; Previous versions of DataFusion implemented early termination
  optimization (TopK) for fully sorted data. DataFusion 47.0.0 extends the optimization for partially sorted data, which is common in many real-world datasets, such as time-series data sorted by day but not within each day. 
  (PR &lt;a href="https://github.com/apache/datafusion/pull/15563"&gt;#15563&lt;/a&gt; by &lt;a href="https://github.com/geoffreyclaude"&gt;geoffreyclaude&lt;/a&gt;).&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;
&lt;p&gt;&lt;strong&gt;Disable re-validation of spilled files:&lt;/strong&gt; DataFusion no longer does unnecessary re-validation of temporary spill files. The validation is unnecessary and expensive as the data is known to be valid when it was written out
  (PR &lt;a href="https://github.com/apache/datafusion/pull/15454"&gt;#15454&lt;/a&gt; by &lt;a href="https://github.com/zebsme"&gt;zebsme&lt;/a&gt;).&lt;/p&gt;
&lt;/li&gt;
&lt;/ul&gt;
&lt;h2 id="highlighted-new-features"&gt;Highlighted New Features&lt;a class="headerlink" href="#highlighted-new-features" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;h3 id="tree-style-explains"&gt;Tree style explains&lt;a class="headerlink" href="#tree-style-explains" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;In previous releases the &lt;a href="https://datafusion.apache.org/user-guide/sql/explain.html"&gt;EXPLAIN statement&lt;/a&gt; results in a formatted table
which is succinct and contains important details for implementers, but was often hard to read
especially with queries that included joins or unions having multiple children.&lt;/p&gt;
&lt;p&gt;DataFusion 47.0.0 includes the new &lt;code&gt;EXPLAIN FORMAT TREE&lt;/code&gt; (default in
&lt;code&gt;datafusion-cli&lt;/code&gt;) rendered in a visual tree style that is much easier to quickly
understand.&lt;/p&gt;
&lt;!-- SQL setup 
create table t1(ti int) as values (1), (2), (3);
create table t2(ti int) as values (1), (2), (3);
--&gt;
&lt;p&gt;Example of the new explain output:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-sql"&gt;&amp;gt; explain select * from t1 inner join t2 on t1.ti=t2.ti;
+---------------+------------------------------------------------------------+
| plan_type     | plan                                                       |
+---------------+------------------------------------------------------------+
| physical_plan | ┌───────────────────────────┐                              |
|               | │    CoalesceBatchesExec    │                              |
|               | │    --------------------   │                              |
|               | │     target_batch_size:    │                              |
|               | │            8192           │                              |
|               | └─────────────┬─────────────┘                              |
|               | ┌─────────────┴─────────────┐                              |
|               | │        HashJoinExec       │                              |
|               | │    --------------------   ├──────────────┐               |
|               | │       on: (ti = ti)       │              │               |
|               | └─────────────┬─────────────┘              │               |
|               | ┌─────────────┴─────────────┐┌─────────────┴─────────────┐ |
|               | │       DataSourceExec      ││       DataSourceExec      │ |
|               | │    --------------------   ││    --------------------   │ |
|               | │         bytes: 112        ││         bytes: 112        │ |
|               | │       format: memory      ││       format: memory      │ |
|               | │          rows: 1          ││          rows: 1          │ |
|               | └───────────────────────────┘└───────────────────────────┘ |
|               |                                                            |
+---------------+------------------------------------------------------------+
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;Example of the &lt;code&gt;EXPLAIN FORMAT INDENT&lt;/code&gt; output for the same query&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-sql"&gt;&amp;gt; explain format indent select * from t1 inner join t2 on t1.ti=t2.ti;
+---------------+----------------------------------------------------------------------+
| plan_type     | plan                                                                 |
+---------------+----------------------------------------------------------------------+
| logical_plan  | Inner Join: t1.ti = t2.ti                                            |
|               |   TableScan: t1 projection=[ti]                                      |
|               |   TableScan: t2 projection=[ti]                                      |
| physical_plan | CoalesceBatchesExec: target_batch_size=8192                          |
|               |   HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ti@0, ti@0)] |
|               |     DataSourceExec: partitions=1, partition_sizes=[1]                |
|               |     DataSourceExec: partitions=1, partition_sizes=[1]                |
|               |                                                                      |
+---------------+----------------------------------------------------------------------+
2 row(s) fetched.
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;Thanks to &lt;a href="https://github.com/irenjj"&gt;irenjj&lt;/a&gt; for the initial work in PR &lt;a href="https://github.com/apache/datafusion/pull/14677"&gt;#14677&lt;/a&gt;
and many others for completing the &lt;a href="https://github.com/apache/datafusion/issues/14914"&gt;followup epic&lt;/a&gt;&lt;/p&gt;
&lt;h3 id="sql-varchar-defaults-to-utf8view"&gt;SQL &lt;code&gt;VARCHAR&lt;/code&gt; defaults to Utf8View&lt;a class="headerlink" href="#sql-varchar-defaults-to-utf8view" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;In previous releases when a column was created in SQL the column would be mapped to the &lt;a href="https://docs.rs/arrow/latest/arrow/datatypes/enum.DataType.html#variant.Utf8"&gt;Utf8 Arrow data type&lt;/a&gt;. In this release
the SQL &lt;code&gt;varchar&lt;/code&gt; columns will be mapped to the &lt;a href="https://docs.rs/arrow/latest/arrow/datatypes/enum.DataType.html#variant.Utf8View"&gt;Utf8View arrow data type&lt;/a&gt; by default, which is a more efficient representation of UTF-8 strings in Arrow.&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-sql"&gt;create table foo(x varchar);
0 row(s) fetched.

&amp;gt; describe foo;
+-------------+-----------+-------------+
| column_name | data_type | is_nullable |
+-------------+-----------+-------------+
| x           | Utf8View  | YES         |
+-------------+-----------+-------------+
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;Previous versions of DataFusion used &lt;code&gt;Utf8View&lt;/code&gt; when reading parquet files and it is faster in most cases.&lt;/p&gt;
&lt;p&gt;Thanks to &lt;a href="https://github.com/zhuqi-lucas"&gt;zhuqi-lucas&lt;/a&gt; for PR &lt;a href="https://github.com/apache/datafusion/pull/15104"&gt;#15104&lt;/a&gt;&lt;/p&gt;
&lt;h3 id="context-propagation-in-spawned-tasks-for-tracing-logging-etc"&gt;Context propagation in spawned tasks (for tracing, logging, etc.)&lt;a class="headerlink" href="#context-propagation-in-spawned-tasks-for-tracing-logging-etc" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;This release introduces an API for propagating user-defined context (such as tracing spans,
logging, or metrics) across thread boundaries without depending on any specific instrumentation library.
You can use the &lt;a href="https://docs.rs/datafusion/latest/datafusion/common/runtime/trait.JoinSetTracer.html"&gt;JoinSetTracer&lt;/a&gt; API to instrument DataFusion plans with your own tracing or logging libraries, or
use pre-integrated community crates such as the &lt;a href="https://github.com/datafusion-contrib/datafusion-tracing"&gt;datafusion-tracing&lt;/a&gt; crate.&lt;/p&gt;
&lt;div class="text-center"&gt;
&lt;a href="https://github.com/datafusion-contrib/datafusion-tracing"&gt;
&lt;img alt="DataFusion telemetry project logo" class="img-fluid" src="/blog/images/datafusion-47.0.0/datafusion-telemetry.png" width="50%"/&gt;
&lt;/a&gt;
&lt;/div&gt;
&lt;p&gt;Previously, tasks spawned on new threads — such as those performing
repartitioning or Parquet file reads — could lose thread-local context, which is
often used in instrumentation libraries. A full example of how to use this new
API is available in the &lt;a href="https://github.com/apache/datafusion/blob/main/datafusion-examples/examples/tracing.rs"&gt;DataFusion examples&lt;/a&gt;, and a simple example is shown below.&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-Rust"&gt;/// Models a simple tracer. Calling `in_current_span()` and `in_scope()` saves thread-specific state
/// for the current span and must be called at the start of each new task or thread.
struct SpanTracer;

/// Implements the `JoinSetTracer` trait so we can inject instrumentation
/// for both async futures and blocking closures.
impl JoinSetTracer for SpanTracer {
    /// Instruments a boxed future to run in the current span. The future's
    /// return type is erased to `Box&amp;lt;dyn Any + Send&amp;gt;`, which we simply
    /// run inside the `Span::current()` context.
    fn trace_future(
        &amp;amp;self,
        fut: BoxFuture&amp;lt;'static, Box&amp;lt;dyn Any + Send&amp;gt;&amp;gt;,
    ) -&amp;gt; BoxFuture&amp;lt;'static, Box&amp;lt;dyn Any + Send&amp;gt;&amp;gt; {
        // Ensures any thread-local context is set in this future 
        fut.in_current_span().boxed()
    }

    /// Instruments a boxed blocking closure by running it inside the
    /// `Span::current()` context.
    fn trace_block(
        &amp;amp;self,
        f: Box&amp;lt;dyn FnOnce() -&amp;gt; Box&amp;lt;dyn Any + Send&amp;gt; + Send&amp;gt;,
    ) -&amp;gt; Box&amp;lt;dyn FnOnce() -&amp;gt; Box&amp;lt;dyn Any + Send&amp;gt; + Send&amp;gt; {
        let span = Span::current();
        // Ensures any thread-local context is set for this closure
        Box::new(move || span.in_scope(f))
    }
}

...
set_join_set_tracer(&amp;amp;SpanTracer).expect("Failed to set tracer");
...
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;Thanks to &lt;a href="https://github.com/geoffreyclaude"&gt;geoffreyclaude&lt;/a&gt; for PR &lt;a href="https://github.com/apache/datafusion/issues/14914"&gt;#14914&lt;/a&gt;&lt;/p&gt;
&lt;h2 id="upgrade-guide-and-changelog"&gt;Upgrade Guide and Changelog&lt;a class="headerlink" href="#upgrade-guide-and-changelog" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;Upgrading to 47.0.0 should be straightforward for most users, but do review
the &lt;a href="https://datafusion.apache.org/library-user-guide/upgrading.html#datafusion-47-0-0"&gt;Upgrade Guide for DataFusion 47.0.0&lt;/a&gt; for detailed
steps and code changes. The upgrade guide covers the breaking changes mentioned above and provides code snippets to help with the
transition. For a comprehensive list of all changes, please refer to the &lt;a href="https://github.com/apache/datafusion/blob/branch-47/dev/changelog/47.0.0.md"&gt;changelog&lt;/a&gt; for 47.0.0. The changelog
enumerates every merged PR in this release, including many smaller fixes and improvements that we couldn’t cover in this post.&lt;/p&gt;
&lt;h2 id="get-involved"&gt;Get Involved&lt;a class="headerlink" href="#get-involved" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;Apache DataFusion is an open-source project, and we welcome involvement from anyone interested. Now is a great time to
take 47.0.0 for a spin: try it out on your workloads, and let us know if you encounter any issues or have suggestions.
You can report bugs or request features on our GitHub issue tracker, or better yet, submit a pull request. Join our
community discussions – whether you have questions, want to share how you’re using DataFusion, or are looking to
contribute, we’d love to hear from you. A list of open issues suitable for beginners
is &lt;a href="https://github.com/apache/arrow-datafusion/issues?q=is%3Aissue+is%3Aopen+label%3A%22good+first+issue%22"&gt;here&lt;/a&gt; and you
can find how to reach us on the &lt;a href="https://datafusion.apache.org/contributor-guide/communication.html"&gt;communication doc&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;Happy querying!&lt;/p&gt;</content><category term="blog"/></entry><entry><title>Apache DataFusion Comet 0.9.0 Release</title><link href="https://datafusion.apache.org/blog/2025/07/01/datafusion-comet-0.9.0" rel="alternate"/><published>2025-07-01T00:00:00+00:00</published><updated>2025-07-01T00:00:00+00:00</updated><author><name>pmc</name></author><id>tag:datafusion.apache.org,2025-07-01:/blog/2025/07/01/datafusion-comet-0.9.0</id><summary type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;p&gt;The Apache DataFusion PMC is pleased to announce version 0.9.0 of the &lt;a href="https://datafusion.apache.org/comet/"&gt;Comet&lt;/a&gt; subproject.&lt;/p&gt;
&lt;p&gt;Comet is an accelerator for Apache Spark that translates Spark physical plans to DataFusion physical plans for
improved performance and efficiency without requiring any code changes.&lt;/p&gt;
&lt;p&gt;This release covers approximately ten weeks of development …&lt;/p&gt;</summary><content type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;p&gt;The Apache DataFusion PMC is pleased to announce version 0.9.0 of the &lt;a href="https://datafusion.apache.org/comet/"&gt;Comet&lt;/a&gt; subproject.&lt;/p&gt;
&lt;p&gt;Comet is an accelerator for Apache Spark that translates Spark physical plans to DataFusion physical plans for
improved performance and efficiency without requiring any code changes.&lt;/p&gt;
&lt;p&gt;This release covers approximately ten weeks of development work and is the result of merging 139 PRs from 24
contributors. See the &lt;a href="https://github.com/apache/datafusion-comet/blob/main/dev/changelog/0.9.0.md"&gt;change log&lt;/a&gt; for more information.&lt;/p&gt;
&lt;h2 id="release-highlights"&gt;Release Highlights&lt;a class="headerlink" href="#release-highlights" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;h3 id="complex-type-support-in-parquet-scans"&gt;Complex Type Support in Parquet Scans&lt;a class="headerlink" href="#complex-type-support-in-parquet-scans" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;Comet now supports complex types (Structs, Maps, and Arrays) when reading Parquet files. This functionality is not
yet available when reading Parquet files from Apache Iceberg.&lt;/p&gt;
&lt;p&gt;This functionality was only available in previous releases when manually specifying one of the new experimental
scan implementations. Comet now automatically chooses the best scan implementation based on the input schema, and no
longer requires manual configuration.&lt;/p&gt;
&lt;h3 id="complex-type-processing-improvements"&gt;Complex Type Processing Improvements&lt;a class="headerlink" href="#complex-type-processing-improvements" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;Numerous improvements have been made to complex type support to ensure Spark-compatible behavior when casting between
structs and accessing fields within deeply nested types.&lt;/p&gt;
&lt;h3 id="shuffle-improvements"&gt;Shuffle Improvements&lt;a class="headerlink" href="#shuffle-improvements" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;Comet now accelerates a broader range of shuffle operations, leading to more queries running fully natively. In
previous releases, some shuffle operations fell back to Spark to avoid some known bugs in Comet, and these bugs have
now been fixed.&lt;/p&gt;
&lt;h3 id="new-features"&gt;New Features&lt;a class="headerlink" href="#new-features" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;Comet 0.9.0 adds support for the following Spark expressions:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;ArrayDistinct&lt;/li&gt;
&lt;li&gt;ArrayMax&lt;/li&gt;
&lt;li&gt;ArrayRepeat&lt;/li&gt;
&lt;li&gt;ArrayUnion&lt;/li&gt;
&lt;li&gt;BitCount&lt;/li&gt;
&lt;li&gt;BitNot&lt;/li&gt;
&lt;li&gt;Expm1&lt;/li&gt;
&lt;li&gt;MapValues&lt;/li&gt;
&lt;li&gt;Signum&lt;/li&gt;
&lt;li&gt;ToPrettyString&lt;/li&gt;
&lt;li&gt;map[]&lt;/li&gt;
&lt;/ul&gt;
&lt;h3 id="improved-spark-sql-test-coverage"&gt;Improved Spark SQL Test Coverage&lt;a class="headerlink" href="#improved-spark-sql-test-coverage" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;Comet now passes 97% of the Spark SQL test suite, with more than 24,000 tests passing (based on testing against
Spark 3.5.6). The remaining 3% of tests are ignored for various reasons, such as being too specific to Spark 
internals, or testing for features that are not relevant to Comet, such as whole-stage code generation, which
is not needed when using a vectorized execution engine.&lt;/p&gt;
&lt;p&gt;This release contains numerous bug fixes to achieve this coverage, including improved support for exchange reuse
when AQE is enabled.&lt;/p&gt;
&lt;style&gt;
  table {
    border-collapse: collapse;
    width: 100%;
    font-family: sans-serif;
  }

  th, td {
    text-align: left;
    padding: 8px 12px;
  }

  th {
    background-color: #f2f2f2;
    font-weight: bold;
  }

  td {
    border-bottom: 1px solid #ddd;
  }

  tbody tr:last-child td {
    font-weight: bold;
    border-top: 2px solid #000;
  }
&lt;/style&gt;
&lt;table class="table"&gt;
&lt;thead&gt;
&lt;tr&gt;
&lt;th&gt;Module&lt;/th&gt;
&lt;th&gt;Passed&lt;/th&gt;
&lt;th&gt;Ignored&lt;/th&gt;
&lt;th&gt;Canceled&lt;/th&gt;
&lt;th&gt;Total&lt;/th&gt;
&lt;/tr&gt;
&lt;/thead&gt;
&lt;tbody&gt;
&lt;tr&gt;&lt;td&gt;catalyst&lt;/td&gt;&lt;td&gt;7,232&lt;/td&gt;&lt;td&gt;5&lt;/td&gt;&lt;td&gt;1&lt;/td&gt;&lt;td&gt;7,238&lt;/td&gt;&lt;/tr&gt;
&lt;tr&gt;&lt;td&gt;core-1&lt;/td&gt;&lt;td&gt;9,186&lt;/td&gt;&lt;td&gt;246&lt;/td&gt;&lt;td&gt;6&lt;/td&gt;&lt;td&gt;9,438&lt;/td&gt;&lt;/tr&gt;
&lt;tr&gt;&lt;td&gt;core-2&lt;/td&gt;&lt;td&gt;2,649&lt;/td&gt;&lt;td&gt;393&lt;/td&gt;&lt;td&gt;0&lt;/td&gt;&lt;td&gt;3,042&lt;/td&gt;&lt;/tr&gt;
&lt;tr&gt;&lt;td&gt;core-3&lt;/td&gt;&lt;td&gt;1,757&lt;/td&gt;&lt;td&gt;136&lt;/td&gt;&lt;td&gt;16&lt;/td&gt;&lt;td&gt;1,909&lt;/td&gt;&lt;/tr&gt;
&lt;tr&gt;&lt;td&gt;hive-1&lt;/td&gt;&lt;td&gt;2,174&lt;/td&gt;&lt;td&gt;14&lt;/td&gt;&lt;td&gt;4&lt;/td&gt;&lt;td&gt;2,192&lt;/td&gt;&lt;/tr&gt;
&lt;tr&gt;&lt;td&gt;hive-2&lt;/td&gt;&lt;td&gt;19&lt;/td&gt;&lt;td&gt;1&lt;/td&gt;&lt;td&gt;4&lt;/td&gt;&lt;td&gt;24&lt;/td&gt;&lt;/tr&gt;
&lt;tr&gt;&lt;td&gt;hive-3&lt;/td&gt;&lt;td&gt;1,058&lt;/td&gt;&lt;td&gt;11&lt;/td&gt;&lt;td&gt;4&lt;/td&gt;&lt;td&gt;1,073&lt;/td&gt;&lt;/tr&gt;
&lt;tr&gt;&lt;td&gt;&lt;strong&gt;Total&lt;/strong&gt;&lt;/td&gt;&lt;td&gt;&lt;strong&gt;24,075&lt;/strong&gt;&lt;/td&gt;&lt;td&gt;&lt;strong&gt;806&lt;/strong&gt;&lt;/td&gt;&lt;td&gt;&lt;strong&gt;31&lt;/strong&gt;&lt;/td&gt;&lt;td&gt;&lt;strong&gt;24,912&lt;/strong&gt;&lt;/td&gt;&lt;/tr&gt;
&lt;/tbody&gt;
&lt;/table&gt;
&lt;h3 id="memory-performance-tracing"&gt;Memory &amp;amp; Performance Tracing&lt;a class="headerlink" href="#memory-performance-tracing" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;Comet now provides a tracing feature for analyzing performance and off-heap versus on-heap memory usage. See the
&lt;a href="https://datafusion.apache.org/comet/contributor-guide/tracing.html"&gt;Comet Tracing Guide&lt;/a&gt; for more information.&lt;/p&gt;
&lt;p&gt;&lt;img alt="Comet Tracing" class="img-fluid" src="/blog/images/comet-0.9.0/tracing.png" width="100%"/&gt;&lt;/p&gt;
&lt;h3 id="spark-compatibility"&gt;Spark Compatibility&lt;a class="headerlink" href="#spark-compatibility" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;ul&gt;
&lt;li&gt;Spark 3.4.3 with JDK 11 &amp;amp; 17, Scala 2.12 &amp;amp; 2.13&lt;/li&gt;
&lt;li&gt;Spark 3.5.4 through 3.5.6 with JDK 11 &amp;amp; 17, Scala 2.12 &amp;amp; 2.13&lt;/li&gt;
&lt;li&gt;Experimental support for Spark 4.0.0 with JDK 17, Scala 2.13&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;We are looking for help from the community to fully support Spark 4.0.0. See &lt;a href="https://github.com/apache/datafusion-comet/issues/1637"&gt;EPIC: Support 4.0.0&lt;/a&gt; for more information.&lt;/p&gt;
&lt;p&gt;Note that Java 8 support was removed from this release because Apache Arrow no longer supports it.&lt;/p&gt;
&lt;h2 id="getting-involved"&gt;Getting Involved&lt;a class="headerlink" href="#getting-involved" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;The Comet project welcomes new contributors. We use the same &lt;a href="https://datafusion.apache.org/contributor-guide/communication.html#slack-and-discord"&gt;Slack and Discord&lt;/a&gt; channels as the main DataFusion
project and have a weekly &lt;a href="https://docs.google.com/document/d/1NBpkIAuU7O9h8Br5CbFksDhX-L9TyO9wmGLPMe0Plc8/edit?usp=sharing"&gt;DataFusion video call&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;The easiest way to get involved is to test Comet with your current Spark jobs and file issues for any bugs or
performance regressions that you find. See the &lt;a href="https://datafusion.apache.org/comet/user-guide/installation.html"&gt;Getting Started&lt;/a&gt; guide for instructions on downloading and installing
Comet.&lt;/p&gt;
&lt;p&gt;There are also many &lt;a href="https://github.com/apache/datafusion-comet/contribute"&gt;good first issues&lt;/a&gt; waiting for contributions.&lt;/p&gt;</content><category term="blog"/></entry><entry><title>Using Rust async for Query Execution and Cancelling Long-Running Queries</title><link href="https://datafusion.apache.org/blog/2025/06/30/cancellation" rel="alternate"/><published>2025-06-30T00:00:00+00:00</published><updated>2025-06-30T00:00:00+00:00</updated><author><name>Pepijn Van Eeckhoudt</name></author><id>tag:datafusion.apache.org,2025-06-30:/blog/2025/06/30/cancellation</id><summary type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;style&gt;
figure {
  margin: 20px 0;
}

figure img {
  display: block;
  max-width: 80%;
  margin: auto;
}

figcaption {
  font-style: italic;
  color: #555;
  font-size: 0.9em;
  max-width: 80%;
  margin: auto;
  text-align: center;
}
&lt;/style&gt;
&lt;p&gt;Have you ever tried to cancel a query that just wouldn't stop?
In this post, we'll review how Rust's &lt;a href="https://doc.rust-lang.org/book/ch17-00-async-await.html"&gt;&lt;code&gt;async&lt;/code&gt; programming model&lt;/a&gt; works, how …&lt;/p&gt;</summary><content type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;style&gt;
figure {
  margin: 20px 0;
}

figure img {
  display: block;
  max-width: 80%;
  margin: auto;
}

figcaption {
  font-style: italic;
  color: #555;
  font-size: 0.9em;
  max-width: 80%;
  margin: auto;
  text-align: center;
}
&lt;/style&gt;
&lt;p&gt;Have you ever tried to cancel a query that just wouldn't stop?
In this post, we'll review how Rust's &lt;a href="https://doc.rust-lang.org/book/ch17-00-async-await.html"&gt;&lt;code&gt;async&lt;/code&gt; programming model&lt;/a&gt; works, how &lt;a href="https://datafusion.apache.org/"&gt;DataFusion&lt;/a&gt; uses that model for CPU intensive tasks, and how this is used to cancel queries.
Then we'll review some cases where queries could not be canceled in DataFusion and what the community did to resolve the problem.&lt;/p&gt;
&lt;h2 id="understanding-rusts-async-model"&gt;Understanding Rust's Async Model&lt;a class="headerlink" href="#understanding-rusts-async-model" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;DataFusion, somewhat unconventionally, &lt;a href="https://docs.rs/datafusion/latest/datafusion/#thread-scheduling-cpu--io-thread-pools-and-tokio-runtimes"&gt;uses the Rust async system and the Tokio task scheduler&lt;/a&gt; for CPU intensive processing.
To really understand the cancellation problem you first need to be familiar with Rust's asynchronous programming model which is a bit different from what you might be used to from other ecosystems.
Let's go over the basics again as a refresher.
If you're familiar with the ins and outs of &lt;code&gt;Future&lt;/code&gt; and &lt;code&gt;async&lt;/code&gt; you can skip this section.&lt;/p&gt;
&lt;h3 id="futures-are-inert"&gt;Futures Are Inert&lt;a class="headerlink" href="#futures-are-inert" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;Rust's asynchronous programming model is built around the &lt;a href="https://doc.rust-lang.org/std/future/trait.Future.html"&gt;&lt;code&gt;Future&amp;lt;T&amp;gt;&lt;/code&gt;&lt;/a&gt; trait.
In contrast to, for instance, Javascript's &lt;a href="https://developer.mozilla.org/en-US/docs/Web/JavaScript/Reference/Global_Objects/Promise"&gt;&lt;code&gt;Promise&lt;/code&gt;&lt;/a&gt; or Java's &lt;a href="https://docs.oracle.com/en/java/javase/21/docs/api/java.base/java/util/concurrent/Future.html"&gt;&lt;code&gt;Future&lt;/code&gt;&lt;/a&gt; a Rust &lt;code&gt;Future&lt;/code&gt; does not necessarily represent an actively running asynchronous job.
Instead, a &lt;code&gt;Future&amp;lt;T&amp;gt;&lt;/code&gt; represents a lazy calculation that only makes progress when explicitly asked to do so.
This is done by calling the &lt;a href="https://doc.rust-lang.org/std/future/trait.Future.html#tymethod.poll"&gt;&lt;code&gt;poll&lt;/code&gt;&lt;/a&gt; method of a &lt;code&gt;Future&lt;/code&gt;.
If nobody polls a &lt;code&gt;Future&lt;/code&gt; explicitly, it is &lt;a href="https://doc.rust-lang.org/std/future/trait.Future.html#runtime-characteristics"&gt;an inert object&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;Calling &lt;code&gt;Future::poll&lt;/code&gt; results in one of two options:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;a href="https://doc.rust-lang.org/std/task/enum.Poll.html#variant.Pending"&gt;&lt;code&gt;Poll::Pending&lt;/code&gt;&lt;/a&gt; if the evaluation is not yet complete, most often because it needs to wait for something like I/O before it can continue&lt;/li&gt;
&lt;li&gt;&lt;a href="https://doc.rust-lang.org/std/task/enum.Poll.html#variant.Ready"&gt;&lt;code&gt;Poll::Ready&amp;lt;T&amp;gt;&lt;/code&gt;&lt;/a&gt; when it has completed and produced a value&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;When a &lt;code&gt;Future&lt;/code&gt; returns &lt;code&gt;Pending&lt;/code&gt;, it saves its internal state so it can pick up where it left off the next time you poll it.
This internal state management makes Rust's &lt;code&gt;Future&lt;/code&gt;s memory-efficient and composable.
Rather than freezing the full call stack leading to a certain point, only the relevant state to resume the future needs to be retained.&lt;/p&gt;
&lt;p&gt;Additionally, a &lt;code&gt;Future&lt;/code&gt; must set up the necessary signaling to notify the caller when it should call &lt;code&gt;poll&lt;/code&gt; again, to avoid a busy-waiting loop.
This is done using a &lt;a href="https://doc.rust-lang.org/std/task/struct.Waker.html"&gt;&lt;code&gt;Waker&lt;/code&gt;&lt;/a&gt; which the &lt;code&gt;Future&lt;/code&gt; receives via the &lt;code&gt;Context&lt;/code&gt; parameter of the &lt;code&gt;poll&lt;/code&gt; function. &lt;/p&gt;
&lt;p&gt;Manual implementations of &lt;code&gt;Future&lt;/code&gt; are most often little finite state machines.
Each state in the process of completing the calculation is modeled as a variant of an &lt;code&gt;enum&lt;/code&gt;.
Before a &lt;code&gt;Future&lt;/code&gt; returns &lt;code&gt;Pending&lt;/code&gt;, it bundles the data required to resume in an enum variant, stores that enum variant in itself, and then returns.
While compact and efficient, the resulting code is often quite verbose.&lt;/p&gt;
&lt;p&gt;The &lt;code&gt;async&lt;/code&gt; keyword was introduced to make life easier on Rust programmers.
It provides elegant syntactic sugar for the manual state machine &lt;code&gt;Future&lt;/code&gt; approach.
When you write an &lt;code&gt;async&lt;/code&gt; function or block, the compiler transforms linear code into a state machine based &lt;code&gt;Future&lt;/code&gt; similar to the one described above for you.
Since all the state management is compiler generated and hidden from sight, async code tends to be easier to write initially, more readable afterward, while maintaining the same underlying mechanics.&lt;/p&gt;
&lt;p&gt;The &lt;code&gt;await&lt;/code&gt; keyword complements &lt;code&gt;async&lt;/code&gt; pausing execution until a &lt;code&gt;Future&lt;/code&gt; completes. &lt;br/&gt;
When you &lt;code&gt;.await&lt;/code&gt; a &lt;code&gt;Future&lt;/code&gt;, you're essentially telling the compiler to generate code that:&lt;/p&gt;
&lt;ol&gt;
&lt;li&gt;Polls the &lt;code&gt;Future&lt;/code&gt; with the current (implicit) asynchronous context&lt;/li&gt;
&lt;li&gt;If &lt;code&gt;poll&lt;/code&gt; returns &lt;code&gt;Poll::Pending&lt;/code&gt;, save the state of the &lt;code&gt;Future&lt;/code&gt; so that it can resume at this point and return &lt;code&gt;Poll::Pending&lt;/code&gt;&lt;/li&gt;
&lt;li&gt;If it returns &lt;code&gt;Poll::Ready(value)&lt;/code&gt;, continue execution with that value&lt;/li&gt;
&lt;/ol&gt;
&lt;h3 id="from-futures-to-streams"&gt;From Futures to Streams&lt;a class="headerlink" href="#from-futures-to-streams" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;The &lt;a href="https://docs.rs/futures/latest/futures/"&gt;&lt;code&gt;futures&lt;/code&gt;&lt;/a&gt; crate extends the &lt;code&gt;Future&lt;/code&gt; model with a trait named &lt;a href="https://docs.rs/futures/latest/futures/prelude/trait.Stream.html"&gt;&lt;code&gt;Stream&lt;/code&gt;&lt;/a&gt;.
&lt;code&gt;Stream&amp;lt;Item = T&amp;gt;&lt;/code&gt; represents a sequence of values that are each produced asynchronously rather than just a single value.
It's the asynchronous equivalent of &lt;code&gt;Iterator&amp;lt;Item = T&amp;gt;&lt;/code&gt;.&lt;/p&gt;
&lt;p&gt;The &lt;code&gt;Stream&lt;/code&gt; trait has one method named &lt;a href="https://docs.rs/futures/latest/futures/prelude/trait.Stream.html#tymethod.poll_next"&gt;&lt;code&gt;poll_next&lt;/code&gt;&lt;/a&gt; that returns:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;code&gt;Poll::Pending&lt;/code&gt; when the next value isn't ready yet, just like a &lt;code&gt;Future&lt;/code&gt; would&lt;/li&gt;
&lt;li&gt;&lt;code&gt;Poll::Ready(Some(value))&lt;/code&gt; when a new value is available&lt;/li&gt;
&lt;li&gt;&lt;code&gt;Poll::Ready(None)&lt;/code&gt; when the stream is exhausted&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;Under the hood, an implementation of &lt;code&gt;Stream&lt;/code&gt; is very similar to a &lt;code&gt;Future&lt;/code&gt;.
Typically, they're also implemented as state machines, the main difference being that they produce multiple values rather than just one.
Just like &lt;code&gt;Future&lt;/code&gt;, a &lt;code&gt;Stream&lt;/code&gt; is inert unless explicitly polled.&lt;/p&gt;
&lt;p&gt;Now that we understand the basics of Rust's async model, let's see how DataFusion leverages these concepts to execute queries.&lt;/p&gt;
&lt;h2 id="how-datafusion-executes-queries"&gt;How DataFusion Executes Queries&lt;a class="headerlink" href="#how-datafusion-executes-queries" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;In DataFusion, the short version of how queries are executed is as follows (you can find more in-depth coverage of this in the &lt;a href="https://docs.rs/datafusion/latest/datafusion/#streaming-execution"&gt;DataFusion documentation&lt;/a&gt;):&lt;/p&gt;
&lt;ol&gt;
&lt;li&gt;First the query is compiled into a tree of &lt;a href="https://docs.rs/datafusion/latest/datafusion/physical_plan/trait.ExecutionPlan.html"&gt;&lt;code&gt;ExecutionPlan&lt;/code&gt;&lt;/a&gt; nodes&lt;/li&gt;
&lt;li&gt;&lt;a href="https://docs.rs/datafusion/latest/datafusion/physical_plan/trait.ExecutionPlan.html#tymethod.execute"&gt;&lt;code&gt;ExecutionPlan::execute&lt;/code&gt;&lt;/a&gt; is called on the root of the tree. &lt;/li&gt;
&lt;li&gt;This method returns a &lt;a href="https://docs.rs/datafusion/latest/datafusion/execution/type.SendableRecordBatchStream.html"&gt;&lt;code&gt;SendableRecordBatchStream&lt;/code&gt;&lt;/a&gt; (a pinned &lt;code&gt;Box&amp;lt;dyn Stream&amp;lt;RecordBatch&amp;gt;&amp;gt;&lt;/code&gt;)&lt;/li&gt;
&lt;li&gt;&lt;code&gt;Stream::poll_next&lt;/code&gt; is called in a loop to get the results&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;In other words, the execution of a DataFusion query boils down to polling an asynchronous stream.
Like all &lt;code&gt;Stream&lt;/code&gt; implementations, we need to explicitly poll the stream for the query to make progress. &lt;/p&gt;
&lt;p&gt;The &lt;code&gt;Stream&lt;/code&gt; we get in step 2 is actually the root of a tree of &lt;code&gt;Streams&lt;/code&gt; that mostly mirrors the execution plan tree.
Each stream tree node processes the record batches it gets from its children.
The leaves of the tree produce record batches themselves.&lt;/p&gt;
&lt;p&gt;Query execution progresses each time you call &lt;code&gt;poll_next&lt;/code&gt; on the root stream.
This call typically cascades down the tree, with each node calling &lt;code&gt;poll_next&lt;/code&gt; on its children to get the data it needs to process.&lt;/p&gt;
&lt;p&gt;Here's where the first signs of problems start to show up: some operations (like aggregations, sorts, or certain join phases) need to process a lot of data before producing any output.
When &lt;code&gt;poll_next&lt;/code&gt; encounters one of these operations, it might require substantial work before it can return a record batch.&lt;/p&gt;
&lt;h3 id="tokio-and-cooperative-scheduling"&gt;Tokio and Cooperative Scheduling&lt;a class="headerlink" href="#tokio-and-cooperative-scheduling" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;We need to make a small detour now via Tokio's scheduler before we can get to the query cancellation problem.
DataFusion makes use of the &lt;a href="https://tokio.rs"&gt;Tokio asynchronous runtime&lt;/a&gt;, which uses a &lt;a href="https://docs.rs/tokio/latest/tokio/task/index.html#what-are-tasks"&gt;cooperative scheduling model&lt;/a&gt;.
This is fundamentally different from preemptive scheduling that you might be used to:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;In &lt;strong&gt;preemptive scheduling&lt;/strong&gt;, the system can interrupt a task at any time to run something else&lt;/li&gt;
&lt;li&gt;In &lt;strong&gt;cooperative scheduling&lt;/strong&gt;, tasks must voluntarily yield control back to the scheduler&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;This distinction is crucial for understanding our cancellation problem.&lt;/p&gt;
&lt;p&gt;A task in Tokio is modeled as a &lt;code&gt;Future&lt;/code&gt; which is passed to one of the task initiation functions like &lt;a href="https://docs.rs/tokio/latest/tokio/task/fn.spawn.html"&gt;&lt;code&gt;spawn&lt;/code&gt;&lt;/a&gt;.
Tokio runs the task by calling &lt;code&gt;Future::poll&lt;/code&gt; in a loop until it returns &lt;code&gt;Poll::Ready&lt;/code&gt;.
While that &lt;code&gt;Future::poll&lt;/code&gt; call is running, Tokio has no way to forcibly interrupt it.
It must cooperate by periodically yielding control, either by returning &lt;code&gt;Poll::Pending&lt;/code&gt; or &lt;code&gt;Poll::Ready&lt;/code&gt;.&lt;/p&gt;
&lt;p&gt;Similarly, when you try to abort a task by calling &lt;a href="https://docs.rs/tokio/latest/tokio/task/struct.JoinHandle.html#method.abort"&gt;&lt;code&gt;JoinHandle::abort()&lt;/code&gt;&lt;/a&gt;, the Tokio runtime can't immediately force it to stop.
You're just telling Tokio: "When this task next yields control, don't call &lt;code&gt;Future::poll&lt;/code&gt; anymore."
If the task never yields, it can't be aborted.&lt;/p&gt;
&lt;h3 id="the-cancellation-problem"&gt;The Cancellation Problem&lt;a class="headerlink" href="#the-cancellation-problem" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;With all the necessary background in place, now let's look at how the DataFusion CLI tries to run and cancel a query.
The code below is a simplified version of &lt;a href="https://github.com/apache/datafusion/blob/db13dd93579945628cd81d534c032f5e6cc77967/datafusion-cli/src/exec.rs#L179-L186"&gt;what the CLI actually does&lt;/a&gt;:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-rust"&gt;fn exec_query() {
    let runtime: tokio::runtime::Runtime = ...;
    let stream: SendableRecordBatchStream = ...;

    runtime.block_on(async {
        tokio::select! {
            next_batch = stream.next() =&amp;gt; ...
            _ = signal::ctrl_c() =&amp;gt; ...,
        }
    })
}
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;First the CLI sets up a Tokio runtime instance.
It then reads the query to execute from standard input or file and turns it into a &lt;code&gt;Stream&lt;/code&gt;.
Then it calls &lt;code&gt;next&lt;/code&gt; on stream which is an &lt;code&gt;async&lt;/code&gt; wrapper for &lt;code&gt;poll_next&lt;/code&gt;.
It passes this to the &lt;a href="https://docs.rs/tokio/latest/tokio/macro.select.html"&gt;&lt;code&gt;select!&lt;/code&gt;&lt;/a&gt; macro along with a ctrl-C handler.&lt;/p&gt;
&lt;p&gt;The &lt;code&gt;select!&lt;/code&gt; macro races these two &lt;code&gt;Future&lt;/code&gt;s and completes when either one finishes.
The intent is that when you press Ctrl+C, the &lt;code&gt;signal::ctrl_c()&lt;/code&gt; &lt;code&gt;Future&lt;/code&gt; should complete.
The &lt;a href="https://docs.rs/datafusion/latest/datafusion/physical_plan/trait.ExecutionPlan.html#cancellation--aborting-execution"&gt;stream is cancelled&lt;/a&gt; when it is dropped as it is inert by itself and nothing will be able to call &lt;code&gt;poll_next&lt;/code&gt; again.&lt;/p&gt;
&lt;p&gt;But there's a catch: &lt;code&gt;select!&lt;/code&gt; still follows cooperative scheduling rules.
It polls each &lt;code&gt;Future&lt;/code&gt; in sequence, and if the first one (our query) gets stuck in a long computation, it never gets around to polling the cancellation signal.&lt;/p&gt;
&lt;p&gt;Imagine a query that needs to calculate something intensive, like sorting billions of rows.
Unless the sorting Stream is written with care (which the one in DataFusion is), the &lt;code&gt;poll_next&lt;/code&gt; call may take several minutes or even longer without returning.
During this time, Tokio can't check if you've pressed Ctrl+C, and the query continues running despite your cancellation request.&lt;/p&gt;
&lt;h2 id="a-closer-look-at-blocking-operators"&gt;A Closer Look at Blocking Operators&lt;a class="headerlink" href="#a-closer-look-at-blocking-operators" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;Let's peel back a layer of the onion and look at what's happening in a blocking &lt;code&gt;poll_next&lt;/code&gt; implementation.
Here's a drastically simplified version of a &lt;code&gt;COUNT(*)&lt;/code&gt; aggregation - something you might use in a query like &lt;code&gt;SELECT COUNT(*) FROM table&lt;/code&gt;:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-rust"&gt;struct BlockingStream {
    // the input: an inner stream that is wrapped
    stream: SendableRecordBatchStream,
    count: usize,
    finished: bool,
}

impl Stream for BlockingStream {
    type Item = Result&amp;lt;RecordBatch&amp;gt;;
    fn poll_next(mut self: Pin&amp;lt;&amp;amp;mut Self&amp;gt;, cx: &amp;amp;mut Context&amp;lt;'_&amp;gt;) -&amp;gt; Poll&amp;lt;Option&amp;lt;Self::Item&amp;gt;&amp;gt; {
        if self.finished {
            // return None if we're finished
            return Poll::Ready(None);
        }

        loop {
            // poll the input stream to get the next batch if ready
            match ready!(self.stream.poll_next_unpin(cx)) {
                // increment the counter if we got a batch
                Some(Ok(batch)) =&amp;gt; self.count += batch.num_rows(),
                // on end-of-stream, create a record batch for the counter
                None =&amp;gt; {
                    self.finished = true;
                    return Poll::Ready(Some(Ok(create_record_batch(self.count))));
                }
                // pass on any errors verbatim
                Some(Err(e)) =&amp;gt; return Poll::Ready(Some(Err(e))),
            }
        }
    }
}
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;How does this code work? Let's break it down step by step:&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;1. Initial check&lt;/strong&gt;: We first check if we've already finished processing. If so, we return &lt;code&gt;Ready(None)&lt;/code&gt; to signal the end of our stream:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-rust"&gt;if self.finished {
    return Poll::Ready(None);
}
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;&lt;strong&gt;2. Processing loop&lt;/strong&gt;: If we're not done yet, we enter a loop to process incoming batches from our input stream:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-rust"&gt;loop {
    match ready!(self.stream.poll_next_unpin(cx)) {
        // Handle different cases...
    }
}
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;The &lt;a href="https://doc.rust-lang.org/beta/std/task/macro.ready.html"&gt;&lt;code&gt;ready!&lt;/code&gt;&lt;/a&gt; macro checks if the input stream returned &lt;code&gt;Pending&lt;/code&gt; and if so, immediately returns &lt;code&gt;Pending&lt;/code&gt; from our function as well.&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;3. Processing data&lt;/strong&gt;: For each batch we receive, we simply add its row count to our running total:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-rust"&gt;Some(Ok(batch)) =&amp;gt; self.count += batch.num_rows(),
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;&lt;strong&gt;4. End of input&lt;/strong&gt;: When the child stream is exhausted (returns &lt;code&gt;None&lt;/code&gt;), we calculate our final result and convert it into a record batch (omitted for brevity):&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-rust"&gt;None =&amp;gt; {
    self.finished = true;
    return Poll::Ready(Some(Ok(create_record_batch(self.count))));
}
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;&lt;strong&gt;5. Error handling&lt;/strong&gt;: If we encounter an error, we pass it along immediately:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-rust"&gt;Some(Err(e)) =&amp;gt; return Poll::Ready(Some(Err(e))),
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;This code looks perfectly reasonable at first glance.
But there's a subtle issue lurking here: what happens if the input stream &lt;em&gt;always&lt;/em&gt; returns &lt;code&gt;Ready&lt;/code&gt; and never returns &lt;code&gt;Pending&lt;/code&gt;?&lt;/p&gt;
&lt;p&gt;In that case, the processing loop will keep running without returning &lt;code&gt;Poll::Pending&lt;/code&gt; and thus never yield control back to Tokio's scheduler.
This means we could be stuck in a single &lt;code&gt;poll_next&lt;/code&gt; call for quite some time - exactly the scenario that prevents query cancellation from working!&lt;/p&gt;
&lt;p&gt;So how do we solve this problem? Let's explore some strategies to ensure our operators yield control periodically.&lt;/p&gt;
&lt;h2 id="unblocking-operators"&gt;Unblocking Operators&lt;a class="headerlink" href="#unblocking-operators" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;Now let's look at how we can ensure we return &lt;code&gt;Pending&lt;/code&gt; every now and then.&lt;/p&gt;
&lt;h3 id="independent-cooperative-operators"&gt;Independent Cooperative Operators&lt;a class="headerlink" href="#independent-cooperative-operators" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;One simple way to return &lt;code&gt;Pending&lt;/code&gt; is using a loop counter.
We do the exact same thing as before, but on each loop iteration we decrement our counter.
If the counter hits zero we return &lt;code&gt;Pending&lt;/code&gt;.
The following example ensures we iterate at most 128 times before yielding.&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-rust"&gt;struct CountingSourceStream {
   counter: usize
}

impl Stream for CountingSourceStream {
    type Item = Result&amp;lt;RecordBatch&amp;gt;;

    fn poll_next(mut self: Pin&amp;lt;&amp;amp;mut Self&amp;gt;, cx: &amp;amp;mut Context&amp;lt;'_&amp;gt;) -&amp;gt; Poll&amp;lt;Option&amp;lt;Self::Item&amp;gt;&amp;gt; {
        if self.counter &amp;gt;= 128 {
            self.counter = 0;
            cx.waker().wake_by_ref();
            return Poll::Pending;
        }

        self.counter += 1;
        let batch = ...;
        Ready(Some(Ok(batch)))
    }
}
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;If &lt;code&gt;CountingSourceStream&lt;/code&gt; was the input for the &lt;code&gt;BlockingStream&lt;/code&gt; example above, 
the &lt;code&gt;BlockingStream&lt;/code&gt; will receive a &lt;code&gt;Pending&lt;/code&gt; periodically causing it to yield too. 
Can we really solve the cancel problem simply by periodically yielding in source streams? &lt;/p&gt;
&lt;p&gt;Unfortunately, no.
Let's look at what happens when we start combining operators in more complex configurations.
Suppose we create a plan like this.&lt;/p&gt;
&lt;figure&gt;
&lt;img alt="Diagram showing a plan that merges two branches that return Pending at different intervals." src="/blog/images/task-cancellation/merge_plan.png"/&gt;
&lt;figcaption&gt;A plan that merges two branches by alternating between them.&lt;/figcaption&gt;
&lt;/figure&gt;
&lt;p&gt;Each &lt;code&gt;CountingSource&lt;/code&gt; produces a &lt;code&gt;Pending&lt;/code&gt; every 128 batches.
The &lt;code&gt;Filter&lt;/code&gt; is a stream that drops a batch every 50 record batches.
Merge is a simple combining operator the uses &lt;code&gt;futures::stream::select&lt;/code&gt; to combine two stream.&lt;/p&gt;
&lt;p&gt;When we set this stream in motion, the merge operator will poll the left and right branch in a round-robin fashion.
The sources will each emit &lt;code&gt;Pending&lt;/code&gt; every 128 batches, but since the &lt;code&gt;Filter&lt;/code&gt; drops batches, they arrive out-of-phase at the merge operator.
As a consequence the merge operator will always have the opportunity of polling the other stream when one returns &lt;code&gt;Pending&lt;/code&gt;.
The &lt;code&gt;Merge&lt;/code&gt; stream thus is an always ready stream, even though the sources are yielding.
If we use &lt;code&gt;Merge&lt;/code&gt; as the input to our aggregating operator we're right back where we started.&lt;/p&gt;
&lt;h3 id="coordinated-cooperation"&gt;Coordinated Cooperation&lt;a class="headerlink" href="#coordinated-cooperation" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;Wouldn't it be great if we could get all the operators to coordinate amongst each other?
When one of them determines that it's time to yield, all the other operators agree and start returning &lt;code&gt;Pending&lt;/code&gt; as well.
That way our task would be coaxed towards yielding even if it tried to poll many different operators.&lt;/p&gt;
&lt;p&gt;Luckily(?), the &lt;a href="https://tokio.rs/blog/2020-04-preemption"&gt;developers of Tokio ran into the exact same problem&lt;/a&gt; described above when network servers were under heavy load and came up with a solution.
Back in 2020, Tokio 0.2.14 introduced a per-task operation budget.
Rather than having individual counters littered throughout the code, the Tokio runtime itself manages a per task counter which is decremented by Tokio resources.
When the counter hits zero, all resources start returning &lt;code&gt;Pending&lt;/code&gt;.
The task will then yield, after which the Tokio runtime resets the counter.&lt;/p&gt;
&lt;p&gt;To illustrate what this process looks like, let's have a look at the execution of the following query &lt;code&gt;Stream&lt;/code&gt; tree when polled in a Tokio task.&lt;/p&gt;
&lt;figure&gt;
&lt;img alt="Diagram showing a plan with a task, AggregateExec, MergeStream and Two sources." src="/blog/images/task-cancellation/tokio_budget_plan.png"/&gt;
&lt;figcaption&gt;Query plan for aggregating a sorted stream from two sources. Each source reads a stream of `RecordBatch`es, which are then merged into a single Stream by the `MergeStream` operator which is then aggregated by the `AggregateExec` operator. Arrows represent the data flow direction&lt;/figcaption&gt;
&lt;/figure&gt;
&lt;p&gt;If we assume a task budget of 1 unit, each time Tokio schedules the task would result in the following sequence of function calls.&lt;/p&gt;
&lt;figure&gt;
&lt;img alt="Sequence diagram showing how the tokio task budget is used and reset." class="img-fluid" src="/blog/images/task-cancellation/tokio_budget.png"/&gt;
&lt;figcaption&gt;Tokio task budget system, assuming the task budget is set to 1, for the plan above.&lt;/figcaption&gt;
&lt;/figure&gt;
&lt;p&gt;The aggregation stream would try to poll the merge stream in a loop.
The first iteration of the loop consumes the single unit of budget, and returns &lt;code&gt;Ready&lt;/code&gt;.
The second iteration polls the merge stream again which now tries to poll the second scan stream.
Since there is no budget remaining &lt;code&gt;Pending&lt;/code&gt; is returned.
The merge stream may now try to poll the first source stream again, but since the budget is still depleted &lt;code&gt;Pending&lt;/code&gt; is returned as well.
The merge stream now has no other option than to return &lt;code&gt;Pending&lt;/code&gt; itself as well, causing the aggregation to break out of its loop.
The &lt;code&gt;Pending&lt;/code&gt; result bubbles all the way up to the Tokio runtime, at which point the runtime regains control.
When the runtime reschedules the task, it resets the budget and calls &lt;code&gt;poll&lt;/code&gt; on the task &lt;code&gt;Future&lt;/code&gt; again for another round of progress.&lt;/p&gt;
&lt;p&gt;The key mechanism that makes this work well is the single task budget that's shared amongst all the scan streams.
Once the budget is depleted, no streams can make any further progress without first returning control to tokio.
This causes all possible avenues the task has to make progress to return &lt;code&gt;Pending&lt;/code&gt; which results in the task being nudged towards yielding control.&lt;/p&gt;
&lt;p&gt;As it turns out DataFusion was already using this mechanism implicitly.
Every exchange-like operator (such as &lt;code&gt;RepartitionExec&lt;/code&gt;) internally makes use of a Tokio multiple producer, single consumer &lt;a href="https://tokio.rs/tokio/tutorial/channels"&gt;&lt;code&gt;Channel&lt;/code&gt;&lt;/a&gt;.
When calling &lt;code&gt;Receiver::recv&lt;/code&gt; for one of these channels, a unit of Tokio task budget is consumed.
As a consequence, query plans that made use of exchange-like operators were
already mostly cancelable.
The plan cancellation bug only showed up when running parts of plans without such operators, such as when using a single core.&lt;/p&gt;
&lt;p&gt;Now let's see how we can explicitly implement this budget-based approach in our own operators.&lt;/p&gt;
&lt;h3 id="depleting-the-tokio-budget"&gt;Depleting The Tokio Budget&lt;a class="headerlink" href="#depleting-the-tokio-budget" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;Let's revisit our original &lt;code&gt;BlockingStream&lt;/code&gt; and adapt it to use Tokio's budget system.&lt;/p&gt;
&lt;p&gt;The examples given here make use of functions from the Tokio &lt;code&gt;coop&lt;/code&gt; module that are still internal at the time of writing.
&lt;a href="https://github.com/tokio-rs/tokio/pull/7405"&gt;PR #7405&lt;/a&gt; on the Tokio project will make these accessible for external use.
The current DataFusion code emulates these functions as well as possible using &lt;a href="https://docs.rs/tokio/latest/tokio/task/coop/fn.has_budget_remaining.html"&gt;&lt;code&gt;has_budget_remaining&lt;/code&gt;&lt;/a&gt; and &lt;a href="https://docs.rs/tokio/latest/tokio/task/coop/fn.consume_budget.html"&gt;&lt;code&gt;consume_budget&lt;/code&gt;&lt;/a&gt;.&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-rust"&gt;struct BudgetSourceStream {
}

impl Stream for BudgetSourceStream {
    type Item = Result&amp;lt;RecordBatch&amp;gt;;

    fn poll_next(mut self: Pin&amp;lt;&amp;amp;mut Self&amp;gt;, cx: &amp;amp;mut Context&amp;lt;'_&amp;gt;) -&amp;gt; Poll&amp;lt;Option&amp;lt;Self::Item&amp;gt;&amp;gt; {
        let coop = ready!(tokio::task::coop::poll_proceed(cx));
        let batch: Poll&amp;lt;Option&amp;lt;Self::Item&amp;gt;&amp;gt; = ...;
        if batch.is_ready() {
            coop.made_progress();
        }
        batch
    }
}
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;The &lt;code&gt;Stream&lt;/code&gt; now goes through the following steps:&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;1. Try to consume budget&lt;/strong&gt;: the first thing the operator does is use &lt;code&gt;poll_proceed&lt;/code&gt; to try to consume a unit of budget.
If the budget is depleted, this function will return &lt;code&gt;Pending&lt;/code&gt;.
Otherwise, we consumed one budget unit and we can continue.&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-rust"&gt;let coop = ready!(tokio::task::coop::poll_proceed(cx));
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;&lt;strong&gt;2. Try to do some work&lt;/strong&gt;: next we try to produce a record batch.
That might not be possible if we're reading from some asynchronous resource that's not ready.&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-rust"&gt;let batch: Poll&amp;lt;Option&amp;lt;Self::Item&amp;gt;&amp;gt; = ...;
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;&lt;strong&gt;3. Commit the budget consumption&lt;/strong&gt;: finally, if we did produce a batch, we need to tell Tokio that we were able to make progress.&lt;/p&gt;
&lt;p&gt;That's done by calling the &lt;code&gt;made_progress&lt;/code&gt; method on the value &lt;code&gt;poll_proceed&lt;/code&gt; returned.&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-rust"&gt;if batch.is_ready() {
   coop.made_progress();
}
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;You might be wondering why the call to &lt;code&gt;made_progress&lt;/code&gt; is necessary.
This clever construct makes it easier to manage the budget.
The value returned by &lt;code&gt;poll_proceed&lt;/code&gt; will actually restore the budget to its original value when it is dropped unless &lt;code&gt;made_progress&lt;/code&gt; is called.
This ensures that if we exit early from our &lt;code&gt;poll_next&lt;/code&gt; implementation by returning &lt;code&gt;Pending&lt;/code&gt;, that the budget we had consumed becomes available again.
The task that invoked &lt;code&gt;poll_next&lt;/code&gt; can then use that budget again to try to make some other &lt;code&gt;Stream&lt;/code&gt; (or any resource for that matter) make progress.&lt;/p&gt;
&lt;h2 id="automatic-cooperation-for-all-operators"&gt;Automatic Cooperation For All Operators&lt;a class="headerlink" href="#automatic-cooperation-for-all-operators" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;DataFusion 49.0.0  integrates the Tokio task budget based fix in all built-in source operators.
This ensures that going forward, most queries will automatically be cancelable. 
See &lt;a href="https://github.com/apache/datafusion/pull/16398"&gt;the PR&lt;/a&gt; for more details.&lt;/p&gt;
&lt;p&gt;The design includes:&lt;/p&gt;
&lt;ol&gt;
&lt;li&gt;A new &lt;code&gt;ExecutionPlan&lt;/code&gt; property that indicates if an operator participates in cooperative scheduling or not.&lt;/li&gt;
&lt;li&gt;A new &lt;code&gt;EnsureCooperative&lt;/code&gt; optimizer rule to inspect query plans and insert &lt;code&gt;CooperativeExec&lt;/code&gt; nodes as needed to ensure custom source operators also participate.&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;These two changes combined already make it very unlikely you'll encounter any query that refuses to stop, even with custom operators.
For those situations where the automatic mechanisms are still not sufficient, there's a new &lt;code&gt;datafusion::physical_plan::coop&lt;/code&gt; module
with utility functions that make it easy to adopt cooperative scheduling in your custom operators as well.  &lt;/p&gt;
&lt;h2 id="acknowledgments"&gt;Acknowledgments&lt;a class="headerlink" href="#acknowledgments" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;Thank you to &lt;a href="https://datadobi.com/"&gt;Datadobi&lt;/a&gt; for sponsoring the development of this feature and to
the DataFusion community contributors including &lt;a href="https://github.com/zhuqi-lucas"&gt;Qi Zhu&lt;/a&gt; and &lt;a href="https://github.com/ozankabak"&gt;Mehmet Ozan
Kabak&lt;/a&gt;.&lt;/p&gt;
&lt;h2 id="about-datafusion"&gt;About DataFusion&lt;a class="headerlink" href="#about-datafusion" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;&lt;a href="https://datafusion.apache.org/"&gt;Apache DataFusion&lt;/a&gt; is an extensible query engine toolkit, written
in Rust, that uses &lt;a href="https://arrow.apache.org/"&gt;Apache Arrow&lt;/a&gt; as its in-memory format. DataFusion and
similar technology are part of the next generation “Deconstructed Database”
architectures, where new systems are built on a foundation of fast, modular
components, rather than as a single tightly integrated system.&lt;/p&gt;
&lt;p&gt;The &lt;a href="https://datafusion.apache.org/contributor-guide/communication.html"&gt;DataFusion community&lt;/a&gt; is always looking for new contributors to help
improve the project. If you are interested in learning more about how query
execution works, help document or improve the DataFusion codebase, or just try
it out, we would love for you to join us.&lt;/p&gt;</content><category term="blog"/></entry><entry><title>Optimizing SQL (and DataFrames) in DataFusion, Part 1: Query Optimization Overview</title><link href="https://datafusion.apache.org/blog/2025/06/15/optimizing-sql-dataframes-part-one" rel="alternate"/><published>2025-06-15T00:00:00+00:00</published><updated>2025-06-15T00:00:00+00:00</updated><author><name>alamb, akurmustafa</name></author><id>tag:datafusion.apache.org,2025-06-15:/blog/2025/06/15/optimizing-sql-dataframes-part-one</id><summary type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;p&gt;&lt;em&gt;Note: this blog was originally published &lt;a href="https://www.influxdata.com/blog/optimizing-sql-dataframes-part-one/"&gt;on the InfluxData blog&lt;/a&gt;&lt;/em&gt;&lt;/p&gt;
&lt;h2 id="introduction"&gt;Introduction&lt;a class="headerlink" href="#introduction" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;Sometimes Query Optimizers are seen as a sort of black magic, &lt;a href="https://15799.courses.cs.cmu.edu/spring2025/"&gt;“the most
challenging problem in computer
science,”&lt;/a&gt; according to Father
Pavlo, or some behind-the-scenes player. We believe this perception is because:&lt;/p&gt;
&lt;ol&gt;
&lt;li&gt;
&lt;p&gt;One must implement the rest of a …&lt;/p&gt;&lt;/li&gt;&lt;/ol&gt;</summary><content type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;p&gt;&lt;em&gt;Note: this blog was originally published &lt;a href="https://www.influxdata.com/blog/optimizing-sql-dataframes-part-one/"&gt;on the InfluxData blog&lt;/a&gt;&lt;/em&gt;&lt;/p&gt;
&lt;h2 id="introduction"&gt;Introduction&lt;a class="headerlink" href="#introduction" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;Sometimes Query Optimizers are seen as a sort of black magic, &lt;a href="https://15799.courses.cs.cmu.edu/spring2025/"&gt;“the most
challenging problem in computer
science,”&lt;/a&gt; according to Father
Pavlo, or some behind-the-scenes player. We believe this perception is because:&lt;/p&gt;
&lt;ol&gt;
&lt;li&gt;
&lt;p&gt;One must implement the rest of a database system (data storage, transactions,
   SQL parser, expression evaluation, plan execution, etc.) &lt;strong&gt;before&lt;/strong&gt; the
   optimizer becomes critical&lt;sup id="fn5"&gt;&lt;a href="#footnote5"&gt;5&lt;/a&gt;&lt;/sup&gt;.&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;
&lt;p&gt;Some parts of the optimizer are tightly tied to the rest of the system (e.g.,
   storage or indexes), so many classic optimizers are described with
   system-specific terminology.&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;
&lt;p&gt;Some optimizer tasks, such as access path selection and join order are known
   challenges and not yet solved (practically)—maybe they really do require
   black magic 🤔.&lt;/p&gt;
&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;However, Query Optimizers are no more complicated in theory or practice than other parts of a database system, as we will argue in a series of posts:&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Part 1: (this post)&lt;/strong&gt;:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;Review what a Query Optimizer is, what it does, and why you need one for SQL and DataFrames.&lt;/li&gt;
&lt;li&gt;Describe how industrial Query Optimizers are structured and standard optimization classes.&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;&lt;strong&gt;Part 2:&lt;/strong&gt;&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;Describe the optimization categories with examples and pointers to implementations.&lt;/li&gt;
&lt;li&gt;Describe &lt;a href="https://datafusion.apache.org/"&gt;Apache DataFusion&lt;/a&gt;’s rationale and approach to query optimization, specifically for access path and join ordering.&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;After reading these blogs, we hope people will use DataFusion to:&lt;/p&gt;
&lt;ol&gt;
&lt;li&gt;Build their own system specific optimizers.&lt;/li&gt;
&lt;li&gt;Perform practical academic research on optimization (especially researchers
   working on new optimizations / join ordering—looking at you &lt;a href="https://15799.courses.cs.cmu.edu/spring2025/"&gt;CMU
   15-799&lt;/a&gt;, next year).&lt;/li&gt;
&lt;/ol&gt;
&lt;h2 id="query-optimizer-background"&gt;Query Optimizer Background&lt;a class="headerlink" href="#query-optimizer-background" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;The key pitch for querying databases, and likely the key to the longevity of SQL
(despite people’s love/hate relationship—see &lt;a href="https://db.cs.cmu.edu/seminar2025/"&gt;SQL or Death? Seminar Series –
Spring 2025&lt;/a&gt;), is that it disconnects the
&lt;code&gt;WHAT&lt;/code&gt; you want to compute from the &lt;code&gt;HOW&lt;/code&gt; to do it. SQL is a &lt;em&gt;declarative&lt;/em&gt;
language—it describes what answers are desired rather than an &lt;em&gt;imperative&lt;/em&gt;
language such as Python, where you describe how to do the computation as shown
in Figure 1.&lt;/p&gt;
&lt;p&gt;&lt;img alt="Fig 1: Query Execution." class="img-fluid" src="/blog/images/optimizing-sql-dataframes/query-execution.png" width="80%"/&gt;&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Figure 1&lt;/strong&gt;: Query Execution: Users describe the answer they want using either
SQL or a DataFrame. For SQL, a Query Planner translates the parsed query 
into an &lt;em&gt;initial plan&lt;/em&gt;. The DataFrame API creates an initial plan directly.
The initial plan is correct, but slow. Then, the Query
Optimizer rewrites the initial plan into an &lt;em&gt;optimized plan&lt;/em&gt;, which computes
the same results but faster and more efficiently. Finally, the Execution Engine
executes the optimized plan producing results.&lt;/p&gt;
&lt;h2 id="sql-dataframes-logicalplan-equivalence"&gt;SQL, DataFrames, LogicalPlan Equivalence&lt;a class="headerlink" href="#sql-dataframes-logicalplan-equivalence" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;Given their name, it is not surprising that Query Optimizers can improve the
performance of SQL queries. However, it is under-appreciated that this also
applies to DataFrame style APIs.&lt;/p&gt;
&lt;p&gt;Classic DataFrame systems such as &lt;a href="https://pandas.pydata.org/"&gt;pandas&lt;/a&gt; and &lt;a href="https://pola.rs/"&gt;Polars&lt;/a&gt; (by default) execute
eagerly and thus have limited opportunities for optimization. However, more
modern APIs such as &lt;a href="https://docs.pola.rs/user-guide/lazy/using/"&gt;Polars' lazy API&lt;/a&gt;, &lt;a href="https://spark.apache.org/docs/latest/sql-programming-guide.html#datasets-and-dataframes),"&gt;Apache Spark's DataFrame&lt;/a&gt;. and
&lt;a href="https://datafusion.apache.org/user-guide/dataframe.html"&gt;DataFusion's DataFrame&lt;/a&gt; are much faster as they use the design shown in Figure
1 and apply many query optimization techniques.&lt;/p&gt;
&lt;h2 id="example-of-query-optimizer"&gt;Example of Query Optimizer&lt;a class="headerlink" href="#example-of-query-optimizer" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;This section motivates the value of a Query Optimizer with an example. Let’s say
you have some observations of animal behavior, as illustrated in Table 1.&lt;/p&gt;
&lt;p&gt;&lt;img alt="Table 1: Observational Data." class="img-fluid" src="/blog/images/optimizing-sql-dataframes/table1.png" width="75%"/&gt;&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Table 1&lt;/strong&gt;: Example observational data.&lt;/p&gt;
&lt;p&gt;If the user wants to know the average population for some species in the last
month, a user can write a SQL query or a DataFrame such as the following:&lt;/p&gt;
&lt;p&gt;SQL:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-sql"&gt;SELECT location, AVG(population)
FROM observations
WHERE species = ‘contrarian spider’ AND 
  observation_time &amp;gt;= now() - interval '1 month'
GROUP BY location
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;DataFrame:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-rust"&gt;df.scan("observations")
  .filter(col("species").eq("contrarian spider"))
  .filter(col("observation_time").ge(now()).sub(interval('1 month')))
  .agg(vec![col(location)], vec![avg(col("population")])
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;Within DataFusion, both the SQL and DataFrame are translated into the same
&lt;a href="https://docs.rs/datafusion/latest/datafusion/logical_expr/enum.LogicalPlan.html"&gt;LogicalPlan&lt;/a&gt;, a “tree of relational operators.” This is a fancy way of
saying data flow graphs where the edges represent tabular data (rows + columns)
and the nodes represent a transformation (see &lt;a href="https://youtu.be/EzZTLiSJnhY"&gt;this DataFusion overview video&lt;/a&gt;
for more details). The initial &lt;code&gt;LogicalPlan&lt;/code&gt; for the queries above is shown in
Figure 2.&lt;/p&gt;
&lt;p&gt;&lt;img alt="Fig 2: Initial Logical Plan." class="img-fluid" src="/blog/images/optimizing-sql-dataframes/initial-logical-plan.png" width="72%"/&gt;&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Figure 2&lt;/strong&gt;: Example initial &lt;code&gt;LogicalPlan&lt;/code&gt; for SQL and DataFrame query. The
plan is read from bottom to top, computing the results in each step.&lt;/p&gt;
&lt;p&gt;The optimizer's job is to take this query plan and rewrite it into an alternate
plan that computes the same results but faster, such as the one shown in Figure
3.&lt;/p&gt;
&lt;p&gt;&lt;img alt="Fig 3: Optimized Logical Plan." class="img-fluid" src="/blog/images/optimizing-sql-dataframes/optimized-logical-plan.png" width="80%"/&gt;&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Figure 3&lt;/strong&gt;: An example optimized plan that computes the same result as the
plan in Figure 2 more efficiently. The diagram highlights where the optimizer
has applied &lt;em&gt;Projection Pushdown&lt;/em&gt;, &lt;em&gt;Filter Pushdown&lt;/em&gt;, and &lt;em&gt;Constant Evaluation&lt;/em&gt;.
Note that this is a simplified example for explanatory purposes, and actual
optimizers such as the one in DataFusion perform additional tasks such as
choosing specific aggregation algorithms.&lt;/p&gt;
&lt;h2 id="query-optimizer-implementation"&gt;Query Optimizer Implementation&lt;a class="headerlink" href="#query-optimizer-implementation" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;Industrial optimizers, such as 
DataFusion’s (&lt;a href="https://github.com/apache/datafusion/tree/334d6ec50f36659403c96e1bffef4228be7c458e/datafusion/optimizer/src"&gt;source&lt;/a&gt;),
ClickHouse (&lt;a href="https://github.com/ClickHouse/ClickHouse/tree/master/src/Analyzer/Passes"&gt;source&lt;/a&gt;, &lt;a href="https://github.com/ClickHouse/ClickHouse/tree/master/src/Processors/QueryPlan/Optimizations"&gt;source&lt;/a&gt;),
DuckDB (&lt;a href="https://github.com/duckdb/duckdb/tree/4afa85c6a4dacc39524d1649fd8eb8c19c28ad14/src/optimizer"&gt;source&lt;/a&gt;),
and Apache Spark (&lt;a href="https://github.com/apache/spark/tree/7bc8e99cde424c59b98fe915e3fdaaa30beadb76/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer"&gt;source&lt;/a&gt;),
are implemented as a series of passes or rules that rewrite a query plan. The
overall optimizer is composed of a sequence of these rules,&lt;sup id="fn6"&gt;&lt;a href="#footnote6"&gt;6&lt;/a&gt;&lt;/sup&gt; as shown in
Figure 4. The specific order of the rules also often matters, but we will not
discuss this detail in this post.&lt;/p&gt;
&lt;p&gt;A multi-pass design is standard because it helps:&lt;/p&gt;
&lt;ol&gt;
&lt;li&gt;Understand, implement, and test each pass in isolation&lt;/li&gt;
&lt;li&gt;Easily extend the optimizer by adding new passes&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;&lt;img alt="Fig 4: Query Optimizer Passes." class="img-fluid" src="/blog/images/optimizing-sql-dataframes/optimizer-passes.png" width="80%"/&gt;&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Figure 4&lt;/strong&gt;: Query Optimizers are implemented as a series of rules that each
rewrite the query plan. Each rule’s algorithm is expressed as a transformation
of a previous plan.&lt;/p&gt;
&lt;p&gt;There are three major classes of optimizations in industrial optimizers:&lt;/p&gt;
&lt;ol&gt;
&lt;li&gt;
&lt;p&gt;&lt;strong&gt;Always Optimizations&lt;/strong&gt;: These are always good to do and thus are always
   applied. This class of optimization includes expression simplification,
   predicate pushdown, and limit pushdown. These optimizations are typically
   simple in theory, though they require nontrivial amounts of code and tests to
   implement in practice.&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;
&lt;p&gt;&lt;strong&gt;Engine Specific Optimizations: &lt;/strong&gt;These optimizations take advantage of
   specific engine features, such as how expressions are evaluated or what
   particular hash or join implementations are available.&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;
&lt;p&gt;&lt;strong&gt;Access Path and Join Order Selection&lt;/strong&gt;: These passes choose one access
   method per table and a join order for execution, typically using heuristics
   and a cost model to make tradeoffs between the options. Databases often have
   multiple ways to access the data (e.g., index scan or full-table scan), as
   well as many potential orders to combine (join) multiple tables. These
   methods compute the same result but can vary drastically in performance.&lt;/p&gt;
&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;This brings us to the end of Part 1. In Part 2, we will explain these classes of
optimizations in more detail and provide examples of how they are implemented in
DataFusion and other systems.&lt;/p&gt;
&lt;h1 id="about-the-authors"&gt;About the Authors&lt;a class="headerlink" href="#about-the-authors" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h1&gt;
&lt;p&gt;&lt;a href="https://www.linkedin.com/in/andrewalamb/"&gt;Andrew Lamb&lt;/a&gt; is a Staff Engineer at
&lt;a href="https://www.influxdata.com/"&gt;InfluxData&lt;/a&gt; and an &lt;a href="https://datafusion.apache.org/"&gt;Apache
DataFusion&lt;/a&gt; PMC member. A Database Optimizer
connoisseur, he worked on the &lt;a href="https://vldb.org/pvldb/vol5/p1790_andrewlamb_vldb2012.pdf"&gt;Vertica Analytic
Database&lt;/a&gt; Query
Optimizer for six years, has several granted US patents related to query
optimization&lt;sup id="fn1"&gt;&lt;a href="#footnote1"&gt;1&lt;/a&gt;&lt;/sup&gt;, co-authored several papers&lt;sup id="fn2"&gt;&lt;a href="#footnote2"&gt;2&lt;/a&gt;&lt;/sup&gt;  about the topic (including in
VLDB 2024&lt;sup id="fn3"&gt;&lt;a href="#footnote3"&gt;3&lt;/a&gt;&lt;/sup&gt;), and spent several weeks&lt;sup id="fn4"&gt;&lt;a href="#footnote4"&gt;4&lt;/a&gt;&lt;/sup&gt; deeply geeking out about this topic
with other experts (thank you Dagstuhl).&lt;/p&gt;
&lt;p&gt;&lt;a href="https://www.linkedin.com/in/akurmustafa/"&gt;Mustafa Akur&lt;/a&gt; is a PhD Student at
&lt;a href="https://www.ohsu.edu/"&gt;OHSU&lt;/a&gt; Knight Cancer Institute and an &lt;a href="https://datafusion.apache.org/"&gt;Apache
DataFusion&lt;/a&gt; PMC member. He was previously a
Software Developer at &lt;a href="https://www.synnada.ai/"&gt;Synnada&lt;/a&gt; where he contributed
significant features to the DataFusion optimizer, including many &lt;a href="https://datafusion.apache.org/blog/2025/03/11/ordering-analysis/"&gt;sort-based
optimizations&lt;/a&gt;.&lt;/p&gt;
&lt;h2 id="notes"&gt;Notes&lt;a class="headerlink" href="#notes" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;&lt;a id="footnote1"&gt;&lt;/a&gt;&lt;sup&gt;[1]&lt;/sup&gt; &lt;em&gt;Modular Query Optimizer, US 8,312,027 · Issued Nov 13, 2012&lt;/em&gt;, Query Optimizer with schema conversion US 8,086,598 · Issued Dec 27, 2011&lt;/p&gt;
&lt;p&gt;&lt;a id="footnote2"&gt;&lt;/a&gt;&lt;sup&gt;[2]&lt;/sup&gt; &lt;a href="https://www.researchgate.net/publication/269306314_The_Vertica_Query_Optimizer_The_case_for_specialized_query_optimizers"&gt;The Vertica Query Optimizer: The case for specialized Query Optimizers&lt;/a&gt;&lt;/p&gt;
&lt;p&gt;&lt;a id="footnote3"&gt;&lt;/a&gt;&lt;sup&gt;[3]&lt;/sup&gt; &lt;a href="https://www.vldb.org/pvldb/vol17/p1350-justen.pdf"&gt;https://www.vldb.org/pvldb/vol17/p1350-justen.pdf&lt;/a&gt;&lt;/p&gt;
&lt;p&gt;&lt;a id="footnote4"&gt;&lt;/a&gt;&lt;sup&gt;[4]&lt;/sup&gt; &lt;a href="https://www.dagstuhl.de/en/seminars/seminar-calendar/seminar-details/24101"&gt;https://www.dagstuhl.de/en/seminars/seminar-calendar/seminar-details/24101&lt;/a&gt;, &lt;a href="https://www.dagstuhl.de/en/seminars/seminar-calendar/seminar-details/22111"&gt;https://www.dagstuhl.de/en/seminars/seminar-calendar/seminar-details/22111&lt;/a&gt;, &lt;a href="https://www.dagstuhl.de/en/seminars/seminar-calendar/seminar-details/12321"&gt;https://www.dagstuhl.de/en/seminars/seminar-calendar/seminar-details/12321&lt;/a&gt;&lt;/p&gt;
&lt;p&gt;&lt;a id="footnote5"&gt;&lt;/a&gt;&lt;sup&gt;[5]&lt;/sup&gt;  And thus in academic classes, by the time you get around to an optimizer the semester is over and everyone is ready for the semester to be done. Once industrial systems mature to the point where the optimizer is a bottleneck, the shiny new-ness of the&lt;a href="https://en.wikipedia.org/wiki/Gartner_hype_cycle"&gt; hype cycle&lt;/a&gt; has worn off and it is likely in the trough of disappointment.&lt;/p&gt;
&lt;p&gt;&lt;a id="footnote6"&gt;&lt;/a&gt;&lt;sup&gt;[6]&lt;/sup&gt; Often systems will classify these passes into different categories, but I am simplifying here&lt;/p&gt;</content><category term="blog"/></entry><entry><title>Optimizing SQL (and DataFrames) in DataFusion, Part 2: Optimizers in Apache DataFusion</title><link href="https://datafusion.apache.org/blog/2025/06/15/optimizing-sql-dataframes-part-two" rel="alternate"/><published>2025-06-15T00:00:00+00:00</published><updated>2025-06-15T00:00:00+00:00</updated><author><name>alamb, akurmustafa</name></author><id>tag:datafusion.apache.org,2025-06-15:/blog/2025/06/15/optimizing-sql-dataframes-part-two</id><summary type="html">
&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;
&lt;p&gt;&lt;em&gt;Note, this blog was originally published &lt;a href="https://www.influxdata.com/blog/optimizing-sql-dataframes-part-two/"&gt;on the InfluxData blog&lt;/a&gt;.&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;In the &lt;a href="https://datafusion.apache.org/blog/2025/06/15/optimizing-sql-dataframes-part-one"&gt;first part of this post&lt;/a&gt;, we discussed what a Query Optimizer is, what
role it plays, and described how industrial optimizers are organized. In this
second post, we describe various optimizations that are found in &lt;a href="https://datafusion.apache.org/"&gt;Apache
DataFusion&lt;/a&gt; and …&lt;/p&gt;</summary><content type="html">
&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;
&lt;p&gt;&lt;em&gt;Note, this blog was originally published &lt;a href="https://www.influxdata.com/blog/optimizing-sql-dataframes-part-two/"&gt;on the InfluxData blog&lt;/a&gt;.&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;In the &lt;a href="https://datafusion.apache.org/blog/2025/06/15/optimizing-sql-dataframes-part-one"&gt;first part of this post&lt;/a&gt;, we discussed what a Query Optimizer is, what
role it plays, and described how industrial optimizers are organized. In this
second post, we describe various optimizations that are found in &lt;a href="https://datafusion.apache.org/"&gt;Apache
DataFusion&lt;/a&gt; and other industrial systems in more
detail.&lt;/p&gt;
&lt;p&gt;DataFusion contains high quality, full-featured implementations for &lt;em&gt;Always
Optimizations&lt;/em&gt; and &lt;em&gt;Engine Specific Optimizations&lt;/em&gt; (defined in Part 1).
Optimizers are implemented as rewrites of &lt;code&gt;LogicalPlan&lt;/code&gt; in the &lt;a href="https://github.com/apache/datafusion/tree/main/datafusion/optimizer"&gt;logical
optimizer&lt;/a&gt;
or rewrites of &lt;code&gt;ExecutionPlan&lt;/code&gt; in the &lt;a href="https://github.com/apache/datafusion/tree/main/datafusion/physical-optimizer"&gt;physical
optimizer&lt;/a&gt;.
This design means the same optimizer passes are applied for SQL queries,
DataFrame queries, as well as plans for other query language frontends such as
&lt;a href="https://github.com/influxdata/influxdb3_core/tree/26a30bf8d6e2b6b3f1dd905c4ec27e3db6e20d5f/iox_query_influxql"&gt;InfluxQL&lt;/a&gt;
in InfluxDB 3.0,
&lt;a href="https://github.com/GreptimeTeam/greptimedb/blob/0bd322a078cae4f128b791475ec91149499de33a/src/query/src/promql/planner.rs#L1"&gt;PromQL&lt;/a&gt;
in &lt;a href="https://greptime.com/"&gt;Greptime&lt;/a&gt;, and
&lt;a href="https://github.com/vega/vegafusion/tree/dc15c1b9fc7d297f12bea919795d58cda1c88fcf/vegafusion-core/src/planning"&gt;vega&lt;/a&gt;
in &lt;a href="https://vegafusion.io/"&gt;VegaFusion&lt;/a&gt;.&lt;/p&gt;
&lt;h2 id="always-optimizations"&gt;Always Optimizations&lt;a class="headerlink" href="#always-optimizations" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;Some optimizations are so important they are found in almost all query engines
and are typically the first implemented as they provide the largest cost /
benefit ratio (and performance is terrible without them).&lt;/p&gt;
&lt;h3 id="predicatefilter-pushdown"&gt;Predicate/Filter Pushdown&lt;a class="headerlink" href="#predicatefilter-pushdown" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;&lt;strong&gt;Why&lt;/strong&gt;: Avoid carrying unneeded &lt;em&gt;rows &lt;/em&gt;as soon as possible&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;What&lt;/strong&gt;: Moves filters “down” in the plan so they run earlier during execution, as shown in Figure 1.&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Example Implementations&lt;/strong&gt;: &lt;a href="https://github.com/apache/datafusion/blob/main/datafusion/optimizer/src/push_down_filter.rs"&gt;DataFusion&lt;/a&gt;, &lt;a href="https://github.com/duckdb/duckdb/blob/main/src/optimizer/filter_pushdown.cpp"&gt;DuckDB&lt;/a&gt;, &lt;a href="https://github.com/ClickHouse/ClickHouse/blob/master/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp"&gt;ClickHouse&lt;/a&gt;&lt;/p&gt;
&lt;p&gt;The earlier data is filtered out in the plan, the less work the rest of the plan
has to do. Most mature databases aggressively use filter pushdown / early
filtering combined with techniques such as partition and storage pruning (e.g.
&lt;a href="https://blog.xiangpeng.systems/posts/parquet-to-arrow/"&gt;Parquet Row Group pruning&lt;/a&gt;) for performance.&lt;/p&gt;
&lt;p&gt;An extreme, and somewhat contrived, is the query&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-sql"&gt;SELECT city, COUNT(*) FROM population GROUP BY city HAVING city = 'BOSTON';
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;Semantically, &lt;code&gt;HAVING&lt;/code&gt; is &lt;a href="https://www.datacamp.com/tutorial/sql-order-of-execution"&gt;evaluated after&lt;/a&gt; &lt;code&gt;GROUP BY&lt;/code&gt; in SQL. However, computing
the population of all cities and discarding everything except Boston is much
slower than only computing the population for Boston and so most Query
Optimizers will evaluate the filter before the aggregation.&lt;/p&gt;
&lt;p&gt;&lt;img alt="Fig 1: Filter Pushdown." class="img-fluid" src="/blog/images/optimizing-sql-dataframes/filter-pushdown.png" width="80%"/&gt;&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Figure 1&lt;/strong&gt;: Filter Pushdown.  In (&lt;strong&gt;A&lt;/strong&gt;) without filter pushdown, the operator
processes more rows, reducing efficiency. In (&lt;strong&gt;B&lt;/strong&gt;) with filter pushdown, the
operator receives fewer rows, resulting in less overall work and leading to a
faster and more efficient query.&lt;/p&gt;
&lt;h3 id="projection-pushdown"&gt;Projection Pushdown&lt;a class="headerlink" href="#projection-pushdown" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;&lt;strong&gt;Why&lt;/strong&gt;: Avoid carrying unneeded &lt;em&gt;columns &lt;/em&gt;as soon as possible&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;What: &lt;/strong&gt;Pushes “projection” (keeping only certain columns) earlier in the plan, as shown in Figure 2.&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Example Implementations: &lt;/strong&gt;Implementations: &lt;a href="https://github.com/apache/datafusion/blob/main/datafusion/physical-optimizer/src/projection_pushdown.rs"&gt;DataFusion&lt;/a&gt;, &lt;a href="https://github.com/duckdb/duckdb/blob/a8a6a080c8809d5d4b3c955e9f113574f6f0bfe0/src/optimizer/pushdown/pushdown_projection.cpp"&gt;DuckDB&lt;/a&gt;, &lt;a href="https://github.com/ClickHouse/ClickHouse/blob/master/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp"&gt;ClickHouse&lt;/a&gt;&lt;/p&gt;
&lt;p&gt;Similarly to the motivation for &lt;em&gt;Filter Pushdown&lt;/em&gt;, the earlier the plan stops
doing something, the less work it does overall and thus the faster it runs. For
Projection Pushdown, if columns are not needed later in a plan, copying the data
to the output of other operators is unnecessary and the costs of copying can add
up. For example, in Figure 3 of Part 1, the &lt;code&gt;species&lt;/code&gt; column is only needed to
evaluate the Filter within the scan and &lt;code&gt;notes&lt;/code&gt; are never used, so it is
unnecessary to copy them through the rest of the plan.&lt;/p&gt;
&lt;p&gt;Projection Pushdown is especially effective and important for column store
databases, where the storage format itself (such as &lt;a href="https://parquet.apache.org/"&gt;Apache Parquet&lt;/a&gt;) supports
efficiently reading only a subset of required columns, and is &lt;a href="https://blog.xiangpeng.systems/posts/parquet-pushdown/"&gt;especially
powerful in combination with filter pushdown&lt;/a&gt;. Projection Pushdown is still
important, but less effective for row oriented formats such as JSON or CSV where
each column in each row must be parsed even if it is not used in the plan.&lt;/p&gt;
&lt;p&gt;&lt;img alt="Fig 2: Projection Pushdown." class="img-fluid" src="/blog/images/optimizing-sql-dataframes/projection-pushdown.png" width="80%"/&gt;&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Figure 2:&lt;/strong&gt; In (&lt;strong&gt;A&lt;/strong&gt;) without projection pushdown, the operator receives more
columns, reducing efficiency. In (&lt;strong&gt;B&lt;/strong&gt;) with projection pushdown, the operator
receives fewer columns, leading to optimized execution.&lt;/p&gt;
&lt;h3 id="limit-pushdown"&gt;Limit Pushdown&lt;a class="headerlink" href="#limit-pushdown" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;&lt;strong&gt;Why&lt;/strong&gt;: The earlier the plan stops generating data, the less overall work it
does, and some operators have more efficient limited implementations.&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;What: &lt;/strong&gt;Pushes limits (maximum row counts) down in a plan as early as possible.&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Example Implementations:&lt;/strong&gt; &lt;a href="https://github.com/apache/datafusion/blob/main/datafusion/optimizer/src/push_down_limit.rs"&gt;DataFusion&lt;/a&gt;, &lt;a href="https://github.com/duckdb/duckdb/blob/main/src/optimizer/limit_pushdown.cpp"&gt;DuckDB&lt;/a&gt;, &lt;a href="https://github.com/ClickHouse/ClickHouse/blob/master/src/Processors/QueryPlan/Optimizations/limitPushDown.cpp"&gt;ClickHouse&lt;/a&gt;, Spark (&lt;a href="https://github.com/apache/spark/blob/7bc8e99cde424c59b98fe915e3fdaaa30beadb76/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/LimitPushDownThroughWindow.scala"&gt;Window&lt;/a&gt; and &lt;a href="https://github.com/apache/spark/blob/7bc8e99cde424c59b98fe915e3fdaaa30beadb76/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PushProjectionThroughLimit.scala"&gt;Projection&lt;/a&gt;)&lt;/p&gt;
&lt;p&gt;Often queries have a &lt;code&gt;LIMIT&lt;/code&gt; or other clause that allows them to stop generating
results early so the sooner they can stop execution, the more efficiently they
will execute.&lt;/p&gt;
&lt;p&gt;In addition, DataFusion and other systems have more efficient implementations of
some operators that can be used if there is a limit. The classic example is
replacing a full sort + limit with a &lt;a href="https://docs.rs/datafusion/latest/datafusion/physical_plan/struct.TopK.html"&gt;TopK&lt;/a&gt; operator that only tracks the top
values using a heap. Similarly,  DataFusion’s Parquet reader stops fetching and
opening additional files once the limit has been hit.&lt;/p&gt;
&lt;p&gt;&lt;img alt="Fig 3: Limit Pushdown." class="img-fluid" src="/blog/images/optimizing-sql-dataframes/limit-pushdown.png" width="80%"/&gt;&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Figure 3&lt;/strong&gt;: In (&lt;strong&gt;A&lt;/strong&gt;), without limit pushdown all data is sorted and
everything except the first few rows are discarded. In (&lt;strong&gt;B&lt;/strong&gt;), with limit
pushdown, Sort is replaced with TopK operator which does much less work.&lt;/p&gt;
&lt;h3 id="expression-simplification-constant-folding"&gt;Expression Simplification / Constant Folding&lt;a class="headerlink" href="#expression-simplification-constant-folding" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;&lt;strong&gt;Why&lt;/strong&gt;: Evaluating the same expression for each row when the value doesn’t change is wasteful.&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;What&lt;/strong&gt;: Partially evaluates and/or algebraically simplify expressions.&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Example Implementations:&lt;/strong&gt; &lt;a href="https://github.com/apache/datafusion/tree/main/datafusion/optimizer/src/simplify_expressions"&gt;DataFusion&lt;/a&gt;, DuckDB (has several &lt;a href="https://github.com/duckdb/duckdb/tree/7b18f0f3691c1b6367cf68ed2598d7034e14f41b/src/optimizer/rule"&gt;rules&lt;/a&gt; such as &lt;a href="https://github.com/duckdb/duckdb/blob/7b18f0f3691c1b6367cf68ed2598d7034e14f41b/src/optimizer/rule/constant_folding.cpp"&gt;constant folding&lt;/a&gt;, and &lt;a href="https://github.com/duckdb/duckdb/blob/7b18f0f3691c1b6367cf68ed2598d7034e14f41b/src/optimizer/rule/comparison_simplification.cpp"&gt;comparison simplification&lt;/a&gt;), &lt;a href="https://github.com/apache/spark/blob/7bc8e99cde424c59b98fe915e3fdaaa30beadb76/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala"&gt;Spark&lt;/a&gt;&lt;/p&gt;
&lt;p&gt;If an expression doesn’t change from row to row, it is better to evaluate the
expression &lt;strong&gt;once&lt;/strong&gt; during planning. This is a classic compiler technique and is
also used in database systems&lt;/p&gt;
&lt;p&gt;For example, given a query that finds all values from the current year&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-sql"&gt;SELECT … WHERE extract(year from time_column) = extract(year from now())
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;Evaluating &lt;code&gt;extract(year from now())&lt;/code&gt; on every row is much more expensive than
evaluating it once during planning time so that the query becomes comparison to
a constant&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-sql"&gt;SELECT … WHERE extract(year from time_column) = 2025
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;Furthermore, it is often possible to push such predicates &lt;strong&gt;into&lt;/strong&gt; scans.&lt;/p&gt;
&lt;h3 id="rewriting-outer-join-inner-join"&gt;Rewriting &lt;code&gt;OUTER JOIN&lt;/code&gt; → &lt;code&gt;INNER JOIN&lt;/code&gt;&lt;a class="headerlink" href="#rewriting-outer-join-inner-join" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;&lt;strong&gt;Why:&lt;/strong&gt; &lt;code&gt;INNER JOIN&lt;/code&gt;  implementations are almost always faster (as they are
simpler) than &lt;code&gt;OUTER JOIN&lt;/code&gt; implementations, and &lt;code&gt;INNER JOIN&lt;/code&gt; s impose fewer
restrictions on other optimizer passes (such as join reordering and additional
filter pushdown).&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;What&lt;/strong&gt;: In cases where it is known that NULL rows introduced by an &lt;code&gt;OUTER
JOIN&lt;/code&gt; will not appear in the results, it can be rewritten to an &lt;code&gt;INNER
JOIN&lt;/code&gt;.&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Example Implementations:&lt;/strong&gt; &lt;a href="https://github.com/apache/datafusion/blob/6028474969f0bfead96eb7f413791470afb6bf82/datafusion/optimizer/src/eliminate_outer_join.rs"&gt;DataFusion&lt;/a&gt;, &lt;a href="https://github.com/apache/spark/blob/7bc8e99cde424c59b98fe915e3fdaaa30beadb76/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/joins.scala#L124-L158"&gt;Spark&lt;/a&gt;, &lt;a href="https://github.com/ClickHouse/ClickHouse/blob/master/src/Processors/QueryPlan/Optimizations/convertOuterJoinToInnerJoin.cpp"&gt;ClickHouse&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;For example, given a query such as the following&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-SQL"&gt;SELECT …
FROM orders LEFT OUTER JOIN customer ON (orders.cid = customer.id)
WHERE customer.last_name = 'Lamb'
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;The &lt;code&gt;LEFT OUTER JOIN&lt;/code&gt; keeps all rows in &lt;code&gt;orders&lt;/code&gt;  that don’t have a matching
customer, but fills in the fields with &lt;code&gt;null&lt;/code&gt;. All such rows will be filtered
out by &lt;code&gt;customer.last_name = 'Lamb'&lt;/code&gt;, and thus an INNER JOIN produces the same
answer. This is illustrated in Figure 4.&lt;/p&gt;
&lt;p&gt;&lt;img alt="Fig 4: Join Rewrite." class="img-fluid" src="/blog/images/optimizing-sql-dataframes/join-rewrite.png" width="80%"/&gt;&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Figure 4&lt;/strong&gt;: Rewriting &lt;code&gt;OUTER JOIN&lt;/code&gt; to &lt;code&gt;INNER JOIN&lt;/code&gt;. In (A) the original query
contains an &lt;code&gt;OUTER JOIN&lt;/code&gt; but also a filter on &lt;code&gt;customer.last_name&lt;/code&gt;, which
filters out all rows that might be introduced by the &lt;code&gt;OUTER JOIN&lt;/code&gt;. In (B) the
&lt;code&gt;OUTER JOIN&lt;/code&gt; is converted to inner join, a more efficient implementation can be
used.&lt;/p&gt;
&lt;h2 id="engine-specific-optimizations"&gt;Engine Specific Optimizations&lt;a class="headerlink" href="#engine-specific-optimizations" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;As discussed in Part 1 of this blog, optimizers also contain a set of passes
that are still always good to do, but are closely tied to the specifics of the
query engine. This section describes some common types&lt;/p&gt;
&lt;h3 id="subquery-rewrites"&gt;Subquery Rewrites&lt;a class="headerlink" href="#subquery-rewrites" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;&lt;strong&gt;Why&lt;/strong&gt;: Actually implementing subqueries by running a query for each row of the outer query is very expensive.&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;What&lt;/strong&gt;: It is possible to rewrite subqueries as joins which often perform much better.&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Example Implementations:&lt;/strong&gt; DataFusion (&lt;a href="https://github.com/apache/datafusion/blob/main/datafusion/optimizer/src/decorrelate.rs"&gt;one&lt;/a&gt;, &lt;a href="https://github.com/apache/datafusion/blob/main/datafusion/optimizer/src/decorrelate_predicate_subquery.rs"&gt;two&lt;/a&gt;, &lt;a href="https://github.com/apache/datafusion/blob/main/datafusion/optimizer/src/scalar_subquery_to_join.rs"&gt;three&lt;/a&gt;), &lt;a href="https://github.com/apache/spark/blob/7bc8e99cde424c59b98fe915e3fdaaa30beadb76/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala"&gt;Spark&lt;/a&gt;&lt;/p&gt;
&lt;p&gt;Evaluating subqueries a row at a time is so expensive that execution engines in
high performance analytic systems such as DataFusion and &lt;a href="https://vertica.com/"&gt;Vertica&lt;/a&gt; may not even
support row-at-a-time evaluation given how terrible the performance would be. 
Instead, analytic systems rewrite such queries into joins which can perform 100s
or 1000s of times faster for large datasets. However, transforming subqueries to
joins requires “exotic” join semantics such as &lt;code&gt;SEMI JOIN&lt;/code&gt;, &lt;code&gt;ANTI JOIN&lt;/code&gt;  and
variations on how to treat equality with null&lt;sup id="fn7"&gt;&lt;a href="#footnote7"&gt;7&lt;/a&gt;.&lt;/sup&gt;&lt;/p&gt;
&lt;p&gt;For a simple example, consider that a query like this:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-sql"&gt;SELECT customer.name 
FROM customer 
WHERE (SELECT sum(value) 
       FROM orders WHERE
       orders.cid = customer.id) &amp;gt; 10;
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;Can be rewritten like this:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-sql"&gt;SELECT customer.name 
FROM customer 
JOIN (
  SELECT customer.id as cid_inner, sum(value) s 
  FROM orders 
  GROUP BY customer.id
 ) ON (customer.id = cid_inner AND s &amp;gt; 10);
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;We don’t have space to detail this transformation or why it is so much faster to
run, but using this and many other transformations allow efficient subquery
evaluation.&lt;/p&gt;
&lt;h3 id="optimized-expression-evaluation"&gt;Optimized Expression Evaluation&lt;a class="headerlink" href="#optimized-expression-evaluation" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;&lt;strong&gt;Why&lt;/strong&gt;: The capabilities of expression evaluation vary from system to system.&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;What&lt;/strong&gt;: Optimize expression evaluation for the particular execution environment.&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Example Implementations&lt;/strong&gt;: There are many examples of this type of
optimization, including DataFusion’s &lt;a href="https://github.com/apache/datafusion/blob/main/datafusion/optimizer/src/common_subexpr_eliminate.rs"&gt;Common Subexpression
Elimination&lt;/a&gt;,
&lt;a href="https://github.com/apache/datafusion/blob/8f3f70877febaa79be3349875e979d3a6e65c30e/datafusion/optimizer/src/simplify_expressions/unwrap_cast.rs#L70"&gt;unwrap_cast&lt;/a&gt;,
and &lt;a href="https://github.com/apache/datafusion/blob/main/datafusion/optimizer/src/extract_equijoin_predicate.rs"&gt;identifying equality join
predicates&lt;/a&gt;.
DuckDB &lt;a href="https://github.com/duckdb/duckdb/blob/main/src/optimizer/in_clause_rewriter.cpp"&gt;rewrites IN
clauses&lt;/a&gt;,
and &lt;a href="https://github.com/duckdb/duckdb/blob/main/src/optimizer/sum_rewriter.cpp"&gt;SUM
expressions&lt;/a&gt;.
Spark also &lt;a href="https://github.com/apache/spark/blob/7bc8e99cde424c59b98fe915e3fdaaa30beadb76/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/UnwrapCastInBinaryComparison.scala"&gt;unwraps casts in binary
comparisons&lt;/a&gt;,
and &lt;a href="https://github.com/apache/spark/blob/7bc8e99cde424c59b98fe915e3fdaaa30beadb76/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InjectRuntimeFilter.scala"&gt;adds special runtime
filters&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;To give a specific example of what DataFusion’s common subexpression elimination
does, consider this query that refers to a complex expression multiple times:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-sql"&gt;SELECT date_bin('1 hour', time, '1970-01-01') 
FROM table 
WHERE date_bin('1 hour', time, '1970-01-01') &amp;gt;= '2025-01-01 00:00:00'
ORDER BY date_bin('1 hour', time, '1970-01-01')
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;Evaluating &lt;code&gt;date_bin('1 hour', time, '1970-01-01')&lt;/code&gt;each time it is encountered
is inefficient compared to calculating its result once, and reusing that result
in when it is encountered again (similar to caching). This reuse is called
&lt;em&gt;Common Subexpression Elimination&lt;/em&gt;.&lt;/p&gt;
&lt;p&gt;Some execution engines implement this optimization internally to their
expression evaluation engine, but DataFusion represents it explicitly using a
separate Projection plan node, as illustrated in Figure 5.  Effectively, the
query above is rewritten to the following&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-sql"&gt;SELECT time_chunk 
FROM(SELECT date_bin('1 hour', time, '1970-01-01') as time_chunk 
     FROM table)
WHERE time_chunk &amp;gt;= '2025-01-01 00:00:00'
ORDER BY time_chunk
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;&lt;img alt="Fig 5: Common Subquery Elimination." class="img-fluid" src="/blog/images/optimizing-sql-dataframes/common-subexpression-elimination.png" width="80%"/&gt;&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Figure 5:&lt;/strong&gt; Adding a Projection to evaluate common complex sub expression
decreases complexity for later stages.&lt;/p&gt;
&lt;h3 id="algorithm-selection"&gt;Algorithm Selection&lt;a class="headerlink" href="#algorithm-selection" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;&lt;strong&gt;Why&lt;/strong&gt;: Different engines have different specialized operators for certain
operations.&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;What: &lt;/strong&gt;Selects specific implementations from the available operators, based
on properties of the query.&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Example Implementations:&lt;/strong&gt; DataFusion’s &lt;a href="https://github.com/apache/datafusion/blob/8f3f70877febaa79be3349875e979d3a6e65c30e/datafusion/physical-optimizer/src/enforce_sorting/mod.rs"&gt;EnforceSorting&lt;/a&gt; pass uses sort optimized implementations, Spark’s &lt;a href="https://github.com/apache/spark/blob/7bc8e99cde424c59b98fe915e3fdaaa30beadb76/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteAsOfJoin.scala"&gt;rewrite to use a special operator for ASOF joins&lt;/a&gt;, and ClickHouse’s&lt;a href="https://github.com/ClickHouse/ClickHouse/blob/7d15deda4b33282f356bb3e40a190d005acf72f2/src/Interpreters/ExpressionAnalyzer.cpp#L1066-L1080"&gt; join algorithm selection &lt;/a&gt; such as &lt;a href="https://github.com/ClickHouse/ClickHouse/blob/7d15deda4b33282f356bb3e40a190d005acf72f2/src/Interpreters/ExpressionAnalyzer.cpp#L1022"&gt;when to use MergeJoin&lt;/a&gt;&lt;/p&gt;
&lt;p&gt;For example, DataFusion uses a &lt;code&gt;TopK&lt;/code&gt; (&lt;a href="https://docs.rs/datafusion/latest/datafusion/physical_plan/struct.TopK.html"&gt;source&lt;/a&gt;) operator rather than a full
&lt;code&gt;Sort&lt;/code&gt; if there is also a limit on the query. Similarly, it may choose to use the
more efficient &lt;code&gt;PartialOrdered&lt;/code&gt; grouping operation when the data is sorted on
group keys or a &lt;code&gt;MergeJoin&lt;/code&gt;&lt;/p&gt;
&lt;p&gt;&lt;img alt="Fig 6: Specialized Grouping." class="img-fluid" src="/blog/images/optimizing-sql-dataframes/specialized-grouping.png" width="80%"/&gt;&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Figure 6: &lt;/strong&gt;An example of specialized operation for grouping. In (&lt;strong&gt;A&lt;/strong&gt;), input data has no specified ordering and DataFusion uses a hashing-based grouping operator (&lt;a href="https://github.com/apache/datafusion/blob/main/datafusion/physical-plan/src/aggregates/row_hash.rs"&gt;source&lt;/a&gt;) to determine distinct groups. In (&lt;strong&gt;B&lt;/strong&gt;), when the input data is ordered by the group keys, DataFusion uses a specialized grouping operator (&lt;a href="https://github.com/apache/datafusion/tree/main/datafusion/physical-plan/src/aggregates/order"&gt;source&lt;/a&gt;) to find boundaries that separate groups.&lt;/p&gt;
&lt;h3 id="using-statistics-directly"&gt;Using Statistics Directly&lt;a class="headerlink" href="#using-statistics-directly" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;&lt;strong&gt;Why&lt;/strong&gt;: Using pre-computed statistics from a table, without actually reading or
opening files, is much faster than processing data.&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;What&lt;/strong&gt;: Replace calculations on data with the value from statistics.&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Example Implementations:&lt;/strong&gt; &lt;a href="https://github.com/apache/datafusion/blob/8f3f70877febaa79be3349875e979d3a6e65c30e/datafusion/physical-optimizer/src/aggregate_statistics.rs"&gt;DataFusion&lt;/a&gt;, &lt;a href="https://github.com/duckdb/duckdb/blob/main/src/optimizer/statistics_propagator.cpp"&gt;DuckDB&lt;/a&gt;,&lt;/p&gt;
&lt;p&gt;Some queries, such as the classic &lt;code&gt;COUNT(*) from my_table&lt;/code&gt; used for data
exploration can be answered using only statistics. Optimizers often have access
to statistics for other reasons (such as Access Path and Join Order Selection)
and statistics are commonly stored in analytic file formats. For example, the
&lt;a href="https://docs.rs/parquet/latest/parquet/file/metadata/index.html"&gt;Metadata&lt;/a&gt; of Apache Parquet files stores &lt;code&gt;MIN&lt;/code&gt;, &lt;code&gt;MAX&lt;/code&gt;, and &lt;code&gt;COUNT&lt;/code&gt; information.&lt;/p&gt;
&lt;p&gt;&lt;img alt="Fig 7: Using Statistics." class="img-fluid" src="/blog/images/optimizing-sql-dataframes/using-statistics.png" width="80%"/&gt;&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Figure 7: &lt;/strong&gt;When the aggregation result is already stored in the statistics,
the query can be evaluated using the values from statistics without looking at
any compressed data. The optimizer replaces the Aggregation operation with
values from statistics.&lt;/p&gt;
&lt;h2 id="access-path-and-join-order-selection"&gt;Access Path and Join Order Selection&lt;a class="headerlink" href="#access-path-and-join-order-selection" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;h3 id="overview"&gt;Overview&lt;a class="headerlink" href="#overview" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;Last, but certainly not least, are optimizations that choose between plans with
potentially (very) different performance. The major options in this category are&lt;/p&gt;
&lt;ol&gt;
&lt;li&gt;&lt;strong&gt;Join Order:&lt;/strong&gt; In what order to combine tables using JOINs?&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Access Paths:&lt;/strong&gt; Which copy of the data or index should be read to find matching tuples?&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;&lt;a href="https://en.wikipedia.org/wiki/Materialized_view"&gt;Materialized View&lt;/a&gt;&lt;/strong&gt;: Can the query can be rewritten to use a materialized view (partially computed query results)? This topic deserves its own blog (or book) and we don’t discuss further here.&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;&lt;img alt="Fig 8: Access Path and Join Order." class="img-fluid" src="/blog/images/optimizing-sql-dataframes/access-path-and-join-order.png" width="80%"/&gt;&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Figure 8:&lt;/strong&gt; Access Path and Join Order Selection in Query Optimizers. Optimizers use heuristics to enumerate some subset of potential join orders (shape) and access paths (color). The plan with the smallest estimated cost according to some cost model is chosen. In this case, Plan 2 with a cost of 180,000 is chosen for execution as it has the lowest estimated cost.&lt;/p&gt;
&lt;p&gt;This class of optimizations is a hard problem for at least the following reasons:&lt;/p&gt;
&lt;ol&gt;
&lt;li&gt;
&lt;p&gt;&lt;strong&gt;Exponential Search Space&lt;/strong&gt;: the number of potential plans increases
   exponentially as the number of joins and indexes increases.&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;
&lt;p&gt;&lt;strong&gt;Performance Sensitivity&lt;/strong&gt;: Often different plans that are very similar in
   structure perform very differently. For example, swapping the input order to
   a hash join can result in 1000x or more (yes, a thousand-fold!) run time
   differences.&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;
&lt;p&gt;&lt;strong&gt;Cardinality Estimation Errors&lt;/strong&gt;: Determining the optimal plan relies on
   cardinality estimates (e.g., how many rows will come out of each join). It is a
   &lt;a href="https://www.vldb.org/pvldb/vol9/p204-leis.pdf"&gt;known hard problem&lt;/a&gt; to estimate this cardinality, and in practice queries with
   as few as 3 joins often have large cardinality estimation errors.&lt;/p&gt;
&lt;/li&gt;
&lt;/ol&gt;
&lt;h3 id="heuristics-and-cost-based-optimization"&gt;Heuristics and Cost-Based Optimization&lt;a class="headerlink" href="#heuristics-and-cost-based-optimization" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;Industrial optimizers handle these problems using a combination of&lt;/p&gt;
&lt;ol&gt;
&lt;li&gt;
&lt;p&gt;&lt;strong&gt;Heuristics:&lt;/strong&gt; to prune the search space and avoid considering plans that
   are (almost) never good. Examples include considering left-deep trees, or
   using &lt;code&gt;Foreign Key&lt;/code&gt; / &lt;code&gt;Primary Key&lt;/code&gt; relationships to pick the build size of a
   hash join.&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;
&lt;p&gt;&lt;strong&gt;Cost Model&lt;/strong&gt;: Given the smaller set of candidate plans, the Optimizer then
   estimates their cost and picks the one using the lowest cost.&lt;/p&gt;
&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;For some examples, you can read about &lt;a href="https://docs.databricks.com/aws/en/optimizations/cbo"&gt;Spark’s cost-based optimizer&lt;/a&gt; or look at
the code for &lt;a href="https://github.com/apache/datafusion/blob/main/datafusion/physical-optimizer/src/join_selection.rs"&gt;DataFusion’s join selection&lt;/a&gt; and &lt;a href="https://github.com/duckdb/duckdb/blob/main/src/optimizer/join_order/cost_model.cpp"&gt;DuckDB’s cost model&lt;/a&gt; and &lt;a href="https://github.com/duckdb/duckdb/blob/84c87b12fa9554a8775dc243b4d0afd5b407321a/src/optimizer/join_order/plan_enumerator.cpp#L469-L472"&gt;join
order enumeration&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;However, the use of heuristics and (imprecise) cost models means optimizers must&lt;/p&gt;
&lt;ol&gt;
&lt;li&gt;
&lt;p&gt;&lt;strong&gt;Make deep assumptions about the execution environment: &lt;/strong&gt;For example the
   heuristics often include assumptions that joins implement &lt;a href="https://www.alibabacloud.com/blog/alibaba-cloud-analyticdb-for-mysql-create-ultimate-runtimefilter-capability_600228"&gt;sideways information
   passing (RuntimeFilters)&lt;/a&gt;, or that Join operators always preserve a particular
   input's order.&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;
&lt;p&gt;&lt;strong&gt;Use one particular objective function: &lt;/strong&gt;There are almost always trade-offs
   between desirable plan properties, such as execution speed, memory use, and
   robustness in the face of cardinality estimation. Industrial optimizers
   typically have one cost function which attempts to balance between the
   properties or a series of hard to use indirect tuning knobs to control the
   behavior.&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;
&lt;p&gt;&lt;strong&gt;Require statistics&lt;/strong&gt;: Typically cost models require up-to-date statistics,
   which can be expensive to compute, must be kept up to date as new data
   arrives, and often have trouble capturing the non-uniformity of real world
   datasets&lt;/p&gt;
&lt;/li&gt;
&lt;/ol&gt;
&lt;h3 id="join-ordering-in-datafusion"&gt;Join Ordering in DataFusion&lt;a class="headerlink" href="#join-ordering-in-datafusion" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;DataFusion purposely does not include a sophisticated cost based optimizer.
Instead, keeping with its &lt;a href="https://docs.rs/datafusion/latest/datafusion/#design-goals"&gt;design goals&lt;/a&gt; it provides a reasonable default
implementation along with extension points to customize behavior.&lt;/p&gt;
&lt;p&gt;Specifically, DataFusion includes&lt;/p&gt;
&lt;ol&gt;
&lt;li&gt;“Syntactic Optimizer” (joins in the order they are listed in the query&lt;sup id="fn8"&gt;&lt;a href="#footnote8"&gt;8&lt;/a&gt;) with basic join re-ordering (&lt;a href="https://github.com/apache/datafusion/blob/main/datafusion/physical-optimizer/src/join_selection.rs"&gt;source&lt;/a&gt;) to prevent join disasters.&lt;/sup&gt;&lt;/li&gt;
&lt;li&gt;Support for &lt;a href="https://docs.rs/datafusion/latest/datafusion/common/struct.ColumnStatistics.html"&gt;ColumnStatistics&lt;/a&gt; and &lt;a href="https://docs.rs/datafusion/latest/datafusion/common/struct.Statistics.html"&gt;Table Statistics&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;The framework for &lt;a href="https://docs.rs/datafusion/latest/datafusion/physical_expr/struct.AnalysisContext.html#structfield.selectivity"&gt;filter selectivity&lt;/a&gt; + join cardinality estimation.&lt;/li&gt;
&lt;li&gt;APIs for easily rewriting plans, such as the &lt;a href="https://docs.rs/datafusion/latest/datafusion/common/tree_node/trait.TreeNode.html#overview"&gt;TreeNode API&lt;/a&gt; and &lt;a href="https://docs.rs/datafusion/latest/datafusion/physical_plan/joins/struct.HashJoinExec.html#method.swap_inputs"&gt;reordering joins&lt;/a&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;This combination of features along with &lt;a href="https://docs.rs/datafusion/latest/datafusion/execution/session_state/struct.SessionStateBuilder.html#method.with_physical_optimizer_rule"&gt;custom optimizer passes&lt;/a&gt; lets users
customize the behavior to their use case, such as custom indexes like &lt;a href="https://uwheel.rs/post/datafusion_uwheel/"&gt;uWheel&lt;/a&gt;
and &lt;a href="https://github.com/datafusion-contrib/datafusion-materialized-views"&gt;materialized views&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;The rationale for including only a basic optimizer is that any one particular
set of heuristics and cost model is unlikely to work well for the wide variety
of DataFusion users because of the tradeoffs involved. &lt;/p&gt;
&lt;p&gt;For example, some users may always have access to adequate resources, and want
the fastest query execution, and are willing to tolerate runtime errors or a
performance cliff when there is insufficient memory. Other users, however, may
be willing to accept a slower maximum performance in return for more predictable
performance when running in a resource constrained environment. This approach is
not universally agreed. One of us has &lt;a href="https://www.researchgate.net/publication/269306314_The_Vertica_Query_Optimizer_The_case_for_specialized_query_optimizers"&gt;previously argued the case for
specialized optimizers&lt;/a&gt; in a more academic paper, and the topic comes up
regularly in the DataFusion community, (e.g. &lt;a href="https://github.com/apache/datafusion/issues/9846#issuecomment-2566568654"&gt;this recent comment&lt;/a&gt;).&lt;/p&gt;
&lt;p&gt;Note: We are &lt;a href="https://github.com/apache/datafusion/issues/3929"&gt;actively improving&lt;/a&gt; this part of the code to help people write
their own optimizers (🎣 come help us define and implement it!)&lt;/p&gt;
&lt;h1 id="conclusion"&gt;Conclusion&lt;a class="headerlink" href="#conclusion" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h1&gt;
&lt;p&gt;Optimizers are awesome, and we hope these two posts have demystified what they
are and how they are implemented in industrial systems. Like many modern query
engine designs, the common techniques are well known, though require substantial
effort to get right.  DataFusion’s industrial strength optimizers can and do
serve many real world systems well and we expect that number to grow over time.&lt;/p&gt;
&lt;p&gt;We also think DataFusion provides interesting opportunities for optimizer
research. As we discussed, there are still unsolved problems such as optimal
join ordering. Experiments in papers often use academic systems or modify
optimizers in tightly integrated open source systems (for example, the recent
&lt;a href="https://www.vldb.org/pvldb/vol17/p1350-justen.pdf"&gt;POLARs paper&lt;/a&gt; uses DuckDB). However, using a tightly integrated system
constrains the research to the set of heuristics and structure provided by that
system. Hopefully DataFusion’s documentation, &lt;a href="https://dl.acm.org/doi/10.1145/3626246.3653368"&gt;newly citeable SIGMOD paper&lt;/a&gt;, and
modular design will encourage more broadly applicable research in this area.&lt;/p&gt;
&lt;p&gt;And finally, as always, if you are interested in working on query engines and
learning more about how they are designed and implemented, please &lt;a href="https://datafusion.apache.org/contributor-guide/communication.html"&gt;join our
community&lt;/a&gt;. We welcome first time contributors as well as long time participants
to the fun of building a database together.&lt;/p&gt;
&lt;h2 id="notes"&gt;Notes&lt;a class="headerlink" href="#notes" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;&lt;a id="footnote7"&gt;&lt;/a&gt;&lt;sup&gt;[7]&lt;/sup&gt; See &lt;a href="https://btw-2015.informatik.uni-hamburg.de/res/proceedings/Hauptband/Wiss/Neumann-Unnesting_Arbitrary_Querie.pdf"&gt;Unnesting Arbitrary Queries&lt;/a&gt; from Neumann and Kemper for a more academic treatment.&lt;/p&gt;
&lt;p&gt;&lt;a id="footnote8"&gt;&lt;/a&gt;&lt;sup&gt;[8]&lt;/sup&gt; One of my favorite terms I learned from Andy Pavlo’s CMU online lectures&lt;/p&gt;</content><category term="blog"/></entry><entry><title>Apache DataFusion Comet 0.8.0 Release</title><link href="https://datafusion.apache.org/blog/2025/05/06/datafusion-comet-0.8.0" rel="alternate"/><published>2025-05-06T00:00:00+00:00</published><updated>2025-05-06T00:00:00+00:00</updated><author><name>pmc</name></author><id>tag:datafusion.apache.org,2025-05-06:/blog/2025/05/06/datafusion-comet-0.8.0</id><summary type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;p&gt;The Apache DataFusion PMC is pleased to announce version 0.8.0 of the &lt;a href="https://datafusion.apache.org/comet/"&gt;Comet&lt;/a&gt; subproject.&lt;/p&gt;
&lt;p&gt;Comet is an accelerator for Apache Spark that translates Spark physical plans to DataFusion physical plans for
improved performance and efficiency without requiring any code changes.&lt;/p&gt;
&lt;p&gt;This release covers approximately six weeks of development …&lt;/p&gt;</summary><content type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;p&gt;The Apache DataFusion PMC is pleased to announce version 0.8.0 of the &lt;a href="https://datafusion.apache.org/comet/"&gt;Comet&lt;/a&gt; subproject.&lt;/p&gt;
&lt;p&gt;Comet is an accelerator for Apache Spark that translates Spark physical plans to DataFusion physical plans for
improved performance and efficiency without requiring any code changes.&lt;/p&gt;
&lt;p&gt;This release covers approximately six weeks of development work and is the result of merging 81 PRs from 11
contributors. See the &lt;a href="https://github.com/apache/datafusion-comet/blob/main/dev/changelog/0.8.0.md"&gt;change log&lt;/a&gt; for more information.&lt;/p&gt;
&lt;h2 id="release-highlights"&gt;Release Highlights&lt;a class="headerlink" href="#release-highlights" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;h3 id="performance-stability"&gt;Performance &amp;amp; Stability&lt;a class="headerlink" href="#performance-stability" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;ul&gt;
&lt;li&gt;Up to 4x speedup in jobs using &lt;code&gt;dropDuplicates&lt;/code&gt;, thanks to optimizations in the &lt;code&gt;first_value&lt;/code&gt; and &lt;code&gt;last_value&lt;/code&gt;
  aggregate functions in DataFusion 47.0.0.&lt;/li&gt;
&lt;li&gt;Introduction of a global Tokio runtime, which resolves potential deadlocks in certain multi-task scenarios.&lt;/li&gt;
&lt;/ul&gt;
&lt;h2 id="native-shuffle-improvements"&gt;Native Shuffle Improvements&lt;a class="headerlink" href="#native-shuffle-improvements" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;Significant enhancements to the native shuffle mechanism include:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;Lower memory usage through using &lt;code&gt;interleave_record_batches&lt;/code&gt; instead of using array builders.&lt;/li&gt;
&lt;li&gt;Support for complex types in shuffle data (note: hash partition expressions still require primitive types).&lt;/li&gt;
&lt;li&gt;Reclaimable shuffle files, reducing disk pressure.&lt;/li&gt;
&lt;li&gt;Respects &lt;code&gt;spark.local.dir&lt;/code&gt; for temporary storage.&lt;/li&gt;
&lt;li&gt;Per-task shuffle metrics are now available, providing better visibility into execution behavior.&lt;/li&gt;
&lt;/ul&gt;
&lt;h2 id="experimental-support-for-datafusions-parquet-scan"&gt;Experimental Support for DataFusion’s Parquet Scan&lt;a class="headerlink" href="#experimental-support-for-datafusions-parquet-scan" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;It is now possible to configure Comet to use DataFusion’s Parquet reader instead of Comet’s current Parquet reader. This
has the advantage of supporting complex types, and also has performance optimizations that are not present in Comet's
existing reader.&lt;/p&gt;
&lt;p&gt;This release continues with the ongoing improvements and bug fixes and supports more use cases, but there are still
some known issues:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;There are schema coercion bugs for nested types containing INT96 columns, which can cause incorrect results.&lt;/li&gt;
&lt;li&gt;There are compatibility issues when reading integer values that are larger than their type annotation, such as the
  value 1024 being stored in a field annotated as int(8).&lt;/li&gt;
&lt;li&gt;A small number of Spark SQL tests remain unsupported (&lt;a href="https://github.com/apache/datafusion-comet/issues/1545"&gt;#1545&lt;/a&gt;).&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;To enable DataFusion’s Parquet reader, either set &lt;code&gt;spark.comet.scan.impl=native_datafusion&lt;/code&gt; or set the environment
variable &lt;code&gt;COMET_PARQUET_SCAN_IMPL=native_datafusion&lt;/code&gt;.&lt;/p&gt;
&lt;h2 id="updates-to-supported-spark-versions"&gt;Updates to Supported Spark Versions&lt;a class="headerlink" href="#updates-to-supported-spark-versions" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;ul&gt;
&lt;li&gt;Added support for Spark 3.5.5&lt;/li&gt;
&lt;li&gt;Dropped support for Spark 3.3.x&lt;/li&gt;
&lt;/ul&gt;
&lt;h2 id="getting-involved"&gt;Getting Involved&lt;a class="headerlink" href="#getting-involved" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;The Comet project welcomes new contributors. We use the same &lt;a href="https://datafusion.apache.org/contributor-guide/communication.html#slack-and-discord"&gt;Slack and Discord&lt;/a&gt; channels as the main DataFusion
project and have a weekly &lt;a href="https://docs.google.com/document/d/1NBpkIAuU7O9h8Br5CbFksDhX-L9TyO9wmGLPMe0Plc8/edit?usp=sharing"&gt;DataFusion video call&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;The easiest way to get involved is to test Comet with your current Spark jobs and file issues for any bugs or
performance regressions that you find. See the &lt;a href="https://datafusion.apache.org/comet/user-guide/installation.html"&gt;Getting Started&lt;/a&gt; guide for instructions on downloading and installing
Comet.&lt;/p&gt;
&lt;p&gt;There are also many &lt;a href="https://github.com/apache/datafusion-comet/contribute"&gt;good first issues&lt;/a&gt; waiting for contributions.&lt;/p&gt;</content><category term="blog"/></entry><entry><title>User defined Window Functions in DataFusion</title><link href="https://datafusion.apache.org/blog/2025/04/19/user-defined-window-functions" rel="alternate"/><published>2025-04-19T00:00:00+00:00</published><updated>2025-04-19T00:00:00+00:00</updated><author><name>Aditya Singh Rathore, Andrew Lamb</name></author><id>tag:datafusion.apache.org,2025-04-19:/blog/2025/04/19/user-defined-window-functions</id><summary type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;p&gt;Window functions are a powerful feature in SQL, allowing for complex analytical computations over a subset of data. However, efficiently implementing them, especially sliding windows, can be quite challenging. With &lt;a href="https://datafusion.apache.org/"&gt;Apache DataFusion&lt;/a&gt;'s user-defined window functions, developers can easily take advantage of all the effort put into DataFusion's implementation.&lt;/p&gt;
&lt;p&gt;In …&lt;/p&gt;</summary><content type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;p&gt;Window functions are a powerful feature in SQL, allowing for complex analytical computations over a subset of data. However, efficiently implementing them, especially sliding windows, can be quite challenging. With &lt;a href="https://datafusion.apache.org/"&gt;Apache DataFusion&lt;/a&gt;'s user-defined window functions, developers can easily take advantage of all the effort put into DataFusion's implementation.&lt;/p&gt;
&lt;p&gt;In this post, we'll explore:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;
&lt;p&gt;What window functions are and why they matter&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;
&lt;p&gt;Understanding sliding windows&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;
&lt;p&gt;The challenges of computing window aggregates efficiently&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;
&lt;p&gt;How to implement user-defined window functions in DataFusion&lt;/p&gt;
&lt;/li&gt;
&lt;/ul&gt;
&lt;h2 id="understanding-window-functions-in-sql"&gt;Understanding Window Functions in SQL&lt;a class="headerlink" href="#understanding-window-functions-in-sql" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;Imagine you're analyzing sales data and want insights without losing the finer details. This is where &lt;strong&gt;&lt;a href="https://en.wikipedia.org/wiki/Window_function_(SQL)"&gt;window functions&lt;/a&gt;&lt;/strong&gt; come into play. Unlike &lt;strong&gt;GROUP BY&lt;/strong&gt;, which condenses data, window functions let you retain each row while performing calculations over a defined &lt;strong&gt;range&lt;/strong&gt; —like having a moving lens over your dataset.&lt;/p&gt;
&lt;p&gt;Picture a business tracking daily sales. They need a running total to understand cumulative revenue trends without collapsing individual transactions. SQL makes this easy:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-sql"&gt;SELECT id, value, SUM(value) OVER (ORDER BY id) AS running_total
FROM sales;
&lt;/code&gt;&lt;/pre&gt;
&lt;pre&gt;&lt;code class="language-text"&gt;example:
+------------+--------+-------------------------------+
|   Date     | Sales  | Rows Considered               |
+------------+--------+-------------------------------+
| Jan 01     | 100    | [100]                         |
| Jan 02     | 120    | [100, 120]                    |
| Jan 03     | 130    | [100, 120, 130]               |
| Jan 04     | 150    | [100, 120, 130, 150]          |
| Jan 05     | 160    | [100, 120, 130, 150, 160]     |
| Jan 06     | 180    | [100, 120, 130, 150, 160, 180]|
| Jan 07     | 170    | [100, ..., 170] (7 days)      |
| Jan 08     | 175    | [120, ..., 175]               |
+------------+--------+-------------------------------+
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;&lt;strong&gt;Figure 1&lt;/strong&gt;: A row-by-row representation of how a 7-day moving average includes the previous 6 days and the current one.&lt;/p&gt;
&lt;p&gt;This helps in analytical queries where we need cumulative sums, moving averages, or ranking without losing individual records.&lt;/p&gt;
&lt;h2 id="user-defined-window-functions"&gt;User Defined Window Functions&lt;a class="headerlink" href="#user-defined-window-functions" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;DataFusion's &lt;a href="https://datafusion.apache.org/user-guide/sql/window_functions.html"&gt;Built-in window functions&lt;/a&gt; such as &lt;code&gt;first_value&lt;/code&gt;, &lt;code&gt;rank&lt;/code&gt; and &lt;code&gt;row_number&lt;/code&gt; serve many common use cases, but sometimes custom logic is needed—for example:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;
&lt;p&gt;Calculating moving averages with complex conditions (e.g. exponential averages, integrals, etc)&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;
&lt;p&gt;Implementing a custom ranking strategy&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;
&lt;p&gt;Tracking non-standard cumulative logic&lt;/p&gt;
&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;Thus, &lt;strong&gt;User-Defined Window Functions (UDWFs)&lt;/strong&gt; allow developers to define their own behavior while allowing DataFusion to handle the calculations of the  windows and grouping specified in the &lt;code&gt;OVER&lt;/code&gt; clause&lt;/p&gt;
&lt;p&gt;Writing a user defined window function is slightly more complex than an aggregate function due
to the variety of ways that window functions are called. I recommend reviewing the
&lt;a href="https://datafusion.apache.org/library-user-guide/adding-udfs.html#registering-a-window-udf"&gt;online documentation&lt;/a&gt;
for a description of which functions need to be implemented. &lt;/p&gt;
&lt;h2 id="understanding-sliding-window"&gt;Understanding Sliding Window&lt;a class="headerlink" href="#understanding-sliding-window" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;Sliding windows define a &lt;strong&gt;moving range&lt;/strong&gt; of data over which aggregations are computed. Unlike simple cumulative functions, these windows are dynamically updated as new data arrives.&lt;/p&gt;
&lt;p&gt;For instance, if we want a 7-day moving average of sales:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-sql"&gt;SELECT date, sales, 
       AVG(sales) OVER (ORDER BY date ROWS BETWEEN 6 PRECEDING AND CURRENT ROW) AS moving_avg
FROM sales;
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;Here, each row’s result is computed based on the last 7 days, making it computationally intensive as data grows.&lt;/p&gt;
&lt;h2 id="why-computing-sliding-windows-is-hard"&gt;Why Computing Sliding Windows Is Hard&lt;a class="headerlink" href="#why-computing-sliding-windows-is-hard" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;Imagine you’re at a café, and the barista is preparing coffee orders. If they made each cup from scratch without using pre-prepared ingredients, the process would be painfully slow. This is exactly the problem with naïve sliding window computations.&lt;/p&gt;
&lt;p&gt;Computing sliding windows efficiently is tricky because:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;
&lt;p&gt;&lt;strong&gt;High Computation Costs:&lt;/strong&gt; Just like making coffee from scratch for each customer, recalculating aggregates for every row is expensive.&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;
&lt;p&gt;&lt;strong&gt;Data Shuffling:&lt;/strong&gt; In large distributed systems, data must often be shuffled between nodes, causing delays—like passing orders between multiple baristas who don’t communicate efficiently.&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;
&lt;p&gt;&lt;strong&gt;State Management:&lt;/strong&gt; Keeping track of past computations is like remembering previous orders without writing them down—error-prone and inefficient.&lt;/p&gt;
&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;Many traditional query engines struggle to optimize these computations effectively, leading to sluggish performance.&lt;/p&gt;
&lt;h2 id="how-datafusion-evaluates-window-functions-quickly"&gt;How DataFusion Evaluates Window Functions Quickly&lt;a class="headerlink" href="#how-datafusion-evaluates-window-functions-quickly" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;In the world of big data, every millisecond counts. Imagine you’re analyzing stock market data, tracking sensor readings from millions of IoT devices, or crunching through massive customer logs—speed matters. This is where &lt;a href="https://datafusion.apache.org/"&gt;DataFusion&lt;/a&gt; shines, making window function computations blazing fast. Let’s break down how it achieves this remarkable performance.&lt;/p&gt;
&lt;p&gt;DataFusion implements the battle tested sort-based approach described in &lt;a href="https://www.vldb.org/pvldb/vol8/p1058-leis.pdf"&gt;this
paper&lt;/a&gt; which is also used in systems such as Postgresql and Vertica. The input
is first sorted by both the &lt;code&gt;PARTITION BY&lt;/code&gt; and &lt;code&gt;ORDER BY&lt;/code&gt; expressions and
then the &lt;a href="https://github.com/apache/datafusion/blob/7ff6c7e68540c69b399a171654d00577e6f886bf/datafusion/physical-plan/src/windows/window_agg_exec.rs"&gt;WindowAggExec&lt;/a&gt; operator efficiently determines the partition boundaries and
creates appropriate &lt;a href="https://docs.rs/datafusion/latest/datafusion/logical_expr/trait.PartitionEvaluator.html#background"&gt;PartitionEvaluator&lt;/a&gt; instances. &lt;/p&gt;
&lt;p&gt;The sort-based approach is well understood, scales to large data sets, and
leverages DataFusion's highly optimized sort implementation. DataFusion minimizes
resorting by leveraging the sort order tracking and optimizations described in
the &lt;a href="https://datafusion.apache.org/blog/2025/03/11/ordering-analysis/"&gt;Using Ordering for Better Plans blog&lt;/a&gt;. &lt;/p&gt;
&lt;p&gt;For example, given the query such as the following to compute the starting,
ending and average price for each stock:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-sql"&gt;SELECT 
  FIRST_VALUE(price) OVER (PARTITION BY date_bin('1 month', time) ORDER BY time DESC) AS start_price, 
  FIRST_VALUE(price) OVER (PARTITION BY date_bin('1 month', time) ORDER BY time DESC) AS end_price,
  AVG(price)         OVER (PARTITION BY date_bin('1 month', time))                    AS avg_price
FROM quotes;
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;If the input data is not sorted, DataFusion will first sort the data by the
&lt;code&gt;date_bin&lt;/code&gt; and &lt;code&gt;time&lt;/code&gt; and then &lt;a href="https://github.com/apache/datafusion/blob/7ff6c7e68540c69b399a171654d00577e6f886bf/datafusion/physical-plan/src/windows/window_agg_exec.rs"&gt;WindowAggExec&lt;/a&gt; computes the partition boundaries
and invokes the appropriate &lt;a href="https://docs.rs/datafusion/latest/datafusion/logical_expr/trait.PartitionEvaluator.html#background"&gt;PartitionEvaluator&lt;/a&gt; API methods depending on the window
definition in the &lt;code&gt;OVER&lt;/code&gt; clause and the declared capabilities of the function.&lt;/p&gt;
&lt;p&gt;For example, evaluating &lt;code&gt;window_func(val) OVER (PARTITION BY col)&lt;/code&gt;
on the following data:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-text"&gt;col | val
--- + ----
 A  | 10
 A  | 10
 C  | 20
 D  | 30
 D  | 30
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;Will instantiate three &lt;a href="https://docs.rs/datafusion/latest/datafusion/logical_expr/trait.PartitionEvaluator.html#background"&gt;PartitionEvaluator&lt;/a&gt;s, one each for the
partitions defined by &lt;code&gt;col=A&lt;/code&gt;, &lt;code&gt;col=B&lt;/code&gt;, and &lt;code&gt;col=C&lt;/code&gt;.&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-text"&gt;col | val
--- + ----
 A  | 10     &amp;lt;--- partition 1
 A  | 10

col | val
--- + ----
 C  | 20     &amp;lt;--- partition 2

col | val
--- + ----
 D  | 30     &amp;lt;--- partition 3
 D  | 30
&lt;/code&gt;&lt;/pre&gt;
&lt;h3 id="creating-your-own-window-function"&gt;Creating your own Window Function&lt;a class="headerlink" href="#creating-your-own-window-function" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;DataFusion supports &lt;a href="https://datafusion.apache.org/library-user-guide/adding-udfs.html"&gt;user-defined window aggregates (UDWAs)&lt;/a&gt;, meaning you can bring your own window function logic using the exact same APIs and performance as the built in functions.&lt;/p&gt;
&lt;p&gt;For example, we will declare a user defined window function that computes a moving average.&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-rust"&gt;use datafusion::arrow::{array::{ArrayRef, Float64Array, AsArray}, datatypes::Float64Type};
use datafusion::logical_expr::{PartitionEvaluator};
use datafusion::common::ScalarValue;
use datafusion::error::Result;
/// This implements the lowest level evaluation for a window function
///
/// It handles calculating the value of the window function for each
/// distinct values of `PARTITION BY`
#[derive(Clone, Debug)]
struct MyPartitionEvaluator {}

impl MyPartitionEvaluator {
    fn new() -&amp;gt; Self {
        Self {}
    }
}
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;Different evaluation methods are called depending on the various
settings of WindowUDF and the query. In the first example, we use the simplest and most
general, &lt;code&gt;evaluate&lt;/code&gt; function. We will see how to use &lt;code&gt;PartitionEvaluator&lt;/code&gt; for the other more
advanced uses later in the article.&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-rust"&gt;impl PartitionEvaluator for MyPartitionEvaluator {
    /// Tell DataFusion the window function varies based on the value
    /// of the window frame.
    fn uses_window_frame(&amp;amp;self) -&amp;gt; bool {
        true
    }

    /// This function is called once per input row.
    ///
    /// `range`specifies which indexes of `values` should be
    /// considered for the calculation.
    ///
    /// Note this is the SLOWEST, but simplest, way to evaluate a
    /// window function. It is much faster to implement
    /// evaluate_all or evaluate_all_with_rank, if possible
    fn evaluate(
        &amp;amp;mut self,
        values: &amp;amp;[ArrayRef],
        range: &amp;amp;std::ops::Range&amp;lt;usize&amp;gt;,
    ) -&amp;gt; Result&amp;lt;ScalarValue&amp;gt; {
        // Again, the input argument is an array of floating
        // point numbers to calculate a moving average
        let arr: &amp;amp;Float64Array = values[0].as_ref().as_primitive::&amp;lt;Float64Type&amp;gt;();

        let range_len = range.end - range.start;

        // our smoothing function will average all the values in the
        let output = if range_len &amp;gt; 0 {
            let sum: f64 = arr.values().iter().skip(range.start).take(range_len).sum();
            Some(sum / range_len as f64)
        } else {
            None
        };

        Ok(ScalarValue::Float64(output))
    }
}

/// Create a `PartitionEvaluator` to evaluate this function on a new
/// partition.
fn make_partition_evaluator() -&amp;gt; Result&amp;lt;Box&amp;lt;dyn PartitionEvaluator&amp;gt;&amp;gt; {
    Ok(Box::new(MyPartitionEvaluator::new()))
}
&lt;/code&gt;&lt;/pre&gt;
&lt;h3 id="registering-a-window-udf"&gt;Registering a Window UDF&lt;a class="headerlink" href="#registering-a-window-udf" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;To register a Window UDF, you need to wrap the function implementation in a &lt;a href="https://docs.rs/datafusion/latest/datafusion/logical_expr/struct.WindowUDF.html"&gt;WindowUDF&lt;/a&gt; struct and then register it with the &lt;code&gt;SessionContext&lt;/code&gt;. DataFusion provides the &lt;a href="https://docs.rs/datafusion/latest/datafusion/logical_expr/fn.create_udwf.html"&gt;create_udwf&lt;/a&gt; helper functions to make this easier. There is a lower level API with more functionality but is more complex, that is documented in &lt;a href="https://github.com/apache/datafusion/blob/main/datafusion-examples/examples/advanced_udwf.rs"&gt;advanced_udwf.rs&lt;/a&gt;.&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-rust"&gt;use datafusion::logical_expr::{Volatility, create_udwf};
use datafusion::arrow::datatypes::DataType;
use std::sync::Arc;

// here is where we define the UDWF. We also declare its signature:
let smooth_it = create_udwf(
    "smooth_it",
    DataType::Float64,
    Arc::new(DataType::Float64),
    Volatility::Immutable,
    Arc::new(make_partition_evaluator),
);
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;The &lt;a href="https://docs.rs/datafusion/latest/datafusion/logical_expr/fn.create_udwf.html"&gt;create_udwf&lt;/a&gt; functions take  five arguments:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;
&lt;p&gt;The &lt;strong&gt;first argument&lt;/strong&gt; is the name of the function. This is the name that will be used in SQL queries.&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;
&lt;p&gt;The &lt;strong&gt;second argument&lt;/strong&gt; is the &lt;code&gt;DataType of&lt;/code&gt; input array (attention: this is not a list of arrays). I.e. in this case, the function accepts &lt;code&gt;Float64&lt;/code&gt; as argument.&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;
&lt;p&gt;The &lt;strong&gt;third argument&lt;/strong&gt; is the return type of the function. I.e. in this case, the function returns an &lt;code&gt;Float64&lt;/code&gt;.&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;
&lt;p&gt;The &lt;strong&gt;fourth argument&lt;/strong&gt; is the volatility of the function. In short, this is used to determine if the function’s performance can be optimized in some situations. In this case, the function is &lt;code&gt;Immutable&lt;/code&gt; because it always returns the same value for the same input. A random number generator would be &lt;code&gt;Volatile&lt;/code&gt; because it returns a different value for the same input.&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;
&lt;p&gt;The &lt;strong&gt;fifth argument&lt;/strong&gt; is the function implementation. This is the function that we defined above.&lt;/p&gt;
&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;That gives us a &lt;strong&gt;WindowUDF&lt;/strong&gt; that we can register with the &lt;code&gt;SessionContext&lt;/code&gt;:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-rust"&gt;use datafusion::execution::context::SessionContext;

let ctx = SessionContext::new();

ctx.register_udwf(smooth_it);
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;For example, if we have a &lt;a href="https://github.com/apache/datafusion/blob/main/datafusion/core/tests/data/cars.csv"&gt;cars.csv&lt;/a&gt; whose contents like&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-text"&gt;car,speed,time
red,20.0,1996-04-12T12:05:03.000000000
red,20.3,1996-04-12T12:05:04.000000000
green,10.0,1996-04-12T12:05:03.000000000
green,10.3,1996-04-12T12:05:04.000000000
...
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;Then, we can query like below:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-rust"&gt;use datafusion::datasource::file_format::options::CsvReadOptions;

#[tokio::main]
async fn main() -&amp;gt; Result&amp;lt;()&amp;gt; {

    let ctx = SessionContext::new();

    let smooth_it = create_udwf(
        "smooth_it",
        DataType::Float64,
        Arc::new(DataType::Float64),
        Volatility::Immutable,
        Arc::new(make_partition_evaluator),
    );
    ctx.register_udwf(smooth_it);

    // register csv table first
    let csv_path = "../../datafusion/core/tests/data/cars.csv".to_string();
    ctx.register_csv("cars", &amp;amp;csv_path, CsvReadOptions::default().has_header(true)).await?;

    // do query with smooth_it
    let df = ctx
        .sql(r#"
            SELECT
                car,
                speed,
                smooth_it(speed) OVER (PARTITION BY car ORDER BY time) as smooth_speed,
                time
            FROM cars
            ORDER BY car
        "#)
        .await?;

    // print the results
    df.show().await?;
    Ok(())
}
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;The output will be like:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-sql"&gt;+-------+-------+--------------------+---------------------+
| car   | speed | smooth_speed       | time                |
+-------+-------+--------------------+---------------------+
| green | 10.0  | 10.0               | 1996-04-12T12:05:03 |
| green | 10.3  | 10.15              | 1996-04-12T12:05:04 |
| green | 10.4  | 10.233333333333334 | 1996-04-12T12:05:05 |
| green | 10.5  | 10.3               | 1996-04-12T12:05:06 |
| green | 11.0  | 10.440000000000001 | 1996-04-12T12:05:07 |
| green | 12.0  | 10.700000000000001 | 1996-04-12T12:05:08 |
| green | 14.0  | 11.171428571428573 | 1996-04-12T12:05:09 |
| green | 15.0  | 11.65              | 1996-04-12T12:05:10 |
| green | 15.1  | 12.033333333333333 | 1996-04-12T12:05:11 |
| green | 15.2  | 12.35              | 1996-04-12T12:05:12 |
| green | 8.0   | 11.954545454545455 | 1996-04-12T12:05:13 |
| green | 2.0   | 11.125             | 1996-04-12T12:05:14 |
| red   | 20.0  | 20.0               | 1996-04-12T12:05:03 |
| red   | 20.3  | 20.15              | 1996-04-12T12:05:04 |
...
...
+-------+-------+--------------------+---------------------+
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;This gives you full flexibility to build &lt;strong&gt;domain-specific logic&lt;/strong&gt; that plugs seamlessly into DataFusion’s engine — all without sacrificing performance.&lt;/p&gt;
&lt;h2 id="final-thoughts-and-recommendations"&gt;Final Thoughts and Recommendations&lt;a class="headerlink" href="#final-thoughts-and-recommendations" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;Window functions may be common in SQL, but &lt;em&gt;efficient and extensible&lt;/em&gt; window functions in engines are rare. 
While many databases support user defined scalar and user defined aggregate functions, user defined window functions are not as common and Datafusion making it easier for all .&lt;/p&gt;
&lt;p&gt;For anyone who is curious about &lt;a href="https://datafusion.apache.org/"&gt;DataFusion&lt;/a&gt; I highly recommend
giving it a try. This post was designed to make it easier for new users to work with User Defined Window Functions by giving a few examples of how one might implement these.&lt;/p&gt;
&lt;p&gt;When it comes to designing UDFs, I strongly recommend reviewing the 
&lt;a href="https://datafusion.apache.org/library-user-guide/adding-udfs.html"&gt;Window functions&lt;/a&gt; documentation.&lt;/p&gt;
&lt;p&gt;A heartfelt thank you to &lt;a href="https://github.com/alamb"&gt;@alamb&lt;/a&gt; and &lt;a href="https://github.com/andygrove"&gt;@andygrove&lt;/a&gt; for their invaluable reviews and thoughtful feedback—they’ve been instrumental in shaping this post.&lt;/p&gt;
&lt;p&gt;The Apache Arrow and Apache DataFusion communities are vibrant, welcoming, and full of passionate developers building something truly powerful. If you’re excited about high-performance analytics and want to be part of an open-source journey, I highly encourage you to explore the &lt;a href="(https://datafusion.apache.org/)"&gt;official documentation&lt;/a&gt; and dive into one of the many &lt;a href="https://github.com/apache/datafusion/issues"&gt;open issues&lt;/a&gt;. There’s never been a better time to get involved!&lt;/p&gt;</content><category term="blog"/></entry><entry><title>tpchgen-rs World’s fastest open source TPC-H data generator, written in Rust</title><link href="https://datafusion.apache.org/blog/2025/04/10/fastest-tpch-generator" rel="alternate"/><published>2025-04-10T00:00:00+00:00</published><updated>2025-04-10T00:00:00+00:00</updated><author><name>Andrew Lamb, Achraf B, and Sean Smith</name></author><id>tag:datafusion.apache.org,2025-04-10:/blog/2025/04/10/fastest-tpch-generator</id><summary type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}x
--&gt;

&lt;p&gt;&lt;strong&gt;TLDR: TPC-H SF=100 in 1min using tpchgen-rs vs 30min+ with dbgen&lt;/strong&gt;.&lt;/p&gt;
&lt;p&gt;3 members of the &lt;a href="https://datafusion.apache.org/"&gt;Apache DataFusion&lt;/a&gt; community used Rust and open source
development to build &lt;a href="https://github.com/clflushopt/tpchgen-rs"&gt;tpchgen-rs&lt;/a&gt;, a fully open TPC-H data generator over 20x
faster than any other implementation we know of.&lt;/p&gt;
&lt;p&gt;It is now possible to create …&lt;/p&gt;</summary><content type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}x
--&gt;

&lt;p&gt;&lt;strong&gt;TLDR: TPC-H SF=100 in 1min using tpchgen-rs vs 30min+ with dbgen&lt;/strong&gt;.&lt;/p&gt;
&lt;p&gt;3 members of the &lt;a href="https://datafusion.apache.org/"&gt;Apache DataFusion&lt;/a&gt; community used Rust and open source
development to build &lt;a href="https://github.com/clflushopt/tpchgen-rs"&gt;tpchgen-rs&lt;/a&gt;, a fully open TPC-H data generator over 20x
faster than any other implementation we know of.&lt;/p&gt;
&lt;p&gt;It is now possible to create the TPC-H SF=100 dataset in 72.23 seconds (1.4 GB/s
😎) on a Macbook Air M3 with 16GB of memory, compared to the classic &lt;code&gt;dbgen&lt;/code&gt;
which takes 30 minutes&lt;sup&gt;1&lt;/sup&gt; (0.05GB/sec). On the same machine, it takes less than
2 minutes to create all 36 GB of SF=100 in &lt;a href="https://parquet.apache.org/"&gt;Apache Parquet&lt;/a&gt; format, which takes 44 minutes using &lt;a href="https://duckdb.org"&gt;DuckDB&lt;/a&gt;.
It is finally convenient and efficient to run TPC-H queries locally when testing
analytical engines such as DataFusion.&lt;/p&gt;
&lt;p&gt;&lt;img alt="Time to create TPC-H parquet dataset for Scale Factor  1, 10, 100 and 1000" class="img-fluid" src="/blog/images/fastest-tpch-generator/parquet-performance.png" width="80%"/&gt;&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Figure 1&lt;/strong&gt;: Time to create TPC-H dataset for Scale Factor (see below) 1, 10,
100 and 1000 as 8 individual SNAPPY compressed parquet files using a 22 core GCP
VM with 88GB of memory. For Scale Factor(SF) 100 &lt;code&gt;tpchgen&lt;/code&gt; takes 1 minute and 14 seconds and
&lt;a href="https://duckdb.org"&gt;DuckDB&lt;/a&gt; takes 17 minutes and 48 seconds. For SF=1000, &lt;code&gt;tpchgen&lt;/code&gt; takes 10
minutes and 26 and uses about 5 GB of RAM at peak, and we could not measure
DuckDB’s time as it &lt;a href="https://duckdb.org/docs/stable/extensions/tpch.html#resource-usage-of-the-data-generator"&gt;requires 647 GB of RAM&lt;/a&gt;, more than the 88 GB that was
available on our test machine. The testing methodology is in the
&lt;a href="https://github.com/clflushopt/tpchgen-rs/blob/main/benchmarks/BENCHMARKS.md"&gt;documentation&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;This blog explains what TPC-H is, how we ported the vintage C data generator to
Rust (yes, &lt;a href="https://www.reddit.com/r/rust/comments/4ri2gn/riir_rewrite_it_in_rust/"&gt;RWIR&lt;/a&gt;) and optimized its performance over the course of a few weeks
of part-time work. We began this project so we can easily generate TPC-H data in
&lt;a href="https://datafusion.apache.org/"&gt;Apache DataFusion&lt;/a&gt; and &lt;a href="https://glaredb.com/"&gt;GlareDB&lt;/a&gt;.&lt;/p&gt;
&lt;h1 id="try-it-for-yourself"&gt;Try it for yourself&lt;a class="headerlink" href="#try-it-for-yourself" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h1&gt;
&lt;p&gt;The tool is entirely open source under the &lt;a href="https://www.apache.org/licenses/LICENSE-2.0"&gt;Apache 2.0 license&lt;/a&gt;. Visit the &lt;a href="https://github.com/clflushopt/tpchgen-rs"&gt;tpchgen-rs repository&lt;/a&gt; or try it for yourself by run the following commands after &lt;a href="https://www.rust-lang.org/tools/install"&gt;installing Rust&lt;/a&gt;:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-shell"&gt;$ cargo install tpchgen-cli

# create SF=1 in classic TBL format
$ tpchgen-cli -s 1 

# create SF=10 in Parquet
$ tpchgen-cli -s 10 --format=parquet
&lt;/code&gt;&lt;/pre&gt;
&lt;h1 id="what-is-tpc-h-dbgen"&gt;What is TPC-H / dbgen?&lt;a class="headerlink" href="#what-is-tpc-h-dbgen" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h1&gt;
&lt;p&gt;The popular &lt;a href="https://www.tpc.org/tpch/"&gt;TPC-H&lt;/a&gt; benchmark (often referred to as TPCH) helps evaluate the
performance of database systems on &lt;a href="https://en.wikipedia.org/wiki/Online_analytical_processing"&gt;OLAP&lt;/a&gt; queries, the kind used to build BI
dashboards.&lt;/p&gt;
&lt;p&gt;TPC-H has become a de facto standard for analytic systems. While there are &lt;a href="https://www.vldb.org/pvldb/vol9/p204-leis.pdf"&gt;well
known&lt;/a&gt; limitations as the data and queries do not well represent many real world
use cases, the majority of analytic database papers and industrial systems still
use TPC-H query performance benchmarks as a baseline. You will inevitably find
multiple results for  “&lt;code&gt;TPCH Performance &amp;lt;your favorite database&amp;gt;&lt;/code&gt;” in any
search engine.&lt;/p&gt;
&lt;p&gt;The benchmark was created at a time when access to high performance analytical
systems was not widespread, so the &lt;a href="https://www.tpc.org/"&gt;Transaction Processing Performance Council&lt;/a&gt;
defined a process of formal result verification. More recently, given the broad
availability of free and open source database systems, it is common for users to
run and verify TPC-H performance themselves.&lt;/p&gt;
&lt;p&gt;TPC-H simulates a business environment with eight tables: &lt;code&gt;REGION&lt;/code&gt;, &lt;code&gt;NATION&lt;/code&gt;,
&lt;code&gt;SUPPLIER&lt;/code&gt;, &lt;code&gt;CUSTOMER&lt;/code&gt;, &lt;code&gt;PART&lt;/code&gt;, &lt;code&gt;PARTSUPP&lt;/code&gt;, &lt;code&gt;ORDERS&lt;/code&gt;, and &lt;code&gt;LINEITEM&lt;/code&gt;. These
tables are linked by foreign keys in a normalized schema representing a supply
chain with parts, suppliers, customers and orders. The benchmark itself is 22
SQL queries containing joins, aggregations, and sorting operations.&lt;/p&gt;
&lt;p&gt;The queries run against data created with &lt;code&gt;&lt;a href="https://github.com/electrum/tpch-dbgen"&gt;dbgen&lt;/a&gt;&lt;/code&gt;, a program
written in a pre &lt;a href="https://en.wikipedia.org/wiki/C99"&gt;C-99&lt;/a&gt; dialect, which generates data in a format called &lt;em&gt;TBL&lt;/em&gt;
(example in Figure 2). &lt;code&gt;dbgen&lt;/code&gt; creates data for each of the 8 tables for a
certain &lt;em&gt;Scale Factor&lt;/em&gt;, commonly abbreviated as SF. Example Scale Factors and
corresponding dataset sizes are shown in Table 1. There is no theoretical upper
bound on the Scale Factor.&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-text"&gt;103|2844|845|3|23|40177.32|0.01|0.04|N|O|1996-09-11|1996-09-18|1996-09-26|NONE|FOB|ironic accou|
229|10540|801|6|29|42065.66|0.04|0.00|R|F|1994-01-14|1994-02-16|1994-01-22|NONE|FOB|uriously pending |
263|2396|649|1|22|28564.58|0.06|0.08|R|F|1994-08-24|1994-06-20|1994-09-09|NONE|FOB|efully express fo|
327|4172|427|2|9|9685.53|0.09|0.05|A|F|1995-05-24|1995-07-11|1995-06-05|NONE|AIR| asymptotes are fu|
450|5627|393|4|40|61304.80|0.05|0.03|R|F|1995-03-20|1995-05-25|1995-04-14|NONE|RAIL|ve. asymptote|
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;&lt;strong&gt;Figure 2&lt;/strong&gt;: Example TBL formatted output of &lt;code&gt;dbgen&lt;/code&gt; for the &lt;code&gt;LINEITEM&lt;/code&gt; table&lt;/p&gt;
&lt;table class="table"&gt;
&lt;tr&gt;
&lt;td&gt;&lt;strong&gt;Scale Factor&lt;/strong&gt;
&lt;/td&gt;
&lt;td&gt;&lt;strong&gt;Data Size (TBL)&lt;/strong&gt;
&lt;/td&gt;
&lt;td&gt;&lt;strong&gt;Data Size (Parquet)&lt;/strong&gt;
&lt;/td&gt;
&lt;/tr&gt;
&lt;tr&gt;
&lt;td&gt;0.1
   &lt;/td&gt;
&lt;td&gt;103 Mb
   &lt;/td&gt;
&lt;td&gt;31 Mb
   &lt;/td&gt;
&lt;/tr&gt;
&lt;tr&gt;
&lt;td&gt;1
   &lt;/td&gt;
&lt;td&gt;1 Gb
   &lt;/td&gt;
&lt;td&gt;340 Mb
   &lt;/td&gt;
&lt;/tr&gt;
&lt;tr&gt;
&lt;td&gt;10
   &lt;/td&gt;
&lt;td&gt;10 Gb
   &lt;/td&gt;
&lt;td&gt;3.6 Gb
   &lt;/td&gt;
&lt;/tr&gt;
&lt;tr&gt;
&lt;td&gt;100
   &lt;/td&gt;
&lt;td&gt;107 Gb
   &lt;/td&gt;
&lt;td&gt;38 Gb
   &lt;/td&gt;
&lt;/tr&gt;
&lt;tr&gt;
&lt;td&gt;1000
   &lt;/td&gt;
&lt;td&gt;1089 Gb
   &lt;/td&gt;
&lt;td&gt;379 Gb
   &lt;/td&gt;
&lt;/tr&gt;
&lt;/table&gt;
&lt;p&gt;&lt;strong&gt;Table 1&lt;/strong&gt;: TPC-H data set sizes at different scale factors for both TBL and &lt;a href="https://parquet.apache.org/"&gt;Apache Parquet&lt;/a&gt;.&lt;/p&gt;
&lt;h1 id="why-do-we-need-a-new-tpc-h-data-generator"&gt;Why do we need a new TPC-H Data generator?&lt;a class="headerlink" href="#why-do-we-need-a-new-tpc-h-data-generator" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h1&gt;
&lt;p&gt;Despite the known limitations of the TPC-H benchmark, it is so well known that it
is used frequently in database performance analysis. To run TPC-H, you must first
load the data, using &lt;code&gt;dbgen&lt;/code&gt;, which is not ideal for several reasons:&lt;/p&gt;
&lt;ol&gt;
&lt;li&gt;You must find and compile a copy of the 15+ year old C program (for example &lt;a href="https://github.com/electrum/tpch-dbgen"&gt;electrum/tpch-dbgen&lt;/a&gt;)&lt;/li&gt;
&lt;li&gt;&lt;code&gt;dbgen&lt;/code&gt; requires substantial time (Figure 3) and is not able to use more than one core.&lt;/li&gt;
&lt;li&gt;It outputs TBL format, which typically requires loading into your database (for example, &lt;a href="https://github.com/apache/datafusion/blob/507f6b6773deac69dd9d90dbe60831f5ea5abed1/datafusion/sqllogictest/test_files/tpch/create_tables.slt.part#L24-L124"&gt;here is how to do so&lt;/a&gt; in Apache DataFusion) prior to query.&lt;/li&gt;
&lt;li&gt;The implementation makes substantial assumptions about the operating environment, making it difficult to extend or embed into other systems.&lt;sup&gt;2&lt;/sup&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;&lt;img alt="Time to generate TPC-H data in TBL format" class="img-fluid" src="/blog/images/fastest-tpch-generator/tbl-performance.png" width="80%"/&gt;&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Figure 3&lt;/strong&gt;: Time to generate TPC-H data in TBL format. &lt;code&gt;tpchgen&lt;/code&gt; is
shown in blue. &lt;code&gt;tpchgen&lt;/code&gt; restricted to a single core is shown in red. Unmodified
&lt;code&gt;dbgen&lt;/code&gt; is shown in green and &lt;code&gt;dbgen&lt;/code&gt; modified to use &lt;code&gt;-O3&lt;/code&gt; optimization level
is shown in yellow.&lt;/p&gt;
&lt;p&gt;&lt;code&gt;dbgen&lt;/code&gt; is so inconvenient and takes so long that vendors often provide
preloaded TPC-H data, for example &lt;a href="https://docs.snowflake.com/en/user-guide/sample-data-tpch"&gt;Snowflake Sample Data&lt;/a&gt;, &lt;a href="https://docs.databricks.com/aws/en/discover/databricks-datasets"&gt;Databricks Sample
datasets&lt;/a&gt; and &lt;a href="https://duckdb.org/docs/stable/extensions/tpch.html#pre-generated-data-sets"&gt;DuckDB Pre-Generated Data Sets&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;In addition to pre-generated datasets, DuckDB also provides a &lt;a href="https://duckdb.org/docs/stable/extensions/tpch.html"&gt;TPC-H extension&lt;/a&gt; 
for generating TPC-H datasets within DuckDB. This is so much easier to use than
the current alternatives that it leads many researchers and other thought
leaders to use DuckDB to evaluate new ideas. For example, &lt;a href="https://github.com/lmwnshn"&gt;Wan Shen
Lim&lt;/a&gt; explicitly &lt;a href="https://github.com/apache/datafusion/issues/14373"&gt;mentioned the ease of creating the TPC-H dataset&lt;/a&gt; as one reason
the first student project of &lt;a href="https://15799.courses.cs.cmu.edu/spring2025/"&gt;CMU-799 Spring 2025&lt;/a&gt; used DuckDB.&lt;/p&gt;
&lt;p&gt;As beneficial as the DuckDB TPC-H extension is, it is non-ideal for several reasons:&lt;/p&gt;
&lt;ol&gt;
&lt;li&gt;Creates data in a proprietary format, which requires export to use in other systems.&lt;/li&gt;
&lt;li&gt;Requires significant time (e.g. 17 minutes for Scale Factor 10).&lt;/li&gt;
&lt;li&gt;Requires unnecessarily large amounts of memory (e.g. 71 GB for Scale Factor 10)&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;The above limitations makes it impractical to generate Scale Factor 100 and
above on laptops or standard workstations, though DuckDB offers &lt;a href="https://duckdb.org/docs/stable/extensions/tpch.html#pre-generated-data-sets"&gt;pre-computed
files&lt;/a&gt; for larger factors&lt;sup&gt;3&lt;/sup&gt;.&lt;/p&gt;
&lt;h1 id="why-rust"&gt;Why Rust?&lt;a class="headerlink" href="#why-rust" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h1&gt;
&lt;p&gt;Realistically we used Rust because we wanted to integrate the data generator
into &lt;a href="https://datafusion.apache.org/"&gt;Apache DataFusion&lt;/a&gt; and &lt;a href="https://glaredb.com/"&gt;GlareDB&lt;/a&gt;. However, we also believe Rust is
superior to C/C++ due to its comparable performance, but much higher programmer
productivity (Figure 4). Productivity in this case refers to the ease of
optimizing and adding multithreading without introducing hard to debug memory
safety or concurrency issues.&lt;/p&gt;
&lt;p&gt;While Rust does allow unsafe access to memory (eliding bounds checking, for
example), when required for performance, our implementation is entirely memory
safe. The only &lt;a href="https://github.com/search?q=repo%3Aclflushopt%2Ftpchgen-rs%20unsafe&amp;amp;type=code"&gt;unsafe&lt;/a&gt; code is used to &lt;a href="https://github.com/clflushopt/tpchgen-rs/blob/c651da1fc309f9cb3872cbdf71e4796904dc62c6/tpchgen/src/text.rs#L72"&gt;skip&lt;/a&gt; UTF8 validation on known ASCII
strings.&lt;/p&gt;
&lt;p&gt;&lt;img alt="Lamb Theory on Evolution of Systems Languages" class="img-fluid" src="/blog/images/fastest-tpch-generator/lamb-theory.png" width="80%"/&gt;&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Figure 4&lt;/strong&gt;: Lamb Theory of System Language Evolution from &lt;a href="https://midas.bu.edu/assets/slides/andrew_lamb_slides.pdf"&gt;Boston University
MiDAS Fall 2024 (Data Systems Seminar)&lt;/a&gt;, &lt;a href="https://www.youtube.com/watch?v=CpnxuBwHbUc"&gt;recording&lt;/a&gt;. Special
thanks to &lt;a href="https://x.com/KurtFehlhauer"&gt;@KurtFehlhauer&lt;/a&gt;&lt;/p&gt;
&lt;h1 id="how-the-journey"&gt;How: The Journey&lt;a class="headerlink" href="#how-the-journey" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h1&gt;
&lt;p&gt;We did it together as a team in the open over the course of a few weeks.
&lt;a href="https://github.com/lmwnshn"&gt;Wan Shen Lim&lt;/a&gt; inspired the project by pointing out the benefits of &lt;a href="https://github.com/apache/datafusion/issues/14373"&gt;easy TPC-H
dataset creation&lt;/a&gt;  and &lt;a href="https://github.com/apache/datafusion/issues/14608#issuecomment-2651044600"&gt;suggesting we check out a Java port on February 11,
2025&lt;/a&gt;. Achraf made &lt;a href="https://github.com/clflushopt/tpchgen-rs/commit/53d3402680422a15349ece0a7ea3c3f001018ba0"&gt;first commit a few days later&lt;/a&gt; on February 16, and &lt;a href="https://github.com/clflushopt/tpchgen-rs/commit/9bb386a4c55b8cf93ffac1b98f29b5da990ee79e"&gt;Andrew
and Sean started helping on March 8, 2025&lt;/a&gt; and we &lt;a href="https://crates.io/crates/tpchgen/0.1.0"&gt;released version 0.1&lt;/a&gt; on
March 30, 2025.&lt;/p&gt;
&lt;h2 id="optimizing-single-threaded-performance"&gt;Optimizing Single Threaded Performance&lt;a class="headerlink" href="#optimizing-single-threaded-performance" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;Archaf &lt;a href="https://github.com/clflushopt/tpchgen-rs/pull/16"&gt;completed the end to end conformance tests&lt;/a&gt;, to ensure correctness, and
an initial &lt;a href="https://github.com/clflushopt/tpchgen-rs/pull/12"&gt;cli check in&lt;/a&gt; on March 15, 2025.&lt;/p&gt;
&lt;p&gt;On a Macbook Pro M3 (Nov 2023), the initial performance numbers were actually
slower than the original Java implementation which was ported 😭. This wasn’t
surprising since the focus of the first version was to get a byte of byte
compatible port, and knew about the performance shortcomings and how to approach
them.&lt;/p&gt;
&lt;table class="table"&gt;
&lt;tr&gt;
&lt;td&gt;&lt;strong&gt;Scale Factor&lt;/strong&gt;
&lt;/td&gt;
&lt;td&gt;&lt;strong&gt;Time&lt;/strong&gt;
&lt;/td&gt;
&lt;/tr&gt;
&lt;tr&gt;
&lt;td&gt;1
   &lt;/td&gt;
&lt;td&gt;0m10.307s
   &lt;/td&gt;
&lt;/tr&gt;
&lt;tr&gt;
&lt;td&gt;10
   &lt;/td&gt;
&lt;td&gt;1m26.530s
   &lt;/td&gt;
&lt;/tr&gt;
&lt;tr&gt;
&lt;td&gt;100
   &lt;/td&gt;
&lt;td&gt;14m56.986s
   &lt;/td&gt;
&lt;/tr&gt;
&lt;/table&gt;
&lt;p&gt;&lt;strong&gt;Table 2&lt;/strong&gt;: Performance of running &lt;a href="https://github.com/clflushopt/tpchgen-rs/pull/12"&gt;the initial tpchgen-cli&lt;/a&gt;, measured with
&lt;code&gt;time target/release/tpchgen-cli -s $SCALE_FACTOR&lt;/code&gt;&lt;/p&gt;
&lt;p&gt;With this strong foundation we began optimizing the code using Rust’s low level
memory management to improve performance while retaining memory safely. We spent
several days obsessing over low level details and implemented a textbook like
list of optimizations:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;a href="https://github.com/clflushopt/tpchgen-rs/pull/19"&gt;Avoiding startup overhead&lt;/a&gt;,&lt;/li&gt;
&lt;li&gt;&lt;a href="https://github.com/clflushopt/tpchgen-rs/pull/26"&gt;not&lt;/a&gt; &lt;a href="https://github.com/clflushopt/tpchgen-rs/pull/32"&gt;copying&lt;/a&gt; strings (many more PRs as well)&lt;/li&gt;
&lt;li&gt;&lt;a href="https://github.com/clflushopt/tpchgen-rs/pull/27"&gt;Rust’s zero overhead abstractions for dates&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;&lt;a href="https://github.com/clflushopt/tpchgen-rs/pull/35"&gt;Static strings&lt;/a&gt; (entirely safely with static lifetimes)&lt;/li&gt;
&lt;li&gt;&lt;a href="https://github.com/clflushopt/tpchgen-rs/pull/33"&gt;Generics to avoid virtual function call overhead&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;&lt;a href="https://github.com/clflushopt/tpchgen-rs/pull/62"&gt;Moving lookups from runtime&lt;/a&gt; to load time&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;At the time of writing, single threaded performance is now 2.5x-2.7x faster than the initial version, as shown in Table 3.&lt;/p&gt;
&lt;table class="table"&gt;
&lt;tr&gt;
&lt;td&gt;&lt;strong&gt;Scale Factor&lt;/strong&gt;
&lt;/td&gt;
&lt;td&gt;&lt;strong&gt;Time&lt;/strong&gt;
&lt;/td&gt;
&lt;td&gt;&lt;strong&gt;Times faster&lt;/strong&gt;
&lt;/td&gt;
&lt;/tr&gt;
&lt;tr&gt;
&lt;td&gt;1
   &lt;/td&gt;
&lt;td&gt;0m4.079s
   &lt;/td&gt;
&lt;td&gt;2.5x
   &lt;/td&gt;
&lt;/tr&gt;
&lt;tr&gt;
&lt;td&gt;10
   &lt;/td&gt;
&lt;td&gt;0m31.616s
   &lt;/td&gt;
&lt;td&gt;2.7x
   &lt;/td&gt;
&lt;/tr&gt;
&lt;tr&gt;
&lt;td&gt;100
   &lt;/td&gt;
&lt;td&gt;5m28.083s
   &lt;/td&gt;
&lt;td&gt;2.7x
   &lt;/td&gt;
&lt;/tr&gt;
&lt;/table&gt;
&lt;p&gt;&lt;strong&gt;Table 3&lt;/strong&gt;: Single threaded &lt;code&gt;tpchgen-cli&lt;/code&gt; performance, measured with &lt;code&gt;time target/release/tpchgen-cli -s $SCALE_FACTOR --num-threads=1&lt;/code&gt;&lt;/p&gt;
&lt;h2 id="multi-threading"&gt;Multi-threading&lt;a class="headerlink" href="#multi-threading" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;Then we applied &lt;a href="https://doc.rust-lang.org/book/ch16-00-concurrency.html"&gt;Rust’s fearless concurrency&lt;/a&gt; – with a single, &lt;a href="https://github.com/clflushopt/tpchgen-rs/commit/ab720a70cdc80a711f4a3dda6bac05445106f499"&gt;small PR&lt;/a&gt; (272
net new lines) we updated the same memory safe code to run with multiple threads
and consume bounded memory using &lt;a href="https://thenewstack.io/using-rustlangs-async-tokio-runtime-for-cpu-bound-tasks/"&gt;tokio for the thread scheduler&lt;/a&gt;&lt;sup&gt;4&lt;/sup&gt;.&lt;/p&gt;
&lt;p&gt;As shown in Table 4, with this change, tpchgen-cli generates the full SF=100
dataset in 32 seconds (which is 3.3 GB/sec 🤯). Further investigation reveals
that at SF=100 our generator is actually IO bound (which is not the case for
&lt;code&gt;dbgen&lt;/code&gt; or &lt;code&gt;duckdb&lt;/code&gt;) – it creates data &lt;strong&gt;faster than can be written to an SSD&lt;/strong&gt;.
When writing to &lt;code&gt;/dev/null&lt;/code&gt; tpchgen  generates the entire dataset in 25 seconds
(4 GB/s).&lt;/p&gt;
&lt;table class="table"&gt;
&lt;tr&gt;
&lt;td&gt;&lt;strong&gt;Scale Factor&lt;/strong&gt;
&lt;/td&gt;
&lt;td&gt;&lt;strong&gt;Time&lt;/strong&gt;
&lt;/td&gt;
&lt;td&gt;&lt;strong&gt;Times faster than initial implementation&lt;/strong&gt;
&lt;/td&gt;
&lt;td&gt;&lt;strong&gt;Times faster than optimized single threaded&lt;/strong&gt;
&lt;/td&gt;
&lt;/tr&gt;
&lt;tr&gt;
&lt;td&gt;1
   &lt;/td&gt;
&lt;td&gt;0m1.369s
   &lt;/td&gt;
&lt;td&gt;7.3x
   &lt;/td&gt;
&lt;td&gt;3x
   &lt;/td&gt;
&lt;/tr&gt;
&lt;tr&gt;
&lt;td&gt;10
   &lt;/td&gt;
&lt;td&gt;0m3.828s
   &lt;/td&gt;
&lt;td&gt;22.6x
   &lt;/td&gt;
&lt;td&gt;8.2x
   &lt;/td&gt;
&lt;/tr&gt;
&lt;tr&gt;
&lt;td&gt;100
   &lt;/td&gt;
&lt;td&gt;0m32.615s
   &lt;/td&gt;
&lt;td&gt;27.5x
   &lt;/td&gt;
&lt;td&gt;10x
   &lt;/td&gt;
&lt;/tr&gt;
&lt;tr&gt;
&lt;td&gt;100 (to /dev/null)
   &lt;/td&gt;
&lt;td&gt;0m25.088s
   &lt;/td&gt;
&lt;td&gt;35.7x
   &lt;/td&gt;
&lt;td&gt;13.1x
   &lt;/td&gt;
&lt;/tr&gt;
&lt;/table&gt;
&lt;p&gt;&lt;strong&gt;Table 4&lt;/strong&gt;: tpchgen-cli (multithreaded) performance measured with &lt;code&gt;time target/release/tpchgen-cli -s $SCALE_FACTOR&lt;/code&gt;&lt;/p&gt;
&lt;p&gt;Using Rust and async streams, the data generator is also fully streaming: memory
use does not increase with increasing data size / scale factors&lt;sup&gt;5&lt;/sup&gt;. The DuckDB
generator seems to &lt;a href="https://duckdb.org/docs/stable/extensions/tpch.html#resource-usage-of-the-data-generator"&gt;require far more memory&lt;/a&gt; than is commonly available on
developer laptops and memory use increases with scale factor. With &lt;code&gt;tpchgen-cli&lt;/code&gt;
it is perfectly possible to create data for SF=10000 or larger on a machine with
16GB of memory (assuming sufficient storage capacity).&lt;/p&gt;
&lt;h2 id="direct-to-parquet"&gt;Direct to parquet&lt;a class="headerlink" href="#direct-to-parquet" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;At this point, &lt;code&gt;tpchgen-cli&lt;/code&gt; could very quickly generate the TBL format.
However, as described above, the TBL is annoying to work with, because&lt;/p&gt;
&lt;ol&gt;
&lt;li&gt;It has no header&lt;/li&gt;
&lt;li&gt;It is like a CSV but the delimiter is &lt;code&gt;|&lt;/code&gt;&lt;/li&gt;
&lt;li&gt;Each line ends with an extra &lt;code&gt;|&lt;/code&gt; delimiter before the newline 🙄&lt;/li&gt;
&lt;li&gt;No system that we know can read them without additional configuration.&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;We next &lt;a href="https://github.com/clflushopt/tpchgen-rs/pull/54"&gt;added support for CSV&lt;/a&gt; generation (special thanks &lt;a href="https://github.com/niebayes"&gt;@niebayes&lt;/a&gt; from
Datalayers for finding and &lt;a href="https://github.com/clflushopt/tpchgen-rs/issues/73"&gt;fixing&lt;/a&gt; &lt;a href="https://github.com/clflushopt/tpchgen-rs/issues/65"&gt;bugs&lt;/a&gt;) which performs at the same
speed as TBL. While CSV files are far more standard than TBL, they must still be
parsed prior to load and automatic type inference may not deduce the types
needed for the TPC-H benchmarks (e.g. floating point vs Decimal).&lt;/p&gt;
&lt;p&gt;What would be far more useful is a typed, efficient columnar format such as
Apache Parquet which is supported by all modern query engines. So we &lt;a href="https://github.com/clflushopt/tpchgen-rs/pull/71"&gt;made&lt;/a&gt; a
&lt;a href="https://crates.io/crates/tpchgen-arrow"&gt;tpchgen-arrow&lt;/a&gt; crate to create &lt;a href="https://arrow.apache.org/"&gt;Apache Arrow&lt;/a&gt; arrays directly and then &lt;a href="https://github.com/clflushopt/tpchgen-rs/pull/61"&gt;a small
300 line PR&lt;/a&gt; to feed those arrays to the &lt;a href="https://crates.io/crates/parquet"&gt;Rust Parquet writer&lt;/a&gt;, again using
tokio for parallelized but memory bound work.&lt;/p&gt;
&lt;p&gt;This approach was simple, fast and scalable, as shown in Table 5. Even though
creating Parquet files is significantly more computationally expensive than TBL
or CSV, tpchgen-cli creates the full SF=100 parquet format dataset in less than
45 seconds.&lt;/p&gt;
&lt;table class="table"&gt;
&lt;tr&gt;
&lt;td&gt;&lt;strong&gt;Scale Factor&lt;/strong&gt;
&lt;/td&gt;
&lt;td&gt;&lt;strong&gt;Time to generate Parquet&lt;/strong&gt;
&lt;/td&gt;
&lt;td&gt;&lt;strong&gt;Speed compared to tbl generation&lt;/strong&gt;
&lt;/td&gt;
&lt;/tr&gt;
&lt;tr&gt;
&lt;td&gt;1
   &lt;/td&gt;
&lt;td&gt;0m1.649s
   &lt;/td&gt;
&lt;td&gt;0.8x
   &lt;/td&gt;
&lt;/tr&gt;
&lt;tr&gt;
&lt;td&gt;10
   &lt;/td&gt;
&lt;td&gt;0m5.643s
   &lt;/td&gt;
&lt;td&gt;0.7x
   &lt;/td&gt;
&lt;/tr&gt;
&lt;tr&gt;
&lt;td&gt;100
   &lt;/td&gt;
&lt;td&gt;0m45.243s
   &lt;/td&gt;
&lt;td&gt;0.7x
   &lt;/td&gt;
&lt;/tr&gt;
&lt;tr&gt;
&lt;td&gt;100 (to /dev/null)
   &lt;/td&gt;
&lt;td&gt;0m45.153s
   &lt;/td&gt;
&lt;td&gt;0.5x
   &lt;/td&gt;
&lt;/tr&gt;
&lt;/table&gt;
&lt;p&gt;&lt;strong&gt;Table 5&lt;/strong&gt;: &lt;code&gt;tpchgen-cli&lt;/code&gt; Parquet generation performance measured with  &lt;code&gt;time
target/release/tpchgen-cli -s $SCALE_FACTOR --format=parquet&lt;/code&gt;&lt;/p&gt;
&lt;h1 id="conclusion"&gt;Conclusion 👊🎤&lt;a class="headerlink" href="#conclusion" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h1&gt;
&lt;p&gt;In just a few days, with some fellow database nerds and the power of Rust, we built something 10x better than what currently exists. We hope it inspires more research
into analytical systems using the TPC-H dataset and that people build awesome
things with it. For example, Sean has already added &lt;a href="https://github.com/GlareDB/glaredb/pull/3549"&gt;on-demand generation of
tables to GlareDB&lt;/a&gt;. Please consider joining us and helping out at
&lt;a href="https://github.com/clflushopt/tpchgen-rs"&gt;https://github.com/clflushopt/tpchgen-rs&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;We met while working together on Apache DataFusion in various capacities. If you
are looking for a community of like minded people hacking on databases, we
welcome you to &lt;a href="https://datafusion.apache.org/contributor-guide/communication.html"&gt;come join us&lt;/a&gt;. We are in the process of integrating this into
DataFusion (see &lt;a href="https://github.com/apache/datafusion/issues/14608"&gt;apache/datafusion#14608&lt;/a&gt;) if you are interested in helping 🎣&lt;/p&gt;
&lt;h1 id="about-the-authors"&gt;About the Authors:&lt;a class="headerlink" href="#about-the-authors" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h1&gt;
&lt;ul&gt;
&lt;li&gt;&lt;a href="https://www.linkedin.com/in/andrewalamb/"&gt;Andrew Lamb&lt;/a&gt; (&lt;a href="https://github.com/alamb"&gt;@alamb&lt;/a&gt;) is a Staff Engineer at &lt;a href="https://www.influxdata.com/"&gt;InfluxData&lt;/a&gt; and a PMC member of &lt;a href="https://datafusion.apache.org/"&gt;Apache DataFusion&lt;/a&gt; and &lt;a href="https://arrow.apache.org/"&gt;Apache Arrow&lt;/a&gt;.&lt;/li&gt;
&lt;li&gt;Achraf B (&lt;a href="https://github.com/clflushopt"&gt;@clflushopt&lt;/a&gt;) is a Software Engineer at &lt;a href="https://optable.co/"&gt;Optable&lt;/a&gt; where he works on data infrastructure.&lt;/li&gt;
&lt;li&gt;&lt;a href="https://www.linkedin.com/in/scsmithr/"&gt;Sean Smith&lt;/a&gt; (&lt;a href="https://github.com/scsmithr"&gt;@scsmithr&lt;/a&gt;) is the founder of &lt;a href="https://glaredb.com/"&gt;GlareDB&lt;/a&gt; focused on building a fast analytics database.&lt;/li&gt;
&lt;/ul&gt;
&lt;!-- Footnotes themselves at the bottom. --&gt;
&lt;h2 id="footnotes"&gt;Footnotes&lt;a class="headerlink" href="#footnotes" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;&lt;em&gt;1&lt;/em&gt;: Actual Time: &lt;code&gt;30:35&lt;/code&gt;&lt;/p&gt;
&lt;p&gt;&lt;em&gt;2&lt;/em&gt;: It is possible to embed the dbgen code, which appears to be the approach taken by DuckDB. This approach was tried in GlareDB (&lt;a href="https://github.com/GlareDB/glaredb/pull/3313"&gt;GlareDB/glaredb#3313&lt;/a&gt;), but ultimately shelved given the amount of effort needed to adapt and isolate the dbgen code.&lt;/p&gt;
&lt;p&gt;&lt;em&gt;3&lt;/em&gt;: It is pretty amazing to imagine the machine required to generate SF300 that had 1.8TB (!!) of RAM&lt;/p&gt;
&lt;p&gt;&lt;em&gt;4&lt;/em&gt;: We tried to &lt;a href="https://github.com/clflushopt/tpchgen-rs/pull/34"&gt;use Rayon (see discussion here)&lt;/a&gt;, but could not easily keep memory bounded.&lt;/p&gt;
&lt;p&gt;&lt;em&gt;5&lt;/em&gt;: &lt;code&gt;tpchgen-cli&lt;/code&gt; memory usage is a function of the number of threads:  each thread needs some buffer space&lt;/p&gt;</content><category term="blog"/></entry><entry><title>Apache DataFusion Python 46.0.0 Released</title><link href="https://datafusion.apache.org/blog/2025/03/30/datafusion-python-46.0.0" rel="alternate"/><published>2025-03-30T00:00:00+00:00</published><updated>2025-03-30T00:00:00+00:00</updated><author><name>timsaucer</name></author><id>tag:datafusion.apache.org,2025-03-30:/blog/2025/03/30/datafusion-python-46.0.0</id><summary type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;p&gt;We are happy to announce that &lt;a href="https://pypi.org/project/datafusion/46.0.0/"&gt;datafusion-python 46.0.0&lt;/a&gt; has been released. This release
brings in all of the new features of the core &lt;a href="https://datafusion.apache.org/blog/2025/03/24/datafusion-46.0.0"&gt;DataFusion 46.0.0&lt;/a&gt; library. Since the last
blog post for &lt;a href="https://datafusion.apache.org/blog/2024/12/14/datafusion-python-43.1.0/"&gt;datafusion-python 43.1.0&lt;/a&gt;, a large number of improvements have been made
that can …&lt;/p&gt;</summary><content type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;p&gt;We are happy to announce that &lt;a href="https://pypi.org/project/datafusion/46.0.0/"&gt;datafusion-python 46.0.0&lt;/a&gt; has been released. This release
brings in all of the new features of the core &lt;a href="https://datafusion.apache.org/blog/2025/03/24/datafusion-46.0.0"&gt;DataFusion 46.0.0&lt;/a&gt; library. Since the last
blog post for &lt;a href="https://datafusion.apache.org/blog/2024/12/14/datafusion-python-43.1.0/"&gt;datafusion-python 43.1.0&lt;/a&gt;, a large number of improvements have been made
that can be found in the &lt;a href="https://github.com/apache/datafusion-python/tree/main/dev/changelog"&gt;changelogs&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;We highly recommend reviewing the upstream &lt;a href="https://datafusion.apache.org/blog/2025/03/24/datafusion-46.0.0"&gt;DataFusion 46.0.0&lt;/a&gt; announcement.&lt;/p&gt;
&lt;h2 id="easier-file-reading"&gt;Easier file reading&lt;a class="headerlink" href="#easier-file-reading" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;In these releases we have introduced two new ways to more easily read files into
DataFrames.&lt;/p&gt;
&lt;p&gt;PR &lt;a href="https://github.com/apache/datafusion-python/pull/982"&gt;#982&lt;/a&gt; introduced a series of easier read functions for Parquet, JSON, CSV, and
AVRO files. This introduces a concept of a global context that is available by
default when using these methods. Now instead of creating a default Session
Context and then calling the read methods, you can simply import these read
alternative methods and begin working with your DataFrames. Below is an example of
how easy to use this new approach is.&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-python"&gt;from datafusion.io import read_parquet
df = read_parquet(path="./examples/tpch/data/customer.parquet")
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;PR &lt;a href="https://github.com/apache/datafusion-python/pull/980"&gt;#980&lt;/a&gt; adds a method for setting up a session context to use URL tables. With
this enabled, you can use a path to a local file as a table name. An example
of how to use this is demonstrated in the following snippet.&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-python"&gt;import datafusion
ctx = datafusion.SessionContext().enable_url_table()
df = ctx.table("./examples/tpch/data/customer.parquet")
&lt;/code&gt;&lt;/pre&gt;
&lt;h2 id="registering-table-views"&gt;Registering Table Views&lt;a class="headerlink" href="#registering-table-views" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;DataFusion supports registering a logical plan as a view with a session context. This
allows creating views in one part of your work flow and passinng the session
context to other places where that logical plan can be reused. This is an useful
feature for building up complex workflows and for code clarity. PR &lt;a href="https://github.com/apache/datafusion-python/pull/1016"&gt;#1016&lt;/a&gt; enables this
feature in &lt;code&gt;datafusion-python&lt;/code&gt;.&lt;/p&gt;
&lt;p&gt;For example, supposing you have a DataFrame called &lt;code&gt;df1&lt;/code&gt;, you could use this code snippet
to register the view and then use it in another place:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-python"&gt;ctx.register_view("view1", df1)
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;And then in another portion of your code which has access to the same session context
you can retrieve the DataFrame with:&lt;/p&gt;
&lt;pre&gt;&lt;code&gt;df2 = ctx.table("view1")
&lt;/code&gt;&lt;/pre&gt;
&lt;h2 id="asynchronous-iteration-of-record-batches"&gt;Asynchronous Iteration of Record Batches&lt;a class="headerlink" href="#asynchronous-iteration-of-record-batches" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;Retrieving a &lt;code&gt;RecordBatch&lt;/code&gt; from a &lt;code&gt;RecordBatchStream&lt;/code&gt; was a synchronous call, which would
require the end user's code to wait for the data retrieval. This is described in
&lt;a href="https://github.com/apache/datafusion-python/issues/974"&gt;Issue 974&lt;/a&gt;. We continue to support this as a synchronous iterator, but we have also added
in the ability to retrieve the &lt;code&gt;RecordBatch&lt;/code&gt; using the Python asynchronous &lt;code&gt;anext&lt;/code&gt;
function.&lt;/p&gt;
&lt;h2 id="default-zstd-compression-for-parquet-files"&gt;Default ZSTD Compression for Parquet files&lt;a class="headerlink" href="#default-zstd-compression-for-parquet-files" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;With PR &lt;a href="https://github.com/apache/datafusion-python/pull/981"&gt;#981&lt;/a&gt;, we change the saving of Parquet files to use zstd compression by default.
Previously the default was uncompressed, causing excessive disk storage. Zstd is an
excellent compression scheme that balances speed and compression ratio. Users can still
save their Parquet files uncompressed by passing in the appropriate value to the
&lt;code&gt;compression&lt;/code&gt; argument when calling &lt;code&gt;DataFrame.write_parquet&lt;/code&gt;.&lt;/p&gt;
&lt;h2 id="udf-decorators"&gt;UDF Decorators&lt;a class="headerlink" href="#udf-decorators" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;In PRs &lt;a href="https://github.com/apache/datafusion-python/pull/1040"&gt;#1040&lt;/a&gt; and &lt;a href="https://github.com/apache/datafusion-python/pull/1061"&gt;#1061&lt;/a&gt; we add methods to make creating user defined functions
easier and take advantage of Python decorators. With these PRs you can save a step
from defining a method and then defining a udf of that method. Instead you can
simply add the appropriate &lt;code&gt;udf&lt;/code&gt; decorator. Similar methods exist for aggregate
and window user defined functions.&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-python"&gt;@udf([pa.int64(), pa.int64()], pa.bool_(), "stable")
def my_custom_function(
    age: pa.Array,
    favorite_number: pa.Array,
) -&amp;gt; pa.Array:
    pass
&lt;/code&gt;&lt;/pre&gt;
&lt;h2 id="uv-package-management"&gt;&lt;code&gt;uv&lt;/code&gt; package management&lt;a class="headerlink" href="#uv-package-management" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;&lt;a href="https://github.com/astral-sh/uv"&gt;uv&lt;/a&gt; is an extremely fast Python package manager, written in Rust. In the previous version
of &lt;code&gt;datafusion-python&lt;/code&gt; we had a combination of settings of PyPi and Conda. Instead, we
switch to using &lt;a href="https://github.com/astral-sh/uv"&gt;uv&lt;/a&gt; is our primary method for dependency management.&lt;/p&gt;
&lt;p&gt;For most users of DataFusion, this change will be transparent. You can still install
via &lt;code&gt;pip&lt;/code&gt; or &lt;code&gt;conda&lt;/code&gt;. For developers, the instructions in the repository have been updated.&lt;/p&gt;
&lt;h2 id="code-cleanup"&gt;Code cleanup&lt;a class="headerlink" href="#code-cleanup" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;In an effort to improve our code cleanliness and ensure we are following Python best
practices, we use &lt;a href="https://docs.astral.sh/ruff/"&gt;ruff&lt;/a&gt; to perform Python linting. Until now we enabled only a portion
of the available linters available. In PRs &lt;a href="https://github.com/apache/datafusion-python/pull/1055"&gt;#1055&lt;/a&gt; and &lt;a href="https://github.com/apache/datafusion-python/pull/1062"&gt;#1062&lt;/a&gt;, we enable many more
of these linters and made code improvements to ensure we are following these
recommendations.&lt;/p&gt;
&lt;h2 id="improved-jupyter-notebook-rendering"&gt;Improved Jupyter Notebook rendering&lt;a class="headerlink" href="#improved-jupyter-notebook-rendering" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;Since PR &lt;a href="https://github.com/apache/datafusion-python/pull/839"&gt;#839&lt;/a&gt; in DataFusion 41.0.0 we have been able to render DataFrames using html in
&lt;a href="https://jupyter.org/"&gt;jupyter&lt;/a&gt; notebooks. This is a big improvement over the &lt;code&gt;show&lt;/code&gt; command when we have the
ability to render tables. In PR &lt;a href="https://github.com/apache/datafusion-python/pull/1036"&gt;#1036&lt;/a&gt; we went a step further and added in a variety
of features.&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;Now html tables are scrollable, vertically and horizontally.&lt;/li&gt;
&lt;li&gt;When data are truncated, we report this to the user.&lt;/li&gt;
&lt;li&gt;Instead of showing a small number of rows, we collect up to 2 megabytes of data to
display. Since we have scrollable tables, we are able to make more data available
to the user without sacrificing notebook usability.&lt;/li&gt;
&lt;li&gt;We report explicitly when the DataFrame is empty. Previously we would not output
anything for an empty table. This indicator is helpful to users to ensure their plans
are written correctly. Sometimes a non-output can be overlooked.&lt;/li&gt;
&lt;li&gt;For long output of data, we generate a collapsed view of the data with an option
for the user to click on it to expand the data.&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;In the below view you can see an example of some of these features such as the
expandable text and scroll bars.&lt;/p&gt;
&lt;figure class="text-center"&gt;
&lt;img alt="Fig 1: Example html rendering in a jupyter notebook." class="img-fluid" src="/blog/images/python-datafusion-46.0.0/html_rendering.png"/&gt;
&lt;figcaption&gt;
&lt;b&gt;Figure 1&lt;/b&gt;: With the html rendering enhancements, tables are more easily
   viewable in jupyter notebooks.
&lt;/figcaption&gt;
&lt;/figure&gt;
&lt;h2 id="extension-documentation"&gt;Extension Documentation&lt;a class="headerlink" href="#extension-documentation" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;We have recently added &lt;a href="https://datafusion.apache.org/python/contributor-guide/ffi.html"&gt;Extension Documentation&lt;/a&gt; to the DataFusion in Python website. We
have received many requests about how to better understand how to integrate DataFusion
in Python with other Rust libraries. To address these questions we wrote an article about
some of the difficulties that we encounter when using Rust libraries in Python and our
approach to addressing them.&lt;/p&gt;
&lt;h2 id="migration-guide"&gt;Migration Guide&lt;a class="headerlink" href="#migration-guide" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;During the upgrade from &lt;a href="https://github.com/apache/datafusion/blob/main/dev/changelog/43.0.0.md"&gt;DataFusion 43.0.0&lt;/a&gt; to &lt;a href="https://github.com/apache/datafusion/blob/main/dev/changelog/44.0.0.md"&gt;DataFusion 44.0.0&lt;/a&gt; as our upstream core
dependency, we discovered a few changes were necessary within our repository and our
unit tests. These notes serve to help guide users who may encounter similar issues when
upgrading.&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;code&gt;RuntimeConfig&lt;/code&gt; is now deprecated in favor of &lt;code&gt;RuntimeEnvBuilder&lt;/code&gt;. The migration is
fairly straightforward, and the corresponding classes have been marked as deprecated. For
end users it should be simply a matter of changing the class name.&lt;/li&gt;
&lt;li&gt;If you perform a &lt;code&gt;concat&lt;/code&gt; of a &lt;code&gt;string_view&lt;/code&gt; and &lt;code&gt;string&lt;/code&gt;, it will now return a
&lt;code&gt;string_view&lt;/code&gt; instead of a &lt;code&gt;string&lt;/code&gt;. This likely only impacts unit tests that are validating
return types. In general, it is recommended to switch to using &lt;code&gt;string_view&lt;/code&gt; whenever 
possible. You can see the blog articles &lt;a href="https://datafusion.apache.org/blog/2024/09/13/string-view-german-style-strings-part-1/"&gt;String View Pt 1&lt;/a&gt; and &lt;a href="https://datafusion.apache.org/blog/2024/09/13/string-view-german-style-strings-part-2/"&gt;Pt 2&lt;/a&gt; for more information
on these performance improvements.&lt;/li&gt;
&lt;li&gt;The function &lt;code&gt;date_part&lt;/code&gt; now returns an &lt;code&gt;int32&lt;/code&gt; instead of a &lt;code&gt;float64&lt;/code&gt;. This is likely
only impactful to unit tests.&lt;/li&gt;
&lt;li&gt;We have upgraded the Python minimum version to 3.9 since 3.8 is no longer officially
supported.&lt;/li&gt;
&lt;/ul&gt;
&lt;h2 id="coming-soon"&gt;Coming Soon&lt;a class="headerlink" href="#coming-soon" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;There is a lot of excitement around the upcoming work. This list is not comprehensive, but
a glimpse into some of the upcoming work includes:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;Reusable DataFusion UDFs: The way user defined functions are currently written in
&lt;code&gt;datafusion-python&lt;/code&gt; is slightly different from those written for the upstream Rust
&lt;code&gt;datafusion&lt;/code&gt;. The core ideas are usually the same, but it means it takes effort for users
to re-implement functions already written for Rust projects to be usable in Python. Issue
&lt;a href="https://github.com/apache/datafusion-python/issues/1017"&gt;#1017&lt;/a&gt; addresses this topic. Work is well underway to make it easier to expose these
user functions through the FFI boundary. This means that the work that already exists in
repositories such as those found in the &lt;a href="https://github.com/datafusion-contrib"&gt;datafusion-contrib&lt;/a&gt; project can be easily
re-used in Python. This will provide a low effort way to expose significant functionality
to the DataFusion in Python community.&lt;/li&gt;
&lt;li&gt;Additional table providers: We have work well underway to provide a host of table providers
to &lt;code&gt;datafusion-python&lt;/code&gt; including: sqlite, duckdb, postgres, odbc, and mysql! In
&lt;a href="https://github.com/datafusion-contrib/datafusion-table-providers/issues/279"&gt;datafusion-contrib #279&lt;/a&gt; we track the progress of this excellent work. Once complete, users
will be able to &lt;code&gt;pip install&lt;/code&gt; this library and get easy access to all of these table
providers. This is another way we are leveraging the FFI work to greatly expand the usability
of &lt;code&gt;datafusion-python&lt;/code&gt; with relatively low effort.&lt;/li&gt;
&lt;li&gt;External catalog and schema providers: For users who wish to go beyond table providers
and have an entire custom catalog with schema, Issue &lt;a href="https://github.com/apache/datafusion-python/issues/1091"&gt;#1091&lt;/a&gt; tracks the progress of exposing
this in Python. With this work, if you have already written a Rust based table catalog you
will be able to interface it in Python similar to the work described for the table
providers above.&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;This is only a sample of the great work that is being done. If there are features you would
love to see, we encourage you to open an issue and join us as we build something wonderful.&lt;/p&gt;
&lt;h2 id="appreciation"&gt;Appreciation&lt;a class="headerlink" href="#appreciation" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;We would like to thank everyone who has helped with these releases through their helpful
conversations, code review, issue descriptions, and code authoring. We would especially
like to thank the following authors of PRs who made these releases possible, listed in
alphabetical order by username: &lt;a href="https://github.com/chenkovsky"&gt;@chenkovsky&lt;/a&gt;, &lt;a href="https://github.com/CrystalZhou0529"&gt;@CrystalZhou0529&lt;/a&gt;, &lt;a href="https://github.com/ion-elgreco"&gt;@ion-elgreco&lt;/a&gt;,
&lt;a href="https://github.com/jsai28"&gt;@jsai28&lt;/a&gt;, &lt;a href="https://github.com/kevinjqliu"&gt;@kevinjqliu&lt;/a&gt;, &lt;a href="https://github.com/kylebarron"&gt;@kylebarron&lt;/a&gt;, &lt;a href="https://github.com/kosiew"&gt;@kosiew&lt;/a&gt;, &lt;a href="https://github.com/nirnayroy"&gt;@nirnayroy&lt;/a&gt;, and &lt;a href="https://github.com/Spaarsh"&gt;@Spaarsh&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;Thank you!&lt;/p&gt;
&lt;h2 id="get-involved"&gt;Get Involved&lt;a class="headerlink" href="#get-involved" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;The DataFusion Python team is an active and engaging community and we would love
to have you join us and help the project.&lt;/p&gt;
&lt;p&gt;Here are some ways to get involved:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;
&lt;p&gt;Learn more by visiting the &lt;a href="https://datafusion.apache.org/python/index.html"&gt;DataFusion Python project&lt;/a&gt; page.&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;
&lt;p&gt;Try out the project and provide feedback, file issues, and contribute code.&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;
&lt;p&gt;Join us on &lt;a href="https://s.apache.org/slack-invite"&gt;ASF Slack&lt;/a&gt; or the &lt;a href="https://discord.gg/Qw5gKqHxUM"&gt;Arrow Rust Discord Server&lt;/a&gt;.&lt;/p&gt;
&lt;/li&gt;
&lt;/ul&gt;</content><category term="blog"/></entry><entry><title>Apache DataFusion 46.0.0 Released</title><link href="https://datafusion.apache.org/blog/2025/03/24/datafusion-46.0.0" rel="alternate"/><published>2025-03-24T00:00:00+00:00</published><updated>2025-03-24T00:00:00+00:00</updated><author><name>Oznur Hanci and Berkay Sahin on behalf of the PMC</name></author><id>tag:datafusion.apache.org,2025-03-24:/blog/2025/03/24/datafusion-46.0.0</id><summary type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;p&gt;We’re excited to announce the release of &lt;strong&gt;Apache DataFusion 46.0.0&lt;/strong&gt;! This new version represents a significant milestone for the project, packing in a wide range of improvements and fixes. You can find the complete details in the full &lt;a href="https://github.com/apache/datafusion/blob/branch-46/dev/changelog/46.0.0.md"&gt;changelog&lt;/a&gt;. We’ll highlight the most important changes below …&lt;/p&gt;</summary><content type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;p&gt;We’re excited to announce the release of &lt;strong&gt;Apache DataFusion 46.0.0&lt;/strong&gt;! This new version represents a significant milestone for the project, packing in a wide range of improvements and fixes. You can find the complete details in the full &lt;a href="https://github.com/apache/datafusion/blob/branch-46/dev/changelog/46.0.0.md"&gt;changelog&lt;/a&gt;. We’ll highlight the most important changes below and guide you through upgrading.&lt;/p&gt;
&lt;h2 id="breaking-changes"&gt;Breaking Changes&lt;a class="headerlink" href="#breaking-changes" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;DataFusion 46.0.0 brings a few &lt;strong&gt;breaking changes&lt;/strong&gt; that may require adjustments to your code as described in the &lt;a href="https://datafusion.apache.org/library-user-guide/upgrading.html"&gt;Upgrade Guide&lt;/a&gt;. Here are the most notable ones:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;a href="https://github.com/apache/datafusion/pull/14224#"&gt;Unified &lt;code&gt;DataSourceExec&lt;/code&gt; Execution Plan&lt;/a&gt;&lt;strong&gt;:&lt;/strong&gt; DataFusion 46.0.0 introduces a major refactor of scan operators. The separate file-format-specific execution plan nodes (&lt;code&gt;ParquetExec&lt;/code&gt;, &lt;code&gt;CsvExec&lt;/code&gt;, &lt;code&gt;JsonExec&lt;/code&gt;, &lt;code&gt;AvroExec&lt;/code&gt;, etc.) have been &lt;strong&gt;deprecated and merged into a single &lt;code&gt;DataSourceExec&lt;/code&gt; plan&lt;/strong&gt;. Format-specific logic is now encapsulated in new &lt;code&gt;DataSource&lt;/code&gt; and &lt;code&gt;FileSource&lt;/code&gt; traits. This change simplifies the execution model, but if you have code that directly references the old plan nodes, you’ll need to update it to use &lt;code&gt;DataSourceExec&lt;/code&gt; (see the &lt;a href="https://datafusion.apache.org/library-user-guide/upgrading.html"&gt;Upgrade Guide&lt;/a&gt; for examples of the new API).&lt;/li&gt;
&lt;li&gt;&lt;a href="https://github.com/apache/arrow-datafusion/issues/7360#:~:text=2"&gt;**Error Handling Improvements&lt;/a&gt; (&lt;code&gt;DataFusionError::Collection&lt;/code&gt;):** We began overhauling DataFusion’s approach to error handling. In this release, a new error variant &lt;code&gt;DataFusionError::Collection&lt;/code&gt; (and related mechanisms) has been introduced to aggregate multiple errors into one. This is part of a broader effort to provide richer error context and reduce internal panics. As a result, some error types or messages have changed. Downstream code that matches on specific &lt;code&gt;DataFusionError&lt;/code&gt; variants might need adjustment.&lt;/li&gt;
&lt;/ul&gt;
&lt;h2 id="performance-improvements"&gt;Performance Improvements&lt;a class="headerlink" href="#performance-improvements" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;DataFusion 46.0.0 comes with a slew of performance enhancements across the board. Here are some of the noteworthy optimizations in this release:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;strong&gt;Faster &lt;code&gt;median()&lt;/code&gt; (no grouping):&lt;/strong&gt; The &lt;code&gt;median()&lt;/code&gt; aggregate function got a special fast path when used without a &lt;code&gt;GROUP BY&lt;/code&gt;. By optimizing its accumulator, median calculation is about &lt;strong&gt;2× faster&lt;/strong&gt; in the single-group case. If you use &lt;code&gt;MEDIAN()&lt;/code&gt; on large datasets (especially as a single value), you should notice reduced query times (PR &lt;a href="https://github.com/apache/datafusion/pull/14399"&gt;#14399&lt;/a&gt; by &lt;a href="https://github.com/2010YOUY01"&gt;@2010YOUY01&lt;/a&gt;).&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Optimized &lt;code&gt;FIRST_VALUE&lt;/code&gt;/&lt;code&gt;LAST_VALUE&lt;/code&gt;:&lt;/strong&gt; The &lt;code&gt;FIRST_VALUE&lt;/code&gt; and &lt;code&gt;LAST_VALUE&lt;/code&gt; window functions have been improved by avoiding an internal sort of rows. Instead of sorting each partition, the implementation now uses a direct approach to pick the first/last element. This yields &lt;strong&gt;10–100% performance improvement&lt;/strong&gt; for these functions, depending on the scenario. Queries using &lt;code&gt;FIRST_VALUE(...) OVER (PARTITION BY ... ORDER BY ...)&lt;/code&gt; will run faster, especially when partitions are large (PR &lt;a href="https://github.com/apache/datafusion/pull/14402"&gt;#14402&lt;/a&gt; by &lt;a href="https://github.com/blaginin"&gt;@blaginin&lt;/a&gt;).&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;&lt;code&gt;repeat()&lt;/code&gt; String Function Boost:&lt;/strong&gt; Repeating strings is now more efficient – the &lt;code&gt;repeat(text, n)&lt;/code&gt; function was optimized by about &lt;strong&gt;50%&lt;/strong&gt;. This was achieved by reducing allocations and using a more efficient concatenation strategy. If you generate large repeated strings in queries, this can cut the time nearly in half (PR &lt;a href="https://github.com/apache/datafusion/pull/14697"&gt;#14697&lt;/a&gt; by &lt;a href="https://github.com/zjregee"&gt;@zjregee&lt;/a&gt;).&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Ultra-fast &lt;code&gt;uuid()&lt;/code&gt; UDF:&lt;/strong&gt; The &lt;code&gt;uuid()&lt;/code&gt; function (which generates random UUID strings) received a major speed-up. It’s now roughly &lt;strong&gt;40× faster&lt;/strong&gt; than before! The new implementation avoids unnecessary string copying and uses a more direct conversion to hex, making bulk UUID generation far more practical (PR &lt;a href="https://github.com/apache/datafusion/pull/14675"&gt;#14675&lt;/a&gt; by &lt;a href="https://github.com/simonvandel"&gt;@simonvandel&lt;/a&gt;).&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Accelerated &lt;code&gt;chr()&lt;/code&gt; and &lt;code&gt;to_hex()&lt;/code&gt;:&lt;/strong&gt; Several scalar functions have been micro-optimized. The &lt;code&gt;chr()&lt;/code&gt; function (which returns the character for a given ASCII code) is about &lt;strong&gt;4× faster&lt;/strong&gt; now, and the &lt;code&gt;to_hex()&lt;/code&gt; function (which converts numbers to hex string) is roughly &lt;strong&gt;2× faster&lt;/strong&gt;. These improvements may be most noticeable in tight loops or when these functions are applied to large arrays of values (PR &lt;a href="https://github.com/apache/datafusion/pull/14700"&gt;#14700&lt;/a&gt; for &lt;code&gt;chr&lt;/code&gt;, &lt;a href="https://github.com/apache/datafusion/pull/14686"&gt;#14686&lt;/a&gt; for &lt;code&gt;to_hex&lt;/code&gt; by &lt;a href="https://github.com/simonvandel"&gt;@simonvandel&lt;/a&gt;).&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;No More RowConverter in Grouped Ordering:&lt;/strong&gt; We removed an inefficient step in the &lt;em&gt;partial grouping&lt;/em&gt; algorithm. The &lt;code&gt;GroupOrderingPartial&lt;/code&gt; operator no longer converts data to “row format” for each batch (via &lt;code&gt;RowConverter&lt;/code&gt;). Instead, it uses a direct arrow-based approach to detect sort key changes. This eliminated overhead and yields a nice speedup for certain aggregation queries. (PR &lt;a href="https://github.com/apache/datafusion/pull/14566"&gt;#14566&lt;/a&gt; by &lt;a href="https://github.com/ctsk"&gt;@ctsk&lt;/a&gt;).&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Predicate Pruning for &lt;code&gt;NOT LIKE&lt;/code&gt;:&lt;/strong&gt; DataFusion’s parquet reader can now prune row groups using &lt;code&gt;NOT LIKE&lt;/code&gt; filters, similar to how it handles &lt;code&gt;LIKE&lt;/code&gt;. This means if you have a filter such as &lt;code&gt;column NOT LIKE 'prefix%'&lt;/code&gt;, DataFusion can use min/max statistics to skip reading files/parts that can be determined to either entirely match or not match the predicate. In particular, a pattern like &lt;code&gt;NOT LIKE 'X%'&lt;/code&gt; can skip data ranges that definitely start with "X". While a niche case, it contributes to query efficiency in those scenarios (PR &lt;a href="https://github.com/apache/datafusion/pull/14567"&gt;#14567&lt;/a&gt; by &lt;a href="https://github.com/UBarney"&gt;@UBarney&lt;/a&gt;).&lt;/li&gt;
&lt;/ul&gt;
&lt;h2 id="google-summer-of-code-2025"&gt;Google Summer of Code 2025&lt;a class="headerlink" href="#google-summer-of-code-2025" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;Another exciting development: &lt;strong&gt;Apache DataFusion has been accepted as a mentoring organization for Google Summer of Code (GSoC) 2025&lt;/strong&gt;! 🎉 This means that this summer, students from around the world will have the opportunity to contribute to DataFusion under the guidance of our committers. We have put together &lt;a href="https://datafusion.apache.org/contributor-guide/gsoc_project_ideas.html"&gt;a list of project ideas&lt;/a&gt; that candidates can choose from.&lt;/p&gt;
&lt;p&gt;If you’re interested, check out our &lt;a href="https://datafusion.apache.org/contributor-guide/gsoc_application_guidelines.html"&gt;GSoC Application Guidelines&lt;/a&gt;. We encourage students to reach out, discuss ideas with us, and apply.&lt;/p&gt;
&lt;h2 id="highlighted-new-features"&gt;Highlighted New Features&lt;a class="headerlink" href="#highlighted-new-features" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;h3 id="improved-diagnostics"&gt;Improved Diagnostics&lt;a class="headerlink" href="#improved-diagnostics" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;DataFusion 46.0.0 introduces a new &lt;a href="https://github.com/apache/datafusion/issues/14429"&gt;&lt;strong&gt;SQL Diagnostics framework&lt;/strong&gt;&lt;/a&gt; to make error messages more understandable. This comes in the form of new &lt;code&gt;Diagnostic&lt;/code&gt; and &lt;code&gt;DiagnosticEntry&lt;/code&gt; types, which allow the system to attach rich context (like source query text spans) to error messages. In practical terms, certain planner errors will now point to the exact location in your SQL query that caused the issue. &lt;/p&gt;
&lt;p&gt;For example, if you reference an unknown table or miss a column in &lt;code&gt;GROUP BY&lt;/code&gt; the error message will include the query snippet causing the error. These diagnostics are meant for end-users of applications built on DataFusion, providing clearer messages instead of generic errors. Here’s an example:&lt;/p&gt;
&lt;p&gt;&lt;img alt="diagnostic-example" class="img-fluid" src="/blog/images/datafusion-46.0.0/diagnostic-example.png" width="80%"/&gt;&lt;/p&gt;
&lt;p&gt;Currently, diagnostics cover unresolved table/column references, missing &lt;code&gt;GROUP BY&lt;/code&gt; columns, ambiguous references, wrong number of UNION columns, type mismatches, and a few others. Future releases will extend this to more error types. This feature should greatly ease debugging of complex SQL by pinpointing errors directly in the query text. We thank &lt;a href="https://github.com/eliaperantoni"&gt;@eliaperantoni&lt;/a&gt; for his contributions in this project.&lt;/p&gt;
&lt;h3 id="unified-datasourceexec-for-table-providers"&gt;Unified &lt;code&gt;DataSourceExec&lt;/code&gt; for Table Providers&lt;a class="headerlink" href="#unified-datasourceexec-for-table-providers" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;As mentioned, DataFusion now uses a unified &lt;code&gt;DataSourceExec&lt;/code&gt; for reading tables, which is both a breaking change and a feature. &lt;em&gt;Why is this important?&lt;/em&gt; The new approach simplifies how custom table providers are integrated and optimized. Namely, the optimizer can treat file scans uniformly and push down filters/limits more consistently when there is one execution plan that handles all data sources. The new &lt;code&gt;DataSourceExec&lt;/code&gt; is paired with a &lt;code&gt;DataSource&lt;/code&gt; trait that encapsulates format-specific behaviors (Parquet, CSV, JSON, Avro, etc.) in a pluggable way.&lt;/p&gt;
&lt;p&gt;All built-in sources (Parquet, CSV, Avro, Arrow, JSON, etc.) have been migrated to this framework. This unification makes the codebase cleaner and sets the stage for future enhancements (like consistent metadata handling and limit pushdown across all formats). Check out PR &lt;a href="https://github.com/apache/datafusion/pull/14224"&gt;#14224&lt;/a&gt; for design details. We thank &lt;a href="https://github.com/mertak-synnada"&gt;@mertak-synnada&lt;/a&gt; and &lt;a href="https://github.com/ozankabak"&gt;@ozankabak&lt;/a&gt; for their contributions.&lt;/p&gt;
&lt;h3 id="ffi-support-for-scalar-udfs"&gt;FFI Support for Scalar UDFs&lt;a class="headerlink" href="#ffi-support-for-scalar-udfs" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;DataFusion’s Foreign Function Interface (FFI) has been extended to support &lt;a href="https://github.com/apache/datafusion/pull/14579"&gt;&lt;strong&gt;user-defined scalar functions&lt;/strong&gt;&lt;/a&gt; defined in external languages. In 46.0.0, you can now expose a custom scalar UDF through the FFI layer and use it in DataFusion as if it were built-in. This is particularly exciting for the &lt;strong&gt;Python bindings&lt;/strong&gt; and other language integrations – it means you could define a function in Python (or C, etc.) and register it with DataFusion’s Rust core via the FFI crate. Thanks, &lt;a href="https://github.com/timsaucer"&gt;@timsaucer&lt;/a&gt;!&lt;/p&gt;
&lt;h3 id="new-statisticsdistribution-framework"&gt;New Statistics/Distribution Framework&lt;a class="headerlink" href="#new-statisticsdistribution-framework" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;This release, thanks mainly to &lt;a href="https://github.com/Fly-Style"&gt;@Fly-Style&lt;/a&gt; with contributions from &lt;a href="https://github.com/ozankabak"&gt;@ozankabak&lt;/a&gt; and &lt;a href="https://github.com/berkaysynnada"&gt;@berkaysynnada&lt;/a&gt;, includes the initial pieces of a &lt;a href="https://github.com/apache/datafusion/pull/14699"&gt;**redesigned statistics framework&lt;/a&gt;.&lt;strong&gt; DataFusion’s optimizer can now represent column data distributions using a new &lt;code&gt;Distribution&lt;/code&gt; enum, instead of the old precision or range estimations. The supported distribution types currently include &lt;/strong&gt;Uniform, Gaussian (normal), Exponential, Bernoulli&lt;strong&gt;, and a &lt;/strong&gt;Generic** catch-all.&lt;/p&gt;
&lt;p&gt;For example, if a filter expression is applied to a column with a known uniform distribution range, the optimizer can propagate that to estimate result selectivity more accurately. Similarly, comparisons (&lt;code&gt;=&lt;/code&gt;, &lt;code&gt;&amp;gt;&lt;/code&gt;, etc.) on columns yield Bernoulli distributions (with true/false probabilities) in this model.&lt;/p&gt;
&lt;p&gt;This is a foundational change with many follow-on PRs underway. Even though the immediate user-visible effect is limited (the optimizer didn't magically improve by an order of magnitude overnight), but it lays groundwork for more advanced query planning in the future. Over time, as statistics information encapsulated in &lt;code&gt;Distribution&lt;/code&gt;s get integrated, DataFusion will be able to make smarter decisions like more aggressive parquet pruning, better join orderings, and so on based on data distribution information. The core framework is now in place and is being hooked up to column and table level statistics.&lt;/p&gt;
&lt;h3 id="aggregate-monotonicity-and-window-ordering"&gt;Aggregate Monotonicity and Window Ordering&lt;a class="headerlink" href="#aggregate-monotonicity-and-window-ordering" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;DataFusion 46.0.0 adds a new concept of &lt;a href="https://github.com/apache/datafusion/pull/14271"&gt;&lt;strong&gt;set-monotonicity&lt;/strong&gt;&lt;/a&gt; for certain transformations, which helps avoid unnecessary sort operations. In particular, the planner now understands when a &lt;strong&gt;window function introduces new orderings of data&lt;/strong&gt;.&lt;/p&gt;
&lt;p&gt;For example, DataFusion now recognizes that a window-aggregate like &lt;code&gt;MAX&lt;/code&gt; on a column can produce a result that is &lt;strong&gt;monotonically increasing&lt;/strong&gt;, even if the input column is unordered — depending on the window frame used.&lt;/p&gt;
&lt;p&gt;Consider the following query:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-sql"&gt;SELECT MAX(c1) OVER (
    ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
) AS max_c1
FROM c1_table
ORDER BY max_c1;
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;In earlier versions of DataFusion, this query would require an additional SortExec on max_c1 to satisfy the ORDER BY clause. However, with the new set-monotonicity logic, the planner knows that MAX(...) OVER (...) produces values that are not smaller than the previous row, making the extra sort redundant. This leads to more efficient query execution.&lt;/p&gt;
&lt;p&gt;PR &lt;a href="https://github.com/apache/datafusion/pull/14271"&gt;#14271&lt;/a&gt; introduced the core monotonicity tracking for aggregates and window functions.
PR &lt;a href="https://github.com/apache/datafusion/pull/14813"&gt;#14813&lt;/a&gt; improved ordering preservation within various window frame types, and brought an extensive test coverage.
Huge thanks to &lt;a href="https://github.com/berkaysynnada"&gt;@berkaysynnada&lt;/a&gt; and &lt;a href="https://github.com/mertak-synnada"&gt;@mertak-synnada&lt;/a&gt; for designing and implementing this optimizer enhancement!&lt;/p&gt;
&lt;h3 id="union-all-distinct-by-name-support"&gt;UNION [ALL | DISTINCT] BY NAME Support&lt;a class="headerlink" href="#union-all-distinct-by-name-support" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;DataFusion now supports UNION BY NAME and UNION ALL BY NAME, which align columns by name instead of position. This matches functionality found in systems like Spark and DuckDB and simplifies combining heterogeneously ordered result sets.&lt;/p&gt;
&lt;p&gt;You no longer need to rewrite column order manually — just write:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-sql"&gt;SELECT col1, col2 FROM t1
UNION ALL BY NAME
SELECT col2, col1 FROM t2;
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;Under the hood, this is supported by the new union_by_name() and union_by_name_distinct() plan builder methods.&lt;/p&gt;
&lt;p&gt;Thanks to &lt;a href="https://github.com/rkrishn7"&gt;@rkrishn7&lt;/a&gt; for PR &lt;a href="https://github.com/apache/datafusion/pull/14538"&gt;#14538&lt;/a&gt;.&lt;/p&gt;
&lt;h3 id="new-range-table-function"&gt;New range() Table Function&lt;a class="headerlink" href="#new-range-table-function" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;A new table-valued function range(start, stop, step) has been added to make it easy to generate integer sequences — similar to PostgreSQL’s generate_series() or Spark’s range().&lt;/p&gt;
&lt;p&gt;Example:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-sql"&gt;SELECT * FROM range(1, 10, 2);
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;This returns: 1, 3, 5, 7, 9. It’s great for testing, cross joins, surrogate keys, and more.&lt;/p&gt;
&lt;p&gt;Thanks to &lt;a href="https://github.com/simonvandel"&gt;@simonvandel&lt;/a&gt; for PR &lt;a href="https://github.com/apache/datafusion/pull/14830"&gt;#14830&lt;/a&gt;.&lt;/p&gt;
&lt;h2 id="upgrade-guide-and-changelog"&gt;Upgrade Guide and Changelog&lt;a class="headerlink" href="#upgrade-guide-and-changelog" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;Upgrading to 46.0.0 should be straightforward for most users, but do review the &lt;a href="https://datafusion.apache.org/library-user-guide/upgrading.html"&gt;Upgrade Guide for DataFusion 46.0.0&lt;/a&gt; for detailed steps and code changes. The upgrade guide covers the breaking changes mentioned (like replacing old exec nodes with &lt;code&gt;DataSourceExec&lt;/code&gt;, updating UDF invocation to &lt;code&gt;invoke_with_args&lt;/code&gt;, etc.) and provides code snippets to help with the transition. For a comprehensive list of all changes, please refer to the &lt;strong&gt;changelog&lt;/strong&gt; for 46.0.0 (linked above and in the repository). The changelog enumerates every merged PR in this release, including many smaller fixes and improvements that we couldn’t cover in this post.&lt;/p&gt;
&lt;h2 id="get-involved"&gt;Get Involved&lt;a class="headerlink" href="#get-involved" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;Apache DataFusion is an open-source project, and we welcome involvement from anyone interested. Now is a great time to take 46.0.0 for a spin: try it out on your workloads, and let us know if you encounter any issues or have suggestions. You can report bugs or request features on our GitHub issue tracker, or better yet, submit a pull request. Join our community discussions – whether you have questions, want to share how you’re using DataFusion, or are looking to contribute, we’d love to hear from you. A list of open issues suitable for beginners is &lt;a href="https://github.com/apache/arrow-datafusion/issues?q=is%3Aissue+is%3Aopen+label%3A%22good+first+issue%22"&gt;here&lt;/a&gt; and you can find how to reach us on the &lt;a href="https://datafusion.apache.org/contributor-guide/communication.html"&gt;communication doc&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;Happy querying!&lt;/p&gt;</content><category term="blog"/></entry><entry><title>Efficient Filter Pushdown in Parquet</title><link href="https://datafusion.apache.org/blog/2025/03/21/parquet-pushdown" rel="alternate"/><published>2025-03-21T00:00:00+00:00</published><updated>2025-03-21T00:00:00+00:00</updated><author><name>Xiangpeng Hao</name></author><id>tag:datafusion.apache.org,2025-03-21:/blog/2025/03/21/parquet-pushdown</id><summary type="html">&lt;style&gt;
figure {
  margin: 20px 0;
}

figure img {
  display: block;
  max-width: 80%;
}

figcaption {
  font-style: italic;
  margin-top: 10px;
  color: #555;
  font-size: 0.9em;
  max-width: 80%;
}
&lt;/style&gt;
&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;p&gt;&lt;em&gt;Editor's Note: This blog was first published on &lt;a href="https://blog.xiangpeng.systems/posts/parquet-pushdown/"&gt;Xiangpeng Hao's blog&lt;/a&gt;. Thanks to &lt;a href="https://www.influxdata.com/"&gt;InfluxData&lt;/a&gt; for sponsoring this work as part of his PhD funding.&lt;/em&gt;&lt;/p&gt;
&lt;hr/&gt;
&lt;p&gt;In the &lt;a href="https://datafusion.apache.org/blog/2025/03/20/parquet-pruning"&gt;previous post …&lt;/a&gt;&lt;/p&gt;</summary><content type="html">&lt;style&gt;
figure {
  margin: 20px 0;
}

figure img {
  display: block;
  max-width: 80%;
}

figcaption {
  font-style: italic;
  margin-top: 10px;
  color: #555;
  font-size: 0.9em;
  max-width: 80%;
}
&lt;/style&gt;
&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;p&gt;&lt;em&gt;Editor's Note: This blog was first published on &lt;a href="https://blog.xiangpeng.systems/posts/parquet-pushdown/"&gt;Xiangpeng Hao's blog&lt;/a&gt;. Thanks to &lt;a href="https://www.influxdata.com/"&gt;InfluxData&lt;/a&gt; for sponsoring this work as part of his PhD funding.&lt;/em&gt;&lt;/p&gt;
&lt;hr/&gt;
&lt;p&gt;In the &lt;a href="https://datafusion.apache.org/blog/2025/03/20/parquet-pruning"&gt;previous post&lt;/a&gt;, we discussed how &lt;a href="https://datafusion.apache.org/"&gt;Apache DataFusion&lt;/a&gt; prunes &lt;a href="https://parquet.apache.org/"&gt;Apache Parquet&lt;/a&gt; files to skip irrelevant &lt;strong&gt;files/row_groups&lt;/strong&gt; (sometimes also &lt;a href="https://parquet.apache.org/docs/file-format/pageindex/"&gt;pages&lt;/a&gt;).&lt;/p&gt;
&lt;p&gt;This post discusses how Parquet readers skip irrelevant &lt;strong&gt;rows&lt;/strong&gt; while scanning data,
leveraging Parquet's columnar layout by first reading only filter columns,
and then selectively reading other columns only for matching rows.&lt;/p&gt;
&lt;h2 id="why-filter-pushdown-in-parquet"&gt;Why filter pushdown in Parquet?&lt;a class="headerlink" href="#why-filter-pushdown-in-parquet" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;Below is an example query that reads sensor data with filters on &lt;code&gt;date_time&lt;/code&gt; and &lt;code&gt;location&lt;/code&gt;.
Without filter pushdown, all rows from &lt;code&gt;location&lt;/code&gt;, &lt;code&gt;val&lt;/code&gt;, and &lt;code&gt;date_time&lt;/code&gt; columns are decoded before &lt;code&gt;location='office'&lt;/code&gt; is evaluated. Filter pushdown is especially useful when the filter is selective, i.e., removes many rows.&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-sql"&gt;SELECT val, location 
FROM sensor_data 
WHERE date_time &amp;gt; '2025-03-11' AND location = 'office';
&lt;/code&gt;&lt;/pre&gt;
&lt;figure&gt;
&lt;img alt="Parquet pruning skips irrelevant files/row_groups, while filter pushdown skips irrelevant rows. Without filter pushdown, all rows from location, val, and date_time columns are decoded before `location='office'` is evaluated. Filter pushdown is especially useful when the filter is selective, i.e., removes many rows." class="img-fluid" src="/blog/images/parquet-pushdown/pushdown-vs-no-pushdown.jpg" width="80%"/&gt;
&lt;figcaption&gt;
    Parquet pruning skips irrelevant files/row_groups, while filter pushdown skips irrelevant rows. Without filter pushdown, all rows from location, val, and date_time columns are decoded before `location='office'` is evaluated. Filter pushdown is especially useful when the filter is selective, i.e., removes many rows.
  &lt;/figcaption&gt;
&lt;/figure&gt;
&lt;p&gt;In our setup, sensor data is aggregated by date — each day has its own Parquet file.
At planning time, DataFusion prunes the unneeded Parquet files, i.e., &lt;code&gt;2025-03-10.parquet&lt;/code&gt; and &lt;code&gt;2025-03-11.parquet&lt;/code&gt;.&lt;/p&gt;
&lt;p&gt;Once the files to read are located, the &lt;a href="https://github.com/apache/datafusion/issues/3463"&gt;&lt;em&gt;DataFusion's current default implementation&lt;/em&gt;&lt;/a&gt; reads all the projected columns (&lt;code&gt;sensor_id&lt;/code&gt;, &lt;code&gt;val&lt;/code&gt;, and &lt;code&gt;location&lt;/code&gt;) into Arrow RecordBatches, then applies the filters over &lt;code&gt;location&lt;/code&gt; to get the final set of rows.&lt;/p&gt;
&lt;p&gt;A better approach is called &lt;strong&gt;filter pushdown&lt;/strong&gt; with &lt;strong&gt;late materialization&lt;/strong&gt;, which evaluates filter conditions first and only decodes data that passes these conditions.
In practice, this works by first processing only the filter columns (&lt;code&gt;date_time&lt;/code&gt; and &lt;code&gt;location&lt;/code&gt;), building a boolean mask of rows that satisfy our conditions, then using this mask to selectively decode only the relevant rows from other columns (&lt;code&gt;sensor_id&lt;/code&gt;, &lt;code&gt;val&lt;/code&gt;). 
This eliminates the waste of decoding rows that will be immediately filtered out.&lt;/p&gt;
&lt;p&gt;While simple in theory, practical implementations often make performance worse.&lt;/p&gt;
&lt;h2 id="how-can-filter-pushdown-be-slower"&gt;How can filter pushdown be slower?&lt;a class="headerlink" href="#how-can-filter-pushdown-be-slower" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;At a high level, the Parquet reader first builds a filter mask -- essentially a boolean array indicating which rows meet the filter criteria -- and then uses this mask to selectively decode only the needed rows from the remaining columns in the projection.&lt;/p&gt;
&lt;p&gt;Let's dig into details of &lt;a href="https://github.com/apache/arrow-rs/blob/d5339f31a60a4bd8a4256e7120fe32603249d88e/parquet/src/arrow/async_reader/mod.rs#L618-L712"&gt;how filter pushdown is implemented&lt;/a&gt; in the current Rust Parquet reader implementation, illustrated in the following figure.&lt;/p&gt;
&lt;figure&gt;
&lt;img alt="Implementation of filter pushdown in Rust Parquet readers" class="img-fluid" src="/blog/images/parquet-pushdown/baseline-impl.jpg" with="70%"/&gt;
&lt;figcaption&gt;
    Implementation of filter pushdown in Rust Parquet readers -- the first phase builds the filter mask, the second phase applies the filter mask to the other columns
  &lt;/figcaption&gt;
&lt;/figure&gt;
&lt;p&gt;The filter pushdown has two phases:&lt;/p&gt;
&lt;ol&gt;
&lt;li&gt;
&lt;p&gt;Build the filter mask (steps 1-3)&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;
&lt;p&gt;Use the filter mask to selectively decode other columns (steps 4-7), e.g., output step 3 is used as input for step 5 and 7.&lt;/p&gt;
&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;Within each phase, it takes three steps from Parquet to Arrow:&lt;/p&gt;
&lt;ol&gt;
&lt;li&gt;
&lt;p&gt;Decompress the Parquet pages using generic decompression algorithms like LZ4, Zstd, etc. (steps 1, 4, 6)&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;
&lt;p&gt;Decode the page content into Arrow format (steps 2, 5, 7)&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;
&lt;p&gt;Evaluate the filter over Arrow data (step 3)&lt;/p&gt;
&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;In the figure above, we can see that &lt;code&gt;location&lt;/code&gt; is &lt;strong&gt;decompressed and decoded twice&lt;/strong&gt;, first when building the filter mask (steps 1, 2), and second when building the output (steps 4, 5).
This happens for all columns that appear both in the filter and output.&lt;/p&gt;
&lt;p&gt;The table below shows the corresponding CPU time on the &lt;a href="https://github.com/apache/datafusion/blob/main/benchmarks/queries/clickbench/queries.sql#L23"&gt;ClickBench query 22&lt;/a&gt;:&lt;/p&gt;
&lt;pre&gt;&lt;code&gt;+------------+--------+-------------+--------+
| Decompress | Decode | Apply filter| Others |
+------------+--------+-------------+--------+
| 206 ms     | 117 ms | 22 ms       | 48 ms  |
+------------+--------+-------------+--------+
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;Clearly, decompress/decode operations dominate the time spent. With filter pushdown, it needs to decompress/decode twice; but without filter pushdown, it only needs to do this once.
This explains why filter pushdown is slower in some cases.&lt;/p&gt;
&lt;blockquote&gt;
&lt;p&gt;&lt;strong&gt;Note:&lt;/strong&gt; Highly selective filters may skip the entire page; but as long as it reads one row from the page, it needs to decompress and often decode the entire page.&lt;/p&gt;
&lt;/blockquote&gt;
&lt;h2 id="attempt-cache-filter-columns"&gt;Attempt: cache filter columns&lt;a class="headerlink" href="#attempt-cache-filter-columns" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;Intuitively, caching the filter columns and reusing them later could help.&lt;/p&gt;
&lt;p&gt;But naively caching decoded pages consumes prohibitively high memory:&lt;/p&gt;
&lt;ol&gt;
&lt;li&gt;
&lt;p&gt;It needs to cache Arrow arrays, which are on average &lt;a href="https://github.com/XiangpengHao/liquid-cache/blob/main/dev/doc/liquid-cache-vldb.pdf"&gt;4x larger than Parquet data&lt;/a&gt;.&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;
&lt;p&gt;It needs to cache the &lt;strong&gt;entire column chunk in memory&lt;/strong&gt;, because in Phase 1 it builds filters over the column chunk, and only use it in Phase 2.  &lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;
&lt;p&gt;The memory usage is proportional to the number of filter columns, which can be unboundedly high. &lt;/p&gt;
&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;Worse, caching filter columns means it needs to read partially from Parquet and partially from cache, which is complex to implement, likely requiring a substantial change to the current implementation. &lt;/p&gt;
&lt;blockquote&gt;
&lt;p&gt;&lt;strong&gt;Feel the complexity:&lt;/strong&gt; consider building a cache that properly handles nested columns, multiple filters, and filters with multiple columns.&lt;/p&gt;
&lt;/blockquote&gt;
&lt;h2 id="real-solution"&gt;Real solution&lt;a class="headerlink" href="#real-solution" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;We need a solution that:&lt;/p&gt;
&lt;ol&gt;
&lt;li&gt;
&lt;p&gt;Is simple to implement, i.e., doesn't require thousands of lines of code.&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;
&lt;p&gt;Incurs minimal memory overhead.&lt;/p&gt;
&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;This section describes my &lt;a href="https://github.com/apache/arrow-rs/pull/6921#issuecomment-2718792433"&gt;&amp;lt;700 LOC PR (with lots of comments and tests)&lt;/a&gt; that &lt;strong&gt;reduces total ClickBench time by 15%, with up to 2x lower latency for some queries, no obvious regression on other queries, and caches at most 2 pages (~2MB) per column in memory&lt;/strong&gt;.&lt;/p&gt;
&lt;figure&gt;
&lt;img alt="New decoding pipeline, building filter mask and output columns are interleaved in a single pass, allowing us to cache minimal pages for minimal amount of time" class="img-fluid" src="/blog/images/parquet-pushdown/new-pipeline.jpg" width="80%"/&gt;
&lt;figcaption&gt;
    New decoding pipeline, building filter mask and output columns are interleaved in a single pass, allowing us to cache minimal pages for minimal amount of time
  &lt;/figcaption&gt;
&lt;/figure&gt;
&lt;p&gt;The new pipeline interleaves the previous two phases into a single pass, so that:&lt;/p&gt;
&lt;ol&gt;
&lt;li&gt;
&lt;p&gt;The page being decompressed is immediately used to build filter masks and output columns.&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;
&lt;p&gt;Decompressed pages are cached for minimal time; after one pass (steps 1-6), the cache memory is released for the next pass. &lt;/p&gt;
&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;This allows the cache to only hold 1 page at a time, and to immediately discard the previous page after it's used, significantly reducing the memory requirement for caching.&lt;/p&gt;
&lt;h3 id="what-pages-are-cached"&gt;What pages are cached?&lt;a class="headerlink" href="#what-pages-are-cached" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;You may have noticed that only &lt;code&gt;location&lt;/code&gt; is cached, not &lt;code&gt;val&lt;/code&gt;, because &lt;code&gt;val&lt;/code&gt; is only used for output.
More generally, only columns that appear both in the filter and output are cached, and at most 1 page is cached for each such column.&lt;/p&gt;
&lt;p&gt;More examples:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-sql"&gt;SELECT val 
FROM sensor_data 
WHERE date_time &amp;gt; '2025-03-11' AND location = 'office';
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;In this case, no columns are cached, because &lt;code&gt;val&lt;/code&gt; is not used for filtering.&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-sql"&gt;SELECT COUNT(*) 
FROM sensor_data 
WHERE date_time &amp;gt; '2025-03-11' AND location = 'office';
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;In this case, again, no columns are cached, because the output projection is empty after query plan optimization.&lt;/p&gt;
&lt;h3 id="then-why-cache-2-pages-per-column-instead-of-1"&gt;Then why cache 2 pages per column instead of 1?&lt;a class="headerlink" href="#then-why-cache-2-pages-per-column-instead-of-1" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;This is another real-world nuance regarding how Parquet layouts the pages.&lt;/p&gt;
&lt;p&gt;Parquet by default encodes data using &lt;a href="https://parquet.apache.org/docs/file-format/data-pages/encodings/"&gt;dictionary encoding&lt;/a&gt;, which writes a dictionary page as the first page of a column chunk, followed by the keys referencing the dictionary.&lt;/p&gt;
&lt;p&gt;You can see this in action using &lt;a href="https://parquet-viewer.xiangpeng.systems"&gt;parquet-viewer&lt;/a&gt;:&lt;/p&gt;
&lt;figure&gt;
&lt;img alt="Parquet viewer shows the page layout of a column chunk" class="img-fluid" src="/blog/images/parquet-pushdown/parquet-viewer.jpg" width="80%"/&gt;
&lt;figcaption&gt;
    Parquet viewer shows the page layout of a column chunk
  &lt;/figcaption&gt;
&lt;/figure&gt;
&lt;p&gt;This means that to decode a page of data, it actually references two pages: the dictionary page and the data page.&lt;/p&gt;
&lt;p&gt;This is why it caches 2 pages per column: one dictionary page and one data page.
The data page slot will move forward as it reads the data; but the dictionary page slot always references the first page.&lt;/p&gt;
&lt;figure&gt;
&lt;img alt="Cached two pages, one for dictionary (pinned), one for data (moves as it reads the data)" class="img-fluid" src="/blog/images/parquet-pushdown/cached-pages.jpg" width="80%"/&gt;
&lt;figcaption&gt;
    Cached two pages, one for dictionary (pinned), one for data (moves as it reads the data)
  &lt;/figcaption&gt;
&lt;/figure&gt;
&lt;h2 id="how-does-it-perform"&gt;How does it perform?&lt;a class="headerlink" href="#how-does-it-perform" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;Here are my results on &lt;a href="https://github.com/apache/datafusion/tree/main/benchmarks#clickbench"&gt;ClickBench&lt;/a&gt; on my AMD 9900X machine. The total time is reduced by 15%, with Q23 being 2.24x faster,
and queries that get slower are likely due to noise.&lt;/p&gt;
&lt;pre&gt;&lt;code&gt;┏━━━━━━━━━━━━━━┳━━━━━━━━━━━━━┳━━━━━━━━━━━━━━┳━━━━━━━━━━━━━━━┓
┃ Query        ┃ no-pushdown ┃ new-pushdown ┃        Change ┃
┡━━━━━━━━━━━━━━╇━━━━━━━━━━━━━╇━━━━━━━━━━━━━━╇━━━━━━━━━━━━━━━┩
│ QQuery 0     │      0.47ms │       0.43ms │ +1.10x faster │
│ QQuery 1     │     51.10ms │      50.10ms │     no change │
│ QQuery 2     │     68.23ms │      64.49ms │ +1.06x faster │
│ QQuery 3     │     90.68ms │      86.73ms │     no change │
│ QQuery 4     │    458.93ms │     458.59ms │     no change │
│ QQuery 5     │    522.06ms │     478.50ms │ +1.09x faster │
│ QQuery 6     │     49.84ms │      49.94ms │     no change │
│ QQuery 7     │     55.09ms │      55.77ms │     no change │
│ QQuery 8     │    565.26ms │     556.95ms │     no change │
│ QQuery 9     │    575.83ms │     575.05ms │     no change │
│ QQuery 10    │    164.56ms │     178.23ms │  1.08x slower │
│ QQuery 11    │    177.20ms │     191.32ms │  1.08x slower │
│ QQuery 12    │    591.05ms │     569.92ms │     no change │
│ QQuery 13    │    861.06ms │     848.59ms │     no change │
│ QQuery 14    │    596.20ms │     580.73ms │     no change │
│ QQuery 15    │    554.96ms │     548.77ms │     no change │
│ QQuery 16    │   1175.08ms │    1146.07ms │     no change │
│ QQuery 17    │   1150.45ms │    1121.49ms │     no change │
│ QQuery 18    │   2634.75ms │    2494.07ms │ +1.06x faster │
│ QQuery 19    │     90.15ms │      89.24ms │     no change │
│ QQuery 20    │    620.15ms │     591.67ms │     no change │
│ QQuery 21    │    782.38ms │     703.15ms │ +1.11x faster │
│ QQuery 22    │   1927.94ms │    1404.35ms │ +1.37x faster │
│ QQuery 23    │   8104.11ms │    3610.76ms │ +2.24x faster │
│ QQuery 24    │    360.79ms │     330.55ms │ +1.09x faster │
│ QQuery 25    │    290.61ms │     252.54ms │ +1.15x faster │
│ QQuery 26    │    395.18ms │     362.72ms │ +1.09x faster │
│ QQuery 27    │    891.76ms │     959.39ms │  1.08x slower │
│ QQuery 28    │   4059.54ms │    4137.37ms │     no change │
│ QQuery 29    │    235.88ms │     228.99ms │     no change │
│ QQuery 30    │    564.22ms │     584.65ms │     no change │
│ QQuery 31    │    741.20ms │     757.87ms │     no change │
│ QQuery 32    │   2652.48ms │    2574.19ms │     no change │
│ QQuery 33    │   2373.71ms │    2327.10ms │     no change │
│ QQuery 34    │   2391.00ms │    2342.15ms │     no change │
│ QQuery 35    │    700.79ms │     694.51ms │     no change │
│ QQuery 36    │    151.51ms │     152.93ms │     no change │
│ QQuery 37    │    108.18ms │      86.03ms │ +1.26x faster │
│ QQuery 38    │    114.64ms │     106.22ms │ +1.08x faster │
│ QQuery 39    │    260.80ms │     239.13ms │ +1.09x faster │
│ QQuery 40    │     60.74ms │      73.29ms │  1.21x slower │
│ QQuery 41    │     58.75ms │      67.85ms │  1.15x slower │
│ QQuery 42    │     65.49ms │      68.11ms │     no change │
└──────────────┴─────────────┴──────────────┴───────────────┘
┏━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┳━━━━━━━━━━━━┓
┃ Benchmark Summary           ┃            ┃
┡━━━━━━━━━━━━━━━━━━━━━━━━━━━━━╇━━━━━━━━━━━━┩
│ Total Time (no-pushdown)    │ 38344.79ms │
│ Total Time (new-pushdown)   │ 32800.50ms │
│ Average Time (no-pushdown)  │   891.74ms │
│ Average Time (new-pushdown) │   762.80ms │
│ Queries Faster              │         13 │
│ Queries Slower              │          5 │
│ Queries with No Change      │         25 │
└─────────────────────────────┴────────────┘
&lt;/code&gt;&lt;/pre&gt;
&lt;h2 id="conclusion"&gt;Conclusion&lt;a class="headerlink" href="#conclusion" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;Despite being simple in theory, filter pushdown in Parquet is non-trivial to implement.
It requires understanding both the Parquet format and reader implementation details. 
The challenge lies in efficiently navigating through the dynamics of decoding, filter evaluation, and memory management.&lt;/p&gt;
&lt;p&gt;If you are interested in this level of optimization and want to help test, document and implement this type of optimization, come find us in the &lt;a href="https://datafusion.apache.org/contributor-guide/communication.html"&gt;DataFusion Community&lt;/a&gt;. We would love to have you. &lt;/p&gt;</content><category term="blog"/></entry><entry><title>Apache DataFusion Comet 0.7.0 Release</title><link href="https://datafusion.apache.org/blog/2025/03/20/datafusion-comet-0.7.0" rel="alternate"/><published>2025-03-20T00:00:00+00:00</published><updated>2025-03-20T00:00:00+00:00</updated><author><name>pmc</name></author><id>tag:datafusion.apache.org,2025-03-20:/blog/2025/03/20/datafusion-comet-0.7.0</id><summary type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;p&gt;The Apache DataFusion PMC is pleased to announce version 0.7.0 of the &lt;a href="https://datafusion.apache.org/comet/"&gt;Comet&lt;/a&gt; subproject.&lt;/p&gt;
&lt;p&gt;Comet is an accelerator for Apache Spark that translates Spark physical plans to DataFusion physical plans for
improved performance and efficiency without requiring any code changes.&lt;/p&gt;
&lt;p&gt;Comet runs on commodity hardware and aims to …&lt;/p&gt;</summary><content type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;p&gt;The Apache DataFusion PMC is pleased to announce version 0.7.0 of the &lt;a href="https://datafusion.apache.org/comet/"&gt;Comet&lt;/a&gt; subproject.&lt;/p&gt;
&lt;p&gt;Comet is an accelerator for Apache Spark that translates Spark physical plans to DataFusion physical plans for
improved performance and efficiency without requiring any code changes.&lt;/p&gt;
&lt;p&gt;Comet runs on commodity hardware and aims to provide 100% compatibility with Apache Spark. Any operators or
expressions that are not fully compatible will fall back to Spark unless explicitly enabled by the user. Refer
to the &lt;a href="https://datafusion.apache.org/comet/user-guide/compatibility.html"&gt;compatibility guide&lt;/a&gt; for more information.&lt;/p&gt;
&lt;p&gt;This release covers approximately four weeks of development work and is the result of merging 46 PRs from 11
contributors. See the &lt;a href="https://github.com/apache/datafusion-comet/blob/main/dev/changelog/0.7.0.md"&gt;change log&lt;/a&gt; for more information.&lt;/p&gt;
&lt;h2 id="release-highlights"&gt;Release Highlights&lt;a class="headerlink" href="#release-highlights" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;h3 id="performance"&gt;Performance&lt;a class="headerlink" href="#performance" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;Comet 0.7.0 has improved performance compared to the previous release due to improvements in the native shuffle 
implementation and performance improvements in DataFusion 46.&lt;/p&gt;
&lt;p&gt;For single-node TPC-H at 100 GB, Comet now delivers a &lt;strong&gt;greater than 2x speedup&lt;/strong&gt; compared to Spark using the same 
CPU and RAM. Even with &lt;strong&gt;half the resources&lt;/strong&gt;, Comet still provides a measurable performance improvement.&lt;/p&gt;
&lt;p&gt;&lt;img alt="Chart showing TPC-H benchmark results for Comet 0.7.0" class="img-fluid" src="/blog/images/comet-0.7.0/performance.png" width="100%"/&gt;&lt;/p&gt;
&lt;p&gt;&lt;em&gt;These benchmarks were performed on a Linux workstation with PCIe 5, AMD 7950X CPU (16 cores), 128 GB RAM, and data 
stored locally in Parquet format on NVMe storage. Spark was running in Kubernetes with hard memory limits.&lt;/em&gt;&lt;/p&gt;
&lt;h2 id="shuffle-improvements"&gt;Shuffle Improvements&lt;a class="headerlink" href="#shuffle-improvements" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;There are several improvements to shuffle in this release:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;When running in off-heap mode (which is the recommended approach), Comet was using the wrong memory allocator 
  implementation for some types of shuffle operation, which could result in OOM rather than spilling to disk.&lt;/li&gt;
&lt;li&gt;The number of spill files is drastically reduced. In previous releases, each instance of ShuffleMapTask could 
  potentially create a new spill file for each output partition each time that spill was invoked. Comet now creates 
  a maximum of one spill file per output partition per instance of ShuffleMapTask, which is appended to in subsequent 
  spills.&lt;/li&gt;
&lt;li&gt;There was a flaw in the memory accounting which resulted in Comet requesting approximately twice the amount of 
  memory that was needed, resulting in premature spilling. This is now resolved.&lt;/li&gt;
&lt;li&gt;The metric for number of spilled bytes is now accurate. It was previously reporting invalid information.&lt;/li&gt;
&lt;/ul&gt;
&lt;h2 id="improved-hash-join-performance"&gt;Improved Hash Join Performance&lt;a class="headerlink" href="#improved-hash-join-performance" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;When using the &lt;code&gt;spark.comet.exec.replaceSortMergeJoin&lt;/code&gt; setting to replace sort-merge joins with hash joins, Comet 
will now do a better job of picking the optimal build side. Thanks to &lt;a href="https://github.com/hayman42"&gt;@hayman42&lt;/a&gt; for suggesting this, and thanks to the 
&lt;a href="https://github.com/apache/incubator-gluten/"&gt;Apache Gluten(incubating)&lt;/a&gt; project for the inspiration in implementing this feature.&lt;/p&gt;
&lt;h2 id="experimental-support-for-datafusions-parquet-scan"&gt;Experimental Support for DataFusion’s Parquet Scan&lt;a class="headerlink" href="#experimental-support-for-datafusions-parquet-scan" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;It is now possible to configure Comet to use DataFusion’s Parquet reader instead of Comet’s current Parquet reader. This 
has the advantage of supporting complex types, and also has performance optimizations that are not present in Comet's 
existing reader.&lt;/p&gt;
&lt;p&gt;Support should still be considered experimental, but most of Comet’s unit tests are now passing with the new reader. 
Known issues include handling of &lt;code&gt;INT96&lt;/code&gt; timestamps and unsigned bytes and shorts.&lt;/p&gt;
&lt;p&gt;To enable DataFusion’s Parquet reader, either set &lt;code&gt;spark.comet.scan.impl=native_datafusion&lt;/code&gt; or set the environment 
variable &lt;code&gt;COMET_PARQUET_SCAN_IMPL=native_datafusion&lt;/code&gt;.&lt;/p&gt;
&lt;h2 id="complex-type-support"&gt;Complex Type Support&lt;a class="headerlink" href="#complex-type-support" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;With DataFusion’s Parquet reader enabled, there is now some early support for reading structs from Parquet. This is 
not thoroughly tested yet. We would welcome additional testing from the community to help determine what is and isn’t 
working, as well as contributions to improve support for structs and other complex types. The tracking issue is 
&lt;a href="https://github.com/apache/datafusion-comet/issues/1043"&gt;https://github.com/apache/datafusion-comet/issues/1043&lt;/a&gt;.&lt;/p&gt;
&lt;h2 id="updates-to-supported-spark-versions"&gt;Updates to supported Spark versions&lt;a class="headerlink" href="#updates-to-supported-spark-versions" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;ul&gt;
&lt;li&gt;Comet 0.7.0 is now tested against Spark 3.5.4 rather than 3.5.1&lt;/li&gt;
&lt;li&gt;This will be the last Comet release to support Spark 3.3.x&lt;/li&gt;
&lt;/ul&gt;
&lt;h2 id="improved-tuning-guide"&gt;Improved Tuning Guide&lt;a class="headerlink" href="#improved-tuning-guide" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;The &lt;a href="https://datafusion.apache.org/comet/user-guide/tuning.html"&gt;Comet Tuning Guide&lt;/a&gt; has been improved and now provides guidance on determining how much memory to allocate to 
Comet.&lt;/p&gt;
&lt;h2 id="getting-involved"&gt;Getting Involved&lt;a class="headerlink" href="#getting-involved" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;The Comet project welcomes new contributors. We use the same &lt;a href="https://datafusion.apache.org/contributor-guide/communication.html#slack-and-discord"&gt;Slack and Discord&lt;/a&gt; channels as the main DataFusion
project and have a weekly &lt;a href="https://docs.google.com/document/d/1NBpkIAuU7O9h8Br5CbFksDhX-L9TyO9wmGLPMe0Plc8/edit?usp=sharing"&gt;DataFusion video call&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;The easiest way to get involved is to test Comet with your current Spark jobs and file issues for any bugs or
performance regressions that you find. See the &lt;a href="https://datafusion.apache.org/comet/user-guide/installation.html"&gt;Getting Started&lt;/a&gt; guide for instructions on downloading and installing
Comet.&lt;/p&gt;
&lt;p&gt;There are also many &lt;a href="https://github.com/apache/datafusion-comet/contribute"&gt;good first issues&lt;/a&gt; waiting for contributions.&lt;/p&gt;</content><category term="blog"/></entry><entry><title>Parquet Pruning in DataFusion: Read Only What Matters</title><link href="https://datafusion.apache.org/blog/2025/03/20/parquet-pruning" rel="alternate"/><published>2025-03-20T00:00:00+00:00</published><updated>2025-03-20T00:00:00+00:00</updated><author><name>Xiangpeng Hao</name></author><id>tag:datafusion.apache.org,2025-03-20:/blog/2025/03/20/parquet-pruning</id><summary type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;p&gt;&lt;em&gt;Editor's Note: This blog was first published on &lt;a href="https://blog.xiangpeng.systems/posts/parquet-to-arrow/"&gt;Xiangpeng Hao's blog&lt;/a&gt;. Thanks to &lt;a href="https://www.influxdata.com/"&gt;InfluxData&lt;/a&gt; for sponsoring this work as part of his PhD funding.&lt;/em&gt;&lt;/p&gt;
&lt;hr/&gt;
&lt;p&gt;&lt;a href="https://parquet.apache.org/"&gt;Apache Parquet&lt;/a&gt; has become the industry standard for storing columnar data, and reading Parquet efficiently -- especially from remote storage -- is crucial for query performance.&lt;/p&gt;
&lt;p&gt;&lt;a href="https://datafusion.apache.org/"&gt;Apache DataFusion …&lt;/a&gt;&lt;/p&gt;</summary><content type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;p&gt;&lt;em&gt;Editor's Note: This blog was first published on &lt;a href="https://blog.xiangpeng.systems/posts/parquet-to-arrow/"&gt;Xiangpeng Hao's blog&lt;/a&gt;. Thanks to &lt;a href="https://www.influxdata.com/"&gt;InfluxData&lt;/a&gt; for sponsoring this work as part of his PhD funding.&lt;/em&gt;&lt;/p&gt;
&lt;hr/&gt;
&lt;p&gt;&lt;a href="https://parquet.apache.org/"&gt;Apache Parquet&lt;/a&gt; has become the industry standard for storing columnar data, and reading Parquet efficiently -- especially from remote storage -- is crucial for query performance.&lt;/p&gt;
&lt;p&gt;&lt;a href="https://datafusion.apache.org/"&gt;Apache DataFusion&lt;/a&gt; implements advanced Parquet pruning techniques to effectively read only the data that matters for a given query.&lt;/p&gt;
&lt;p&gt;Achieving high performance adds complexity.
This post provides an overview of the techniques used in DataFusion to selectively read Parquet files.&lt;/p&gt;
&lt;h3 id="the-pipeline"&gt;The pipeline&lt;a class="headerlink" href="#the-pipeline" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;The diagram below illustrates the &lt;a href="https://docs.rs/datafusion/46.0.0/datafusion/datasource/physical_plan/parquet/source/struct.ParquetSource.html"&gt;Parquet reading pipeline&lt;/a&gt; in DataFusion, highlighting how data flows through various pruning stages before being converted to Arrow format:&lt;/p&gt;
&lt;p&gt;&lt;img alt="Parquet pruning pipeline in DataFusion" class="img-fluid" src="/blog/images/parquet-pruning/read-parquet.jpg" width="100%"/&gt;&lt;/p&gt;
&lt;h4 id="background-parquet-file-structure"&gt;Background: Parquet file structure&lt;a class="headerlink" href="#background-parquet-file-structure" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h4&gt;
&lt;p&gt;As shown in the figure above, each Parquet file has multiple row groups. Each row group contains a set of columns, and each column contains a set of pages.&lt;/p&gt;
&lt;p&gt;Pages are the smallest units of data in Parquet files and typically contain compressed and encoded values for a specific column. This hierarchical structure enables efficient columnar access and forms the foundation for the pruning techniques we'll discuss.&lt;/p&gt;
&lt;p&gt;Check out &lt;a href="https://www.influxdata.com/blog/querying-parquet-millisecond-latency/"&gt;Querying Parquet with Millisecond Latency&lt;/a&gt; for more details on the Parquet file structure.&lt;/p&gt;
&lt;h4 id="1-read-metadata"&gt;1. Read metadata&lt;a class="headerlink" href="#1-read-metadata" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h4&gt;
&lt;p&gt;DataFusion first reads the &lt;a href="https://parquet.apache.org/docs/file-format/metadata/"&gt;Parquet metadata&lt;/a&gt; to understand the data in the file. 
Metadata often includes data schema, the exact location of each row group and column chunk, and their corresponding statistics (e.g., min/max values).
It also optionally includes &lt;a href="https://parquet.apache.org/docs/file-format/pageindex/"&gt;page-level stats&lt;/a&gt; and &lt;a href="https://www.influxdata.com/blog/using-parquets-bloom-filters/"&gt;Bloom filters&lt;/a&gt;.
This information is used to prune the file before reading the actual data.&lt;/p&gt;
&lt;p&gt;&lt;a href="https://github.com/apache/datafusion/blob/31701b8dc9c6486856c06a29a32107d9f4549cec/datafusion/core/src/datasource/physical_plan/parquet/reader.rs#L118"&gt;Fetching metadata&lt;/a&gt; requires up to two network requests: one to read the footer size from the end of the file, and another to read the footer itself. &lt;/p&gt;
&lt;p&gt;&lt;a href="https://www.influxdata.com/blog/how-good-parquet-wide-tables/"&gt;Decoding metadata&lt;/a&gt; is generally fast since it only requires parsing a small amount of data. However, for tables with hundreds or thousands of columns, the metadata can become quite large and decoding it can become a bottleneck. This is particularly noticeable when scanning many small files.&lt;/p&gt;
&lt;p&gt;Reading metadata is latency-critical, so DataFusion allows users to cache metadata through the &lt;a href="https://github.com/apache/datafusion/blob/31701b8dc9c6486856c06a29a32107d9f4549cec/datafusion/core/src/datasource/physical_plan/parquet/reader.rs#L39"&gt;ParquetFileReaderFactory&lt;/a&gt; trait.&lt;/p&gt;
&lt;h4 id="2-prune-by-projection"&gt;2. Prune by projection&lt;a class="headerlink" href="#2-prune-by-projection" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h4&gt;
&lt;p&gt;The simplest yet perhaps most effective pruning is to read only the columns that are needed.
This is because queries usually don't select all columns, e.g., &lt;code&gt;SELECT a FROM table&lt;/code&gt; only reads column &lt;code&gt;a&lt;/code&gt;.
As a &lt;strong&gt;columnar&lt;/strong&gt; format, Parquet allows DataFusion to &lt;a href="https://github.com/apache/datafusion/blob/31701b8dc9c6486856c06a29a32107d9f4549cec/datafusion/core/src/datasource/physical_plan/parquet/mod.rs#L778"&gt;only read&lt;/a&gt; the &lt;strong&gt;columns&lt;/strong&gt; that are needed.&lt;/p&gt;
&lt;p&gt;This projection pruning happens at the column level and can dramatically reduce I/O when working with wide tables where queries typically access only a small subset of columns.&lt;/p&gt;
&lt;h4 id="3-prune-by-row-group-stats-and-bloom-filters"&gt;3. Prune by row group stats and Bloom filters&lt;a class="headerlink" href="#3-prune-by-row-group-stats-and-bloom-filters" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h4&gt;
&lt;p&gt;Each row group has &lt;a href="https://github.com/apache/datafusion/blob/31701b8dc9c6486856c06a29a32107d9f4549cec/datafusion/core/src/physical_optimizer/pruning.rs#L81"&gt;basic stats&lt;/a&gt; like min/max values for each column.
DataFusion applies the query predicates to these stats to prune row groups, e.g., &lt;code&gt;SELECT * FROM table WHERE a &amp;gt; 10&lt;/code&gt; will only read row groups where &lt;code&gt;a&lt;/code&gt; has a max value greater than 10.&lt;/p&gt;
&lt;p&gt;Sometimes min/max stats are too simple to prune effectively, so Parquet also supports &lt;a href="https://www.influxdata.com/blog/using-parquets-bloom-filters/"&gt;Bloom filters&lt;/a&gt;. DataFusion &lt;a href="https://github.com/apache/datafusion/blob/31701b8dc9c6486856c06a29a32107d9f4549cec/datafusion/core/src/datasource/physical_plan/parquet/opener.rs#L202"&gt;uses Bloom filters when available&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;Bloom filters are particularly effective for equality predicates (&lt;code&gt;WHERE a = 10&lt;/code&gt;) and can significantly reduce the number of row groups that need to be read for point queries or queries with highly selective predicates.&lt;/p&gt;
&lt;h4 id="4-prune-by-page-stats"&gt;4. Prune by page stats&lt;a class="headerlink" href="#4-prune-by-page-stats" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h4&gt;
&lt;p&gt;Parquet optionally supports &lt;a href="https://github.com/apache/parquet-format/blob/master/PageIndex.md"&gt;page-level stats&lt;/a&gt; -- similar to row group stats but more fine-grained.
DataFusion implements &lt;a href="https://github.com/apache/datafusion/blob/31701b8dc9c6486856c06a29a32107d9f4549cec/datafusion/core/src/datasource/physical_plan/parquet/opener.rs#L219"&gt;page pruning&lt;/a&gt; when the stats are present.&lt;/p&gt;
&lt;p&gt;Page-level pruning provides an additional layer of filtering after row group pruning. It allows DataFusion to skip individual pages within a row group, further reducing the amount of data that needs to be read and decoded.&lt;/p&gt;
&lt;h4 id="5-read-from-storage"&gt;5. Read from storage&lt;a class="headerlink" href="#5-read-from-storage" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h4&gt;
&lt;p&gt;Now we (hopefully) have pruned the Parquet file into small ranges of bytes, i.e., the &lt;a href="https://github.com/apache/datafusion/blob/76a7789ace33ced54c973fa0d5fc9d1866e1bf19/datafusion/datasource-parquet/src/access_plan.rs#L86"&gt;Access Plan&lt;/a&gt;.
The last step is to &lt;a href="https://github.com/apache/datafusion/blob/31701b8dc9c6486856c06a29a32107d9f4549cec/datafusion/core/src/datasource/physical_plan/parquet/reader.rs#L103"&gt;make requests&lt;/a&gt; to fetch those bytes and decode them into Arrow RecordBatch. &lt;/p&gt;
&lt;h3 id="preview-of-coming-attractions-filter-pushdown"&gt;Preview of coming attractions: filter pushdown&lt;a class="headerlink" href="#preview-of-coming-attractions-filter-pushdown" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;So far we have discussed techniques that prune the Parquet file using only the metadata, i.e., before reading the actual data.&lt;/p&gt;
&lt;p&gt;Filter pushdown, also known as predicate pushdown or late materialization, is a technique that prunes data during scanning, with filters being generated and applied in the Parquet reader.&lt;/p&gt;
&lt;p&gt;&lt;img alt="Filter pushdown in DataFusion" class="img-fluid" src="/blog/images/parquet-pruning/filter-pushdown.jpg" width="100%"/&gt;&lt;/p&gt;
&lt;p&gt;Unlike metadata-based pruning which works at the row group or page level, filter pushdown operates at the row level, allowing DataFusion to filter out individual rows that don't match the query predicates during the decoding process.&lt;/p&gt;
&lt;p&gt;DataFusion &lt;a href="https://github.com/apache/datafusion/blob/31701b8dc9c6486856c06a29a32107d9f4549cec/datafusion/core/src/datasource/physical_plan/parquet/row_filter.rs#L154"&gt;implements filter pushdown&lt;/a&gt; but has &lt;a href="https://github.com/apache/datafusion/blob/31701b8dc9c6486856c06a29a32107d9f4549cec/datafusion/common/src/config.rs#L382"&gt;not enabled it by default&lt;/a&gt; due to &lt;a href="https://github.com/apache/datafusion/issues/3463"&gt;some performance regressions&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;We are working to remove the &lt;a href="https://github.com/apache/arrow-rs/issues/5523#issuecomment-2429470872"&gt;remaining performance issues&lt;/a&gt; and enable it by default, which we will discuss in the next blog post.&lt;/p&gt;
&lt;h3 id="conclusion"&gt;Conclusion&lt;a class="headerlink" href="#conclusion" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;DataFusion employs a multi-step approach to Parquet pruning, from column projection to row group stats, page stats, and potentially row-level filtering. 
Each step may reduce the amount of data to be read and processed, significantly improving query performance.&lt;/p&gt;</content><category term="blog"/></entry><entry><title>Using Ordering for Better Plans in Apache DataFusion</title><link href="https://datafusion.apache.org/blog/2025/03/11/ordering-analysis" rel="alternate"/><published>2025-03-11T00:00:00+00:00</published><updated>2025-03-11T00:00:00+00:00</updated><author><name>Mustafa Akur, Andrew Lamb</name></author><id>tag:datafusion.apache.org,2025-03-11:/blog/2025/03/11/ordering-analysis</id><summary type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;!-- see https://github.com/apache/datafusion/issues/11631 for details --&gt;
&lt;h2 id="introduction"&gt;Introduction&lt;a class="headerlink" href="#introduction" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;In this blog post, we explain when an ordering requirement of an operator is satisfied by its input data. This analysis is essential for order-based optimizations and is often more complex than one might initially think.&lt;/p&gt;
&lt;blockquote class="border-start border-primary border-4 ps-3 py-2 bg-light"&gt;
&lt;strong&gt;Ordering Requirement&lt;/strong&gt; for an operator describes how the input data to that operator …&lt;/blockquote&gt;</summary><content type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;!-- see https://github.com/apache/datafusion/issues/11631 for details --&gt;
&lt;h2 id="introduction"&gt;Introduction&lt;a class="headerlink" href="#introduction" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;In this blog post, we explain when an ordering requirement of an operator is satisfied by its input data. This analysis is essential for order-based optimizations and is often more complex than one might initially think.&lt;/p&gt;
&lt;blockquote class="border-start border-primary border-4 ps-3 py-2 bg-light"&gt;
&lt;strong&gt;Ordering Requirement&lt;/strong&gt; for an operator describes how the input data to that operator must be sorted for the operator to compute the correct result. It is the job of the planner to make sure that these requirements are satisfied during execution (See DataFusion &lt;a href="https://docs.rs/datafusion/latest/datafusion/physical_optimizer/enforce_sorting/struct.EnforceSorting.html" target="_blank"&gt;EnforceSorting&lt;/a&gt; for an implementation of such a rule).
&lt;/blockquote&gt;
&lt;p&gt;There are various use cases where this type of analysis can be useful such as the following examples.&lt;/p&gt;
&lt;h3 id="removing-unnecessary-sorts"&gt;Removing Unnecessary Sorts&lt;a class="headerlink" href="#removing-unnecessary-sorts" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;Imagine a user wants to execute the following query:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-SQL"&gt;SELECT hostname, log_line 
FROM telemetry ORDER BY time ASC limit 10
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;If we don't know anything about the &lt;code&gt;telemetry&lt;/code&gt; table we need to sort it by &lt;code&gt;time ASC&lt;/code&gt; and then retrieve the first 10 rows to get the correct result. However, if the table is already ordered by &lt;code&gt;time ASC&lt;/code&gt;, we can simply retrieve the first 10 rows. This approach executes much faster and uses less memory compared to resorting the entire table, even when the &lt;a href="https://docs.rs/datafusion/latest/datafusion/physical_plan/struct.TopK.html"&gt;TopK&lt;/a&gt; operator is used. &lt;/p&gt;
&lt;p&gt;In order to avoid the sort the query optimizer must determine the data is already sorted. For simple queries the analysis is straightforward however it gets complicated fast. For example, what if your data is sorted by &lt;code&gt;[hostname, time ASC]&lt;/code&gt; and your query is&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-sql"&gt;SELECT hostname, log_line 
FROM telemetry WHERE hostname = 'app.example.com' ORDER BY time ASC;
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;In this case a sort still isn't needed, but the analysis must reason about the sortedness of the stream when it knows &lt;code&gt;hostname&lt;/code&gt; has a single value.&lt;/p&gt;
&lt;h3 id="optimized-operator-implementations"&gt;Optimized Operator Implementations&lt;a class="headerlink" href="#optimized-operator-implementations" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;As another use case, some operators can utilize the ordering information to change its underlying algorithm to execute more efficiently. Consider the following query:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-SQL"&gt;SELECT COUNT(log_line) 
FROM telemetry GROUP BY hostname;
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;Most analytic systems, including DataFusion, by default implement such a query using a hash table keyed on values of &lt;code&gt;hostname&lt;/code&gt; to store the counts. However, if the &lt;code&gt;telemetry&lt;/code&gt; table is sorted by &lt;code&gt;hostname&lt;/code&gt;,  there are much more efficient algorithms for grouping on &lt;code&gt;hostname&lt;/code&gt; values than hashing every value and storing it in memory. However, the more efficient algorithm can only be used when the input is sorted correctly. To see this in practice, check out the &lt;a href="https://github.com/apache/datafusion/tree/main/datafusion/physical-plan/src/aggregates/order"&gt;source&lt;/a&gt; for ordered variant of the &lt;code&gt;Aggregation&lt;/code&gt; in &lt;code&gt;DataFusion&lt;/code&gt;.&lt;/p&gt;
&lt;h3 id="streaming-friendly-execution"&gt;Streaming-Friendly Execution&lt;a class="headerlink" href="#streaming-friendly-execution" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;Stream processing aims to produce results immediately as they become available ensuring minimal latency for real-time workloads. However, some operators need to consume all input data before producing any output. Consider the &lt;code&gt;Sort&lt;/code&gt; operation: before it can start generating output, the algorithm must first process all input data. As a result, data flow halts whenever such an operator is encountered until all input is consumed. When a physical query plan contains such an operator (&lt;code&gt;Sort&lt;/code&gt;, &lt;code&gt;CrossJoin&lt;/code&gt;, ..) we refer to this as pipeline breaking, meaning the query cannot be executed in a streaming fashion.&lt;/p&gt;
&lt;p&gt;For a query to be executed in a streaming fashion we need to satisfy 2 conditions:&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Logically Streamable&lt;/strong&gt;&lt;br/&gt;
It should be possible to generate what user wants in streaming fashion. Consider following query:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-SQL"&gt;SELECT SUM(amount)  
FROM orders  
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;Here, the user wants to compute the sum of all amounts in the orders table. By the nature of the query this requires scanning the entire table to generate a result making it impossible to execute in a streaming fashion.&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Streaming Aware Planner&lt;/strong&gt;&lt;br/&gt;
Being logically streamable does not guarantee that a query will execute in a streaming fashion. SQL is a declarative language, meaning it specifies 'WHAT' user wants. It is up to the planner 'HOW' to generate the result. In most cases there are many ways to compute the correct result for a given query. The query planner is responsible for choosing "a way" (ideally the best&lt;sup id="optimal1"&gt;&lt;a href="#optimal"&gt;*&lt;/a&gt;&lt;/sup&gt; one) among the all alternatives to generate what user asks for. If a plan contains a pipeline-breaking operator the execution will not be streaming—even if the query is logically streamable. To generate truly streaming plans from logically streamable queries the planner must carefully analyze the existing orderings in the source tables to ensure that the final plan does not contain any pipeline-breaking operators.&lt;/p&gt;
&lt;h2 id="analysis"&gt;Analysis&lt;a class="headerlink" href="#analysis" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;Let's start by creating an example table that we will refer throughout the post. This table models the input data of an operator for the analysis:&lt;/p&gt;
&lt;h3 id="example-virtual-table"&gt;Example Virtual Table&lt;a class="headerlink" href="#example-virtual-table" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;style&gt;
  table {
    border-collapse: collapse;
    width: 80%;
    font-family: -apple-system, BlinkMacSystemFont, "Segoe UI", Roboto, Helvetica, Arial, sans-serif;
  }
  th, td {
    padding: 12px 16px;
    text-align: left;
    border-bottom: 1px solid #e0e0e0;
  }
  th {
    background-color: #f9f9f9;
    font-weight: 600;
  }
  tr:hover {
    background-color: #f1f1f1;
  }
&lt;/style&gt;
&lt;table class="table"&gt;
&lt;tr&gt;
&lt;th&gt;amount&lt;/th&gt; &lt;th&gt;price&lt;/th&gt; &lt;th&gt;hostname&lt;/th&gt;&lt;th&gt;currency&lt;/th&gt;&lt;th&gt;time_bin&lt;/th&gt; &lt;th&gt;time&lt;/th&gt; &lt;th&gt;price_cloned&lt;/th&gt; &lt;th&gt;time_cloned&lt;/th&gt;
&lt;/tr&gt;
&lt;tr&gt;
&lt;td&gt;12&lt;/td&gt; &lt;td&gt;25&lt;/td&gt; &lt;td&gt;app.example.com&lt;/td&gt; &lt;td&gt;USD&lt;/td&gt; &lt;td&gt;08:00:00&lt;/td&gt; &lt;td&gt;08:01:30&lt;/td&gt; &lt;td&gt;25&lt;/td&gt; &lt;td&gt;08:01:30&lt;/td&gt;
&lt;/tr&gt;
&lt;tr&gt;
&lt;td&gt;12&lt;/td&gt; &lt;td&gt;26&lt;/td&gt; &lt;td&gt;app.example.com&lt;/td&gt; &lt;td&gt;USD&lt;/td&gt; &lt;td&gt;08:00:00&lt;/td&gt; &lt;td&gt;08:11:30&lt;/td&gt; &lt;td&gt;26&lt;/td&gt; &lt;td&gt;08:11:30&lt;/td&gt;
&lt;/tr&gt;
&lt;tr&gt;
&lt;td&gt;15&lt;/td&gt; &lt;td&gt;30&lt;/td&gt; &lt;td&gt;app.example.com&lt;/td&gt; &lt;td&gt;USD&lt;/td&gt; &lt;td&gt;08:00:00&lt;/td&gt; &lt;td&gt;08:41:30&lt;/td&gt; &lt;td&gt;30&lt;/td&gt; &lt;td&gt;08:41:30&lt;/td&gt;
&lt;/tr&gt;
&lt;tr&gt;
&lt;td&gt;15&lt;/td&gt; &lt;td&gt;32&lt;/td&gt; &lt;td&gt;app.example.com&lt;/td&gt; &lt;td&gt;USD&lt;/td&gt; &lt;td&gt;08:00:00&lt;/td&gt; &lt;td&gt;08:55:15&lt;/td&gt; &lt;td&gt;32&lt;/td&gt; &lt;td&gt;08:55:15&lt;/td&gt;
&lt;/tr&gt;
&lt;tr&gt;
&lt;td&gt;15&lt;/td&gt; &lt;td&gt;35&lt;/td&gt; &lt;td&gt;app.example.com&lt;/td&gt; &lt;td&gt;USD&lt;/td&gt; &lt;td&gt;09:00:00&lt;/td&gt; &lt;td&gt;09:10:23&lt;/td&gt; &lt;td&gt;35&lt;/td&gt; &lt;td&gt;09:10:23&lt;/td&gt;
&lt;/tr&gt;
&lt;tr&gt;
&lt;td&gt;20&lt;/td&gt; &lt;td&gt;18&lt;/td&gt; &lt;td&gt;app.example.com&lt;/td&gt; &lt;td&gt;USD&lt;/td&gt; &lt;td&gt;09:00:00&lt;/td&gt; &lt;td&gt;09:20:33&lt;/td&gt; &lt;td&gt;18&lt;/td&gt; &lt;td&gt;09:20:33&lt;/td&gt;
&lt;/tr&gt;
&lt;tr&gt;
&lt;td&gt;20&lt;/td&gt; &lt;td&gt;22&lt;/td&gt; &lt;td&gt;app.example.com&lt;/td&gt; &lt;td&gt;USD&lt;/td&gt; &lt;td&gt;09:00:00&lt;/td&gt; &lt;td&gt;09:40:15&lt;/td&gt; &lt;td&gt;22&lt;/td&gt; &lt;td&gt;09:40:15&lt;/td&gt;
&lt;/tr&gt;
&lt;/table&gt;
&lt;p&gt;&lt;br/&gt;&lt;/p&gt;
&lt;blockquote class="border-start border-primary border-4 ps-3 py-2 bg-light"&gt;
&lt;strong&gt;How can a table have multiple orderings?&lt;/strong&gt; At first glance it may seem counterintuitive for a table to have more than one valid ordering. However, during query execution such scenarios can arise.

For example consider the following query:


&lt;pre&gt;&lt;code class="language-sql"&gt;SELECT time, date_bin('1 hour', time, '1970-01-01') as time_bin  
FROM table;
&lt;/code&gt;&lt;/pre&gt;

If we know that the table is ordered by &lt;code&gt;time ASC&lt;/code&gt; we can infer that &lt;code&gt;time_bin ASC&lt;/code&gt; is also a valid ordering. This is because the &lt;code&gt;date_bin&lt;/code&gt; function is monotonic, meaning it preserves the order of its input.

DataFusion leverages these functional dependencies to infer new orderings as data flows through different query operators. For details on the implementation see the &lt;a ,="" href="https://github.com/apache/datafusion/blob/main/datafusion/common/src/functional_dependencies.rs" target="_blank"&gt;source&lt;/a&gt; code.
&lt;/blockquote&gt;
&lt;p&gt;By inspection, you can see this table is sorted by the &lt;code&gt;amount&lt;/code&gt; column, but It is also sorted by &lt;code&gt;time&lt;/code&gt; and &lt;code&gt;time_bin&lt;/code&gt; as well as the compound &lt;code&gt;(time_bin, amount)&lt;/code&gt; and many other variations. While this example is an extreme case, real-world data often has multiple sort orders. &lt;/p&gt;
&lt;p&gt;A naive approach for analyzing whether the ordering requirement of an operator is satisfied by its input would be:  &lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;Store all the valid ordering expressions that the tables satisfies  &lt;/li&gt;
&lt;li&gt;Check whether the ordering requirement by the operator is among valid orderings.  &lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;This naive algorithm works and correct. However, listing all valid orderings can be quite lengthy and is of exponential complexity as the number of orderings grows. For the example table here is a (small) subset of the valid orderings:&lt;/p&gt;
&lt;p&gt;&lt;code&gt;[amount ASC]&lt;/code&gt;&lt;br/&gt;
&lt;code&gt;[amount ASC, price ASC]&lt;/code&gt;&lt;br/&gt;
&lt;code&gt;[amount ASC, price_cloned ASC]&lt;/code&gt;&lt;br/&gt;
&lt;code&gt;[hostname ASC, amount ASC, price_cloned ASC]&lt;/code&gt;&lt;br/&gt;
&lt;code&gt;[amount ASC, hostname ASC,  price_cloned ASC]&lt;/code&gt;&lt;br/&gt;
&lt;code&gt;[amount ASC, price_cloned ASC, hostname ASC]&lt;/code&gt;&lt;br/&gt;
.&lt;br/&gt;
.&lt;br/&gt;
.  &lt;/p&gt;
&lt;p&gt;As can be seen from the listing above storing all valid orderings is wasteful and contains significant redundancy. Here are some observations which suggest that we can do much better:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;
&lt;p&gt;Storing a prefix of another valid ordering is redundant. If the table satisfies the lexicographic ordering&lt;sup id="fn1"&gt;&lt;a href="#footnote1"&gt;1&lt;/a&gt;&lt;/sup&gt;: &lt;code&gt;[amount ASC, price ASC]&lt;/code&gt;, it already satisfies ordering &lt;code&gt;[amount ASC]&lt;/code&gt; trivially. Hence, once we store &lt;code&gt;[amount ASC, price ASC]&lt;/code&gt; storing &lt;code&gt;[amount ASC]&lt;/code&gt; is redundant.&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;
&lt;p&gt;Using all columns that are equal to each other in the listings is redundant. If we know the table is ordered by &lt;code&gt;[amount ASC, price ASC]&lt;/code&gt;, it is also ordered by &lt;code&gt;[amount ASC, price_cloned ASC]&lt;/code&gt; since &lt;code&gt;price&lt;/code&gt; and &lt;code&gt;price_cloned&lt;/code&gt; are copy of each other. It is enough to use just one expression among the expressions that exact copy of each other.&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;
&lt;p&gt;Constant expressions can be inserted anywhere in a valid ordering with an arbitrary direction (e.g. &lt;code&gt;ASC&lt;/code&gt;, &lt;code&gt;DESC&lt;/code&gt;). Hence, if the table is ordered by &lt;code&gt;[amount ASC, price ASC]&lt;/code&gt;, it is also ordered by: &lt;br/&gt;
&lt;code&gt;[hostname ASC, amount ASC, price ASC]&lt;/code&gt;,&lt;br/&gt;
&lt;code&gt;[hostname DESC, amount ASC, price ASC]&lt;/code&gt;,&lt;br/&gt;
&lt;code&gt;[amount ASC, hostname ASC, price ASC]&lt;/code&gt;,&lt;br/&gt;
   .&lt;br/&gt;
   .    &lt;/p&gt;
&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;This is clearly redundant. For this reason, it is better to avoid explicitly encoding constant expressions in valid sort orders.&lt;/p&gt;
&lt;p&gt;In summary,&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;We should store only the longest lexicographic ordering (shouldn't use any prefix of it)&lt;/li&gt;
&lt;li&gt;Using expressions that are exact copies of each other is redundant.&lt;/li&gt;
&lt;li&gt;Ordering expressions shouldn't contain any constant expression.&lt;/li&gt;
&lt;/ul&gt;
&lt;h2 id="key-concepts-for-analyzing-orderings"&gt;Key Concepts for Analyzing Orderings&lt;a class="headerlink" href="#key-concepts-for-analyzing-orderings" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;To solve the shortcomings above DataFusion needs to track of following properties for the table:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;Constant Expressions  &lt;/li&gt;
&lt;li&gt;Equivalent Expression Groups (will be explained shortly)&lt;/li&gt;
&lt;li&gt;Succinct Valid Orderings (will be explained shortly)&lt;/li&gt;
&lt;/ul&gt;
&lt;blockquote class="border-start border-primary border-4 ps-3 py-2 bg-light"&gt;
&lt;strong&gt;Note:&lt;/strong&gt; These properties are implemented in the &lt;code&gt;EquivalenceProperties&lt;/code&gt; structure in &lt;code&gt;DataFusion&lt;/code&gt;, please see the &lt;a href="https://github.com/apache/datafusion/blob/f47ea73b87eec4af044f9b9923baf042682615b2/datafusion/physical-expr/src/equivalence/properties/mod.rs#L134" target="_blank"&gt;source&lt;/a&gt; for more details&lt;br/&gt;
&lt;/blockquote&gt;
&lt;p&gt;These properties allow us to analyze whether the ordering requirement is satisfied by the data already.&lt;/p&gt;
&lt;h3 id="1-constant-expressions"&gt;1. Constant Expressions&lt;a class="headerlink" href="#1-constant-expressions" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;Constant expressions are those where each row in the expression has the same value across all rows. Although constant expressions may seem odd in a table they can arise after operations like &lt;code&gt;Filter&lt;/code&gt; or &lt;code&gt;Join&lt;/code&gt; occur. &lt;/p&gt;
&lt;p&gt;For instance in the example table:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;Columns &lt;code&gt;hostname&lt;/code&gt; and &lt;code&gt;currency&lt;/code&gt; are constant because every row in the table has the same value (&lt;code&gt;'app.example.com'&lt;/code&gt; for &lt;code&gt;hostname&lt;/code&gt;, and &lt;code&gt;'USD'&lt;/code&gt; for &lt;code&gt;currency&lt;/code&gt;) for these columns.&lt;/li&gt;
&lt;/ul&gt;
&lt;blockquote class="border-start border-primary border-4 ps-3 py-2 bg-light"&gt;
&lt;strong&gt;Note:&lt;/strong&gt; Constant expressions can arise during query execution. For example, in following query:&lt;br/&gt;
&lt;code&gt;SELECT hostname FROM logs&lt;/code&gt;&lt;br/&gt;&lt;code&gt;WHERE hostname='app.example.com'&lt;/code&gt; &lt;br/&gt;
    after filtering is done, for subsequent operators the &lt;code&gt;hostname&lt;/code&gt; column will be constant.
&lt;/blockquote&gt;
&lt;h3 id="2-equivalent-expression-groups"&gt;2. Equivalent Expression Groups&lt;a class="headerlink" href="#2-equivalent-expression-groups" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;Equivalent expression groups are expressions that always hold the same value across rows. These expressions can be thought of as clones of each other and may arise from operations like &lt;code&gt;Filter&lt;/code&gt;, &lt;code&gt;Join&lt;/code&gt;, or &lt;code&gt;Projection&lt;/code&gt;.&lt;/p&gt;
&lt;p&gt;In the example table, the expressions &lt;code&gt;price&lt;/code&gt; and &lt;code&gt;price_cloned&lt;/code&gt; form one equivalence group, and &lt;code&gt;time&lt;/code&gt; and &lt;code&gt;time_cloned&lt;/code&gt; form another equivalence group.&lt;/p&gt;
&lt;blockquote class="border-start border-primary border-4 ps-3 py-2 bg-light"&gt;
&lt;strong&gt;Note:&lt;/strong&gt; Equivalent expression groups can arise during the query execution. For example, in the following query:&lt;br/&gt;
&lt;code&gt;SELECT time, time as time_cloned FROM logs&lt;/code&gt; &lt;br/&gt;
    after the projection is done, for subsequent operators &lt;code&gt;time&lt;/code&gt; and &lt;code&gt;time_cloned&lt;/code&gt; will form an equivalence group. As another example, in the following query:&lt;br/&gt;
&lt;code&gt;SELECT employees.id, employees.name, departments.department_name&lt;/code&gt;
&lt;code&gt;FROM employees&lt;/code&gt;
&lt;code&gt;JOIN departments ON employees.department_id = departments.id;&lt;/code&gt; &lt;br/&gt;
after joining, &lt;code&gt;employees.department_id&lt;/code&gt; and &lt;code&gt;departments.id&lt;/code&gt; will form an equivalence group.
&lt;/blockquote&gt;
&lt;h3 id="3-succinct-encoding-of-valid-orderings"&gt;3. Succinct Encoding of Valid Orderings&lt;a class="headerlink" href="#3-succinct-encoding-of-valid-orderings" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;Valid orderings are the orderings that the table already satisfies. However, naively listing them requires exponential space as the number of columns grows as discussed before. Instead, we list all valid orderings after following constraints are applied:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;Do not use any constant expressions in the valid ordering construction&lt;/li&gt;
&lt;li&gt;Use only one entry (by convention the first entry) in the equivalent expression group.&lt;/li&gt;
&lt;li&gt;Lexicographic ordering shouldn't contain any leading ordering&lt;sup id="fn2"&gt;&lt;a href="#footnote2"&gt;2&lt;/a&gt;&lt;/sup&gt;except the first position &lt;sup id="fn3"&gt;&lt;a href="#footnote3"&gt;3&lt;/a&gt;&lt;/sup&gt;.&lt;/li&gt;
&lt;li&gt;Do not use any prefix of a valid lexicographic ordering&lt;sup id="fn4"&gt;&lt;a href="#footnote4"&gt;4&lt;/a&gt;&lt;/sup&gt;.&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;After applying the first and second constraint, the example table simplifies to &lt;/p&gt;
&lt;style&gt;
  table {
    border-collapse: collapse;
    width: 80%;
    font-family: -apple-system, BlinkMacSystemFont, "Segoe UI", Roboto, Helvetica, Arial, sans-serif;
  }
  th, td {
    padding: 12px 16px;
    text-align: left;
    border-bottom: 1px solid #e0e0e0;
  }
  th {
    background-color: #f9f9f9;
    font-weight: 600;
  }
  tr:hover {
    background-color: #f1f1f1;
  }
&lt;/style&gt;
&lt;table class="table"&gt;
&lt;tr&gt;
&lt;th&gt;amount&lt;/th&gt; &lt;th&gt;price&lt;/th&gt;&lt;th&gt;time_bin&lt;/th&gt; &lt;th&gt;time&lt;/th&gt;
&lt;/tr&gt;
&lt;tr&gt;
&lt;td&gt;12&lt;/td&gt; &lt;td&gt;25&lt;/td&gt;&lt;td&gt;08:00:00&lt;/td&gt; &lt;td&gt;08:01:30&lt;/td&gt;
&lt;/tr&gt;
&lt;tr&gt;
&lt;td&gt;12&lt;/td&gt; &lt;td&gt;26&lt;/td&gt;&lt;td&gt;08:00:00&lt;/td&gt; &lt;td&gt;08:11:30&lt;/td&gt;
&lt;/tr&gt;
&lt;tr&gt;
&lt;td&gt;15&lt;/td&gt; &lt;td&gt;30&lt;/td&gt;&lt;td&gt;08:00:00&lt;/td&gt; &lt;td&gt;08:41:30&lt;/td&gt;
&lt;/tr&gt;
&lt;tr&gt;
&lt;td&gt;15&lt;/td&gt; &lt;td&gt;32&lt;/td&gt;&lt;td&gt;08:00:00&lt;/td&gt; &lt;td&gt;08:55:15&lt;/td&gt;
&lt;/tr&gt;
&lt;tr&gt;
&lt;td&gt;15&lt;/td&gt; &lt;td&gt;35&lt;/td&gt;&lt;td&gt;09:00:00&lt;/td&gt; &lt;td&gt;09:10:23&lt;/td&gt;
&lt;/tr&gt;
&lt;tr&gt;
&lt;td&gt;20&lt;/td&gt; &lt;td&gt;18&lt;/td&gt;&lt;td&gt;09:00:00&lt;/td&gt; &lt;td&gt;09:20:33&lt;/td&gt;
&lt;/tr&gt;
&lt;tr&gt;
&lt;td&gt;20&lt;/td&gt; &lt;td&gt;22&lt;/td&gt;&lt;td&gt;09:00:00&lt;/td&gt; &lt;td&gt;09:40:15&lt;/td&gt;
&lt;/tr&gt;
&lt;/table&gt;
&lt;p&gt;&lt;br/&gt;
Following third and fourth constraints for the simplified table, the succinct valid orderings are:&lt;br/&gt;
&lt;code&gt;[amount ASC, price ASC]&lt;/code&gt;, &lt;br/&gt;
&lt;code&gt;[time_bin ASC]&lt;/code&gt;,&lt;br/&gt;
&lt;code&gt;[time ASC]&lt;/code&gt; &lt;/p&gt;
&lt;blockquote class="border-start border-primary border-4 ps-3 py-2 bg-light"&gt;
&lt;p&gt;&lt;strong&gt;How can DataFusion find orderings?&lt;/strong&gt;&lt;/p&gt; 
DataFusion's &lt;code&gt;CREATE EXTERNAL TABLE&lt;/code&gt; has a &lt;code&gt;WITH ORDER&lt;/code&gt; clause (see &lt;a href="https://datafusion.apache.org/user-guide/sql/ddl.html#create-external-table"&gt;docs&lt;/a&gt;) to specify the known orderings of the table during table creation. For example the following query:&lt;br/&gt;
&lt;pre&gt;&lt;code&gt;
CREATE EXTERNAL TABLE source (
    amount INT NOT NULL,
    price DOUBLE NOT NULL,
    time TIMESTAMP NOT NULL,
    ...
)
STORED AS CSV
WITH ORDER (time ASC)
WITH ORDER (amount ASC, price ASC)
LOCATION '/path/to/FILE_NAME.csv'
OPTIONS ('has_header' 'true');
&lt;/code&gt;&lt;/pre&gt;
communicates that &lt;code&gt;source&lt;/code&gt; table has the orderings: &lt;code&gt;[time ASC]&lt;/code&gt; and &lt;code&gt;[amount ASC, price ASC]&lt;/code&gt;.&lt;br/&gt;
When orderings are communicated from the source, DataFusion tracks the orderings through each operator while optimizing the plan.&lt;br/&gt;
&lt;ul&gt;
&lt;li&gt;add new orderings (such as when "date_bin" function is applied to the "time" column)&lt;/li&gt;
&lt;li&gt;Remove orderings, if operation doesn't preserve the ordering of the data at its input&lt;/li&gt;
&lt;li&gt;Update equivalent groups&lt;/li&gt;
&lt;li&gt;Update constant expressions&lt;/li&gt;
&lt;/ul&gt;

Figure 1 shows an example how DataFusion generates an efficient plan for the query:
&lt;pre&gt;&lt;code&gt;
SELECT 
  row_number() OVER (ORDER BY time) as rn,
  time
FROM events
ORDER BY rn, time
&lt;/code&gt;&lt;/pre&gt;
using the orderings of the query intermediates.&lt;br/&gt;
&lt;br/&gt;
&lt;figure&gt;
&lt;img alt="Window Query Datafusion Optimization" class="img-fluid" src="/blog/images/ordering_analysis/query_window_plan.png" width="80%"/&gt;
&lt;figcaption&gt;&lt;strong&gt;Figure 1:&lt;/strong&gt; DataFusion analyzes orderings of the sources and query intermediates to generate efficient plans&lt;/figcaption&gt;
&lt;/figure&gt;
&lt;/blockquote&gt;
&lt;h3 id="table-properties"&gt;Table Properties&lt;a class="headerlink" href="#table-properties" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;In summary, for the example table, the following properties correctly describe the sort properties:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;strong&gt;Constant Expressions&lt;/strong&gt; = &lt;code&gt;hostname, currency&lt;/code&gt; &lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Equivalent Expression Groups&lt;/strong&gt; = &lt;code&gt;[price, price_cloned], [time, time_cloned]&lt;/code&gt; &lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Valid Orderings&lt;/strong&gt; = &lt;code&gt;[amount ASC, price ASC], [time_bin ASC], [time ASC]&lt;/code&gt; &lt;/li&gt;
&lt;/ul&gt;
&lt;h3 id="algorithm-for-analyzing-ordering-requirements"&gt;Algorithm for Analyzing Ordering Requirements&lt;a class="headerlink" href="#algorithm-for-analyzing-ordering-requirements" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;After deriving these properties for the data, following algorithm can be used to check whether an ordering requirement is satisfied by the table:&lt;/p&gt;
&lt;ol&gt;
&lt;li&gt;&lt;strong&gt;Prune constant expressions&lt;/strong&gt;: Remove any constant expressions from the ordering requirement.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Normalize the requirement&lt;/strong&gt;: Replace each expression in the ordering requirement with the first entry from its equivalence group.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;De-duplicate expressions&lt;/strong&gt;: If an expression appears more than once, remove duplicates, keeping only the first occurrence.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Match leading orderings&lt;/strong&gt;: Check whether the leading ordering requirement&lt;sup id="fn5"&gt;&lt;a href="#footnote5"&gt;5&lt;/a&gt;&lt;/sup&gt; matches the leading valid orderings&lt;sup id="fn6"&gt;&lt;a href="#footnote6"&gt;6&lt;/a&gt;&lt;/sup&gt; of table. If so:&lt;ul&gt;
&lt;li&gt;Remove the leading ordering requirement from the ordering requirement &lt;/li&gt;
&lt;li&gt;Remove the matching leading valid ordering from the valid orderings of table. &lt;/li&gt;
&lt;/ul&gt;
&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Iterate through the remaining expressions&lt;/strong&gt;: Go back to step 4 until ordering requirement is empty or leading ordering requirement is not found among the leading valid orderings of table.&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;If, at the end of the procedure above, the ordering requirement is an empty list, we can conclude that the requirement is satisfied by the table.&lt;/p&gt;
&lt;h3 id="example-walkthrough"&gt;Example Walkthrough&lt;a class="headerlink" href="#example-walkthrough" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;Let's say the user provided a query such as the following:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-sql"&gt;SELECT * FROM table
ORDER BY hostname DESC, amount ASC, time_bin ASC, price_cloned ASC, time ASC, currency ASC, price DESC;
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;And the input has the same properties explained above&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;strong&gt;Constant Expressions&lt;/strong&gt; = &lt;code&gt;hostname, currency&lt;/code&gt;&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Equivalent Expressions Groups&lt;/strong&gt; = &lt;code&gt;[price, price_cloned], [time, time_cloned]&lt;/code&gt;&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Succinct Valid Orderings&lt;/strong&gt; = &lt;code&gt;[amount ASC, price ASC], [time_bin ASC], [time ASC]&lt;/code&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;In order to remove a sort the optimizer must check if the ordering requirement &lt;code&gt;[hostname DESC, amount ASC, time_bin ASC, price_cloned ASC, time ASC, currency ASC, price DESC]&lt;/code&gt; is satisfied by the properties.&lt;/p&gt;
&lt;h3 id="algorithm-steps"&gt;Algorithm Steps&lt;a class="headerlink" href="#algorithm-steps" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;ol&gt;
&lt;li&gt;
&lt;p&gt;&lt;strong&gt;Prune constant expressions&lt;/strong&gt;:&lt;br/&gt;
   Remove &lt;code&gt;hostname&lt;/code&gt; and &lt;code&gt;currency&lt;/code&gt; from the requirement. The requirement becomes:&lt;br/&gt;
&lt;code&gt;[amount ASC, time_bin ASC, price_cloned ASC, time ASC, price DESC]&lt;/code&gt;.&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;
&lt;p&gt;&lt;strong&gt;Normalize using equivalent groups&lt;/strong&gt;:&lt;br/&gt;
   Replace &lt;code&gt;price_cloned&lt;/code&gt; with &lt;code&gt;price&lt;/code&gt; and &lt;code&gt;time_cloned&lt;/code&gt; with &lt;code&gt;time&lt;/code&gt;. The requirement becomes:&lt;br/&gt;
&lt;code&gt;[amount ASC, time_bin ASC, price ASC, time ASC, price DESC]&lt;/code&gt;.&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;
&lt;p&gt;&lt;strong&gt;De-duplicate expressions&lt;/strong&gt;:&lt;br/&gt;
   Since &lt;code&gt;price&lt;/code&gt; appears twice, we simplify the requirement to:&lt;br/&gt;
&lt;code&gt;[amount ASC, time_bin ASC, price ASC, time ASC]&lt;/code&gt; (keeping the first occurrence from the left side).&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;
&lt;p&gt;&lt;strong&gt;Match leading orderings&lt;/strong&gt;:&lt;br/&gt;
  Check if leading ordering requirement &lt;code&gt;amount ASC&lt;/code&gt; is among the leading valid orderings: &lt;code&gt;amount ASC, time_bin ASC, time ASC&lt;/code&gt;. Since this is the case, we remove &lt;code&gt;amount ASC&lt;/code&gt; from both the ordering requirement and the valid orderings of the table.&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Iterate through the remaining expressions&lt;/strong&gt;:
Now, the problem is converted from&lt;br/&gt;
&lt;em&gt;"whether the requirement: &lt;code&gt;[amount ASC, time_bin ASC, price ASC, time ASC]&lt;/code&gt; is satisfied by valid orderings:  &lt;code&gt;[amount ASC, price ASC], [time_bin ASC], [time ASC]&lt;/code&gt;"&lt;/em&gt;&lt;br/&gt;
into&lt;br/&gt;
&lt;em&gt;"whether the requirement: &lt;code&gt;[time_bin ASC, price ASC, time ASC]&lt;/code&gt; is satisfied by valid orderings:  &lt;code&gt;[price ASC], [time_bin ASC], [time ASC]&lt;/code&gt;"&lt;/em&gt;&lt;br/&gt;
We go back to step 4 until the ordering requirement list is exhausted or its length no longer decreases.&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;At the end of stages above we end up with an empty ordering requirement list. Given this, we can conclude that the table satisfies the ordering requirement and thus no sort is required. &lt;/p&gt;
&lt;h2 id="conclusion"&gt;Conclusion&lt;a class="headerlink" href="#conclusion" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;In this post, we described the conditions under which an ordering requirement is satisfied based on the properties of a table. We introduced key concepts such as constant expressions, equivalence groups, and valid orderings, and used them to determine whether a given ordering requirement are satisfied by an input table.&lt;/p&gt;
&lt;p&gt;This analysis plays a crucial role in:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;Choosing more efficient algorithm variants&lt;/li&gt;
&lt;li&gt;Generating streaming-friendly plans&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;The &lt;code&gt;DataFusion&lt;/code&gt; query engine employs this analysis (and many more) during its planning stage to ensure correct and efficient query execution. We &lt;a href="https://datafusion.apache.org/contributor-guide/index.html"&gt;welcome you&lt;/a&gt; to come and join the project.&lt;/p&gt;
&lt;h2 id="appendix"&gt;Appendix&lt;a class="headerlink" href="#appendix" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;!--
&lt;p id="footnote1"&gt;&lt;sup&gt;[1]&lt;/sup&gt;The ordering requirement refers to the condition that input data must be sorted in a certain way for a specific operator to function as intended.&lt;/p&gt;
--&gt;
&lt;p id="footnote1"&gt;&lt;sup&gt;[1]&lt;/sup&gt;Lexicographic order is a way of ordering sequences (like strings, list of expressions) based on the order of their components, similar to how words are ordered in a dictionary. It compares each element of the sequences one by one, from left to right.&lt;/p&gt;
&lt;p id="footnote2"&gt;&lt;sup&gt;[2]&lt;/sup&gt;Leading ordering is the first ordering in a lexicographic ordering list. As an example, for the ordering: &lt;code&gt;[amount ASC, price ASC]&lt;/code&gt;, leading ordering will be: &lt;code&gt;amount ASC&lt;/code&gt;. &lt;/p&gt;
&lt;p id="footnote3"&gt;&lt;sup&gt;[3]&lt;/sup&gt;This means that, if we know that &lt;code&gt;[amount ASC]&lt;/code&gt; and &lt;code&gt;[time ASC]&lt;/code&gt; are both valid orderings for the table. We shouldn't enlist &lt;code&gt;[amount ASC, time ASC]&lt;/code&gt; or &lt;code&gt;[time ASC, amount ASC]&lt;/code&gt; as valid orderings. These orderings can be deduced if we know that table satisfies the ordering &lt;code&gt;[amount ASC]&lt;/code&gt; and &lt;code&gt;[time ASC]&lt;/code&gt;.&lt;/p&gt;
&lt;p id="footnote4"&gt;&lt;sup&gt;[4]&lt;/sup&gt;This means that, if ordering &lt;code&gt;[amount ASC, price ASC]&lt;/code&gt; is a valid ordering for the table. We shouldn't enlist &lt;code&gt;[amount ASC]&lt;/code&gt; as valid ordering. Validity of it can be deduced from the ordering: &lt;code&gt;[amount ASC, price ASC]&lt;/code&gt;
&lt;p id="footnote5"&gt;&lt;sup&gt;[5]&lt;/sup&gt;Leading ordering requirement is the first ordering requirement in the list of lexicographic ordering requirement expression. As an example for the requirement: &lt;code&gt;[amount ASC, time_bin ASC, prices ASC, time ASC]&lt;/code&gt;, leading ordering requirement is: &lt;code&gt;amount ASC&lt;/code&gt;.&lt;/p&gt;
&lt;p id="footnote6"&gt;&lt;sup&gt;[6]&lt;/sup&gt;Leading valid orderings are the first ordering for each valid ordering list in the table. As an example, for the valid orderings: &lt;code&gt;[amount ASC, prices ASC], [time_bin ASC], [time ASC]&lt;/code&gt;, leading valid orderings will be: &lt;code&gt;amount ASC, time_bin ASC, time ASC&lt;/code&gt;. &lt;/p&gt;
&lt;p id="optimal"&gt;&lt;sup&gt;*&lt;/sup&gt;Best depends on the use case, &lt;code&gt;DataFusion&lt;/code&gt; has many various flags to communicate what user thinks the best plan is (e.g. streamable, fastest, lowest memory, etc.). See &lt;a href="https://datafusion.apache.org/user-guide/configs.html" target="_blank"&gt;configurations&lt;/a&gt; for detail.&lt;/p&gt;&lt;/p&gt;</content><category term="blog"/></entry><entry><title>Apache DataFusion 45.0.0 Released</title><link href="https://datafusion.apache.org/blog/2025/02/20/datafusion-45.0.0" rel="alternate"/><published>2025-02-20T00:00:00+00:00</published><updated>2025-02-20T00:00:00+00:00</updated><author><name>pmc</name></author><id>tag:datafusion.apache.org,2025-02-20:/blog/2025/02/20/datafusion-45.0.0</id><summary type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;!-- see https://github.com/apache/datafusion/issues/11631 for details --&gt;
&lt;h2 id="introduction"&gt;Introduction&lt;a class="headerlink" href="#introduction" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;We are very proud to announce &lt;a href="https://crates.io/crates/datafusion/45.0.0"&gt;DataFusion 45.0.0&lt;/a&gt;. This blog highlights some of the
many major improvements since we released &lt;a href="https://datafusion.apache.org/blog/2024/07/24/datafusion-40.0.0/"&gt;DataFusion 40.0.0&lt;/a&gt; and a preview of
what the community is thinking about in the next 6 months. It has been an exciting
period of development …&lt;/p&gt;</summary><content type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;!-- see https://github.com/apache/datafusion/issues/11631 for details --&gt;
&lt;h2 id="introduction"&gt;Introduction&lt;a class="headerlink" href="#introduction" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;We are very proud to announce &lt;a href="https://crates.io/crates/datafusion/45.0.0"&gt;DataFusion 45.0.0&lt;/a&gt;. This blog highlights some of the
many major improvements since we released &lt;a href="https://datafusion.apache.org/blog/2024/07/24/datafusion-40.0.0/"&gt;DataFusion 40.0.0&lt;/a&gt; and a preview of
what the community is thinking about in the next 6 months. It has been an exciting
period of development for DataFusion!&lt;/p&gt;
&lt;p&gt;&lt;a href="https://datafusion.apache.org/"&gt;Apache DataFusion&lt;/a&gt; is an extensible query engine, written in &lt;a href="https://www.rust-lang.org/"&gt;Rust&lt;/a&gt;, that
uses &lt;a href="https://arrow.apache.org"&gt;Apache Arrow&lt;/a&gt; as its in-memory format. DataFusion is used by developers to
create new, fast data centric systems such as databases, dataframe libraries,
machine learning and streaming applications. While &lt;a href="https://datafusion.apache.org/user-guide/introduction.html#project-goals"&gt;DataFusion’s primary design
goal&lt;/a&gt; is to accelerate the creation of other data centric systems, it has a
reasonable experience directly out of the box as a &lt;a href="https://datafusion.apache.org/user-guide/dataframe.html"&gt;dataframe library&lt;/a&gt;,
&lt;a href="https://datafusion.apache.org/python/"&gt;python library&lt;/a&gt; and &lt;a href="https://datafusion.apache.org/user-guide/cli/"&gt;command line SQL tool&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;DataFusion's core thesis is that as a community, together we can build much more
advanced technology than any of us as individuals or companies could do alone. 
Without DataFusion, highly performant vectorized query engines would remain
the domain of a few large companies and world-class research institutions. 
With DataFusion, we can all build on top of a shared foundation, and focus on
what makes our projects unique.&lt;/p&gt;
&lt;h2 id="community-growth"&gt;Community Growth  📈&lt;a class="headerlink" href="#community-growth" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;In the last 6 months, between &lt;code&gt;40.0.0&lt;/code&gt; and &lt;code&gt;45.0.0&lt;/code&gt;, our community continues to
grow in new and exciting ways.&lt;/p&gt;
&lt;ol&gt;
&lt;li&gt;We added several PMC members and new committers: &lt;a href="https://github.com/jayzhan211"&gt;@jayzhan211&lt;/a&gt; and &lt;a href="https://github.com/jonahgao"&gt;@jonahgao&lt;/a&gt; joined the PMC,
   &lt;a href="https://github.com/2010YOUY01"&gt;@2010YOUY01&lt;/a&gt;, &lt;a href="https://github.com/rachelint"&gt;@rachelint&lt;/a&gt;, &lt;a href="https://github.com/findepi/"&gt;@findpi&lt;/a&gt;, &lt;a href="https://github.com/iffyio"&gt;@iffyio&lt;/a&gt;, &lt;a href="https://github.com/goldmedal"&gt;@goldmedal&lt;/a&gt;, &lt;a href="https://github.com/Weijun-H"&gt;@Weijun-H&lt;/a&gt;, &lt;a href="https://github.com/Michael-J-Ward"&gt;@Michael-J-Ward&lt;/a&gt; and &lt;a href="https://github.com/korowa"&gt;@korowa&lt;/a&gt;
   joined as committers. See the &lt;a href="https://lists.apache.org/list.html?dev@datafusion.apache.org"&gt;mailing list&lt;/a&gt; for more details.&lt;/li&gt;
&lt;li&gt;In the &lt;a href="https://github.com/apache/arrow-datafusion"&gt;core DataFusion repo&lt;/a&gt; alone we reviewed and accepted almost 1600 PRs from 206 different
   committers, created over 1100 issues and closed 751 of them 🚀. All changes are listed in the detailed
   &lt;a href="https://github.com/apache/datafusion/tree/main/dev/changelog"&gt;changelogs&lt;/a&gt;.&lt;/li&gt;
&lt;li&gt;DataFusion focused meetups happened in multiple cities around the world: &lt;a href="https://github.com/apache/datafusion/discussions/10341#discussioncomment-10110273"&gt;Hangzhou&lt;/a&gt;, &lt;a href="https://github.com/apache/datafusion/discussions/11431"&gt;Belgrade&lt;/a&gt;, &lt;a href="https://github.com/apache/datafusion/discussions/11213"&gt;New York&lt;/a&gt;, 
   &lt;a href="https://github.com/apache/datafusion/discussions/10348"&gt;Seattle&lt;/a&gt;, &lt;a href="https://github.com/apache/datafusion/discussions/12894"&gt;Chicago&lt;/a&gt;, &lt;a href="https://github.com/apache/datafusion/discussions/13165"&gt;Boston&lt;/a&gt; and &lt;a href="https://github.com/apache/datafusion/discussions/12988"&gt;Amsterdam&lt;/a&gt; as well as a Rust NYC meetup in NYC focused on DataFusion.&lt;/li&gt;
&lt;/ol&gt;
&lt;!--
$ git log --pretty=oneline 40.0.0..45.0.0 . | wc -l
     1532 (up from 1453)

$ git shortlog -sn 40.0.0..45.0.0 . | wc -l
     206 (up from 182)

https://crates.io/crates/datafusion/45.0.0
DataFusion 45 released Feb 7, 2025

https://crates.io/crates/datafusion/40.0.0
DataFusion 40 released July 12, 2024

Issues created in this time: 375 open, 751 closed (from 321 open, 781 closed)
https://github.com/apache/datafusion/issues?q=is%3Aissue+created%3A2024-07-12..2025-02-07

Issues closed: 956 (up from 911)
https://github.com/apache/arrow-datafusion/issues?q=is%3Aissue+closed%3A2024-07-12..2025-02-07

PRs merged in this time 1597 (up from 1490)
https://github.com/apache/arrow-datafusion/pulls?q=is%3Apr+merged%3A2024-07-12..2025-02-07

--&gt;
&lt;p&gt;DataFusion has put in an application to be part of &lt;a href="https://summerofcode.withgoogle.com/"&gt;Google Summer of Code&lt;/a&gt; with a 
&lt;a href="https://github.com/apache/datafusion/issues/14478"&gt;number of ideas&lt;/a&gt; for projects with mentors already selected. Additionally, &lt;a href="https://github.com/apache/datafusion/issues/14373"&gt;some ideas&lt;/a&gt; on
how to make DataFusion an ideal selection for university database projects such as the 
&lt;a href="https://15445.courses.cs.cmu.edu/spring2025/"&gt;CMU database classes&lt;/a&gt; have been put forward.&lt;/p&gt;
&lt;p&gt;In addition, DataFusion has been appearing publicly more and more, both online and offline. Here are some highlights:&lt;/p&gt;
&lt;ol&gt;
&lt;li&gt;A &lt;a href="https://uwheel.rs/post/datafusion_uwheel/"&gt;demonstration of how uwheel&lt;/a&gt; is integrated into DataFusion&lt;/li&gt;
&lt;li&gt;Integrating StringView into DataFusion - &lt;a href="https://www.influxdata.com/blog/faster-queries-with-stringview-part-one-influxdb/"&gt;part 1&lt;/a&gt; and &lt;a href="https://www.influxdata.com/blog/faster-queries-with-stringview-part-two-influxdb/"&gt;part 2&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;&lt;a href="https://techontherocks.show/3"&gt;Building streams&lt;/a&gt; with DataFusion&lt;/li&gt;
&lt;li&gt;&lt;a href="https://blog.haoxp.xyz/posts/caching-datafusion"&gt;Caching in DataFusion&lt;/a&gt;: Don't read twice&lt;/li&gt;
&lt;li&gt;&lt;a href="https://blog.haoxp.xyz/posts/parquet-to-arrow/"&gt;Parquet pruning in DataFusion&lt;/a&gt;: Read no more than you need&lt;/li&gt;
&lt;li&gt;DataFusion is one of &lt;a href="https://www.crn.com/news/software/2024/the-10-coolest-open-source-software-tools-of-2024?page=3"&gt;The 10 coolest open source software tools&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;&lt;a href="https://www.denormalized.io/blog/building-databases"&gt;Building databases over a weekend&lt;/a&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;h2 id="improved-performance"&gt;Improved Performance 🚀&lt;a class="headerlink" href="#improved-performance" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;DataFusion hit a milestone in its development by becoming &lt;a href="https://datafusion.apache.org/blog/2024/11/18/datafusion-fastest-single-node-parquet-clickbench/"&gt;the fastest single node engine&lt;/a&gt; 
for querying Apache Parquet files in &lt;a href="https://benchmark.clickhouse.com/"&gt;clickbench&lt;/a&gt; benchmark for the 43.0.0 release. A &lt;a href="https://github.com/apache/datafusion/issues/12821"&gt;lot 
of work&lt;/a&gt; went into making this happen! While other engines have subsequently gotten faster,
displacing DataFusion from the top spot, DataFusion still remains near the top and we &lt;a href="https://github.com/apache/datafusion/issues/14586"&gt;are planning
more improvements&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;&lt;img alt="ClickBench performance results over time for DataFusion" class="img-fluid" src="/blog/images/datafusion-45.0.0/performance_over_time.png" width="100%"/&gt;&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Figure 1&lt;/strong&gt;: ClickBench performance improved over 33% between DataFusion 33
(released Nov. 2023) and DataFusion 45 (released Feb. 2025). &lt;/p&gt;
&lt;p&gt;The task of &lt;a href="https://github.com/apache/datafusion/issues/10918"&gt;integrating&lt;/a&gt; the new &lt;a href="https://docs.rs/arrow/latest/arrow/array/struct.GenericByteViewArray.html"&gt;Arrow StringView&lt;/a&gt; which significantly improves performance 
for workloads that scan, filter and group by variable length string and binary data was completed 
and enabled by default in the past 6 months. The improvement is especially pronounced for Parquet 
files due to &lt;a href="https://github.com/apache/arrow-rs/issues/5530"&gt;upstream work in the parquet reader&lt;/a&gt;. Kudos to &lt;a href="https://github.com/XiangpengHong"&gt;@XiangpengHong&lt;/a&gt;, &lt;a href="https://github.com/AriesDevil"&gt;@AriesDevil&lt;/a&gt;, 
&lt;a href="https://github.com/PsiACE"&gt;@PsiACE&lt;/a&gt;, &lt;a href="https://github.com/Weijun-H"&gt;@Weijun-H&lt;/a&gt;, &lt;a href="https://github.com/a10y"&gt;@a10y&lt;/a&gt;, and &lt;a href="https://github.com/RinChanNOWWW"&gt;@RinChanNOWWW&lt;/a&gt; for driving this project.&lt;/p&gt;
&lt;h2 id="improved-quality"&gt;Improved Quality 📋&lt;a class="headerlink" href="#improved-quality" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;DataFusion continues to improve overall in quality. In addition to ongoing bug
fixes, one of the most exciting improvements in the last 6 months was the addition of the 
&lt;a href="https://github.com/apache/datafusion/pull/13936"&gt;SQLite sqllogictest suite&lt;/a&gt; thanks to &lt;a href="https://github.com/Omega359"&gt;@Omega359&lt;/a&gt;. These tests run over 5 million 
sql statements on every push to the main branch.&lt;/p&gt;
&lt;p&gt;Support for &lt;a href="https://github.com/apache/datafusion/pull/13651"&gt;explicitly checking logical plan invariants&lt;/a&gt; was added by &lt;a href="https://github.com/wiedld"&gt;@wiedld&lt;/a&gt; which 
can help catch implicit changes that might cause problems during upgrades.&lt;/p&gt;
&lt;p&gt;We have also started other quality initiatives to make it &lt;a href="https://github.com/apache/datafusion/issues/13525"&gt;easier to use DataFusion&lt;/a&gt; 
based on &lt;a href="https://glaredb.com/"&gt;GlareDB&lt;/a&gt;'s experience along with more &lt;a href="https://github.com/apache/datafusion/issues/13661"&gt;extensive prerelease testing&lt;/a&gt;.  &lt;/p&gt;
&lt;h2 id="improved-documentation"&gt;Improved Documentation 📚&lt;a class="headerlink" href="#improved-documentation" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;We continue to improve the documentation to make it easier to get started using DataFusion. 
During the last 6 months two projects were initiated to migrate the function documentation
from strictly static markdown files. First, &lt;a href="https://github.com/apache/datafusion/pull/12668"&gt;@Omega359&lt;/a&gt; to allow function
documentation to be generated from code and &lt;a href="https://github.com/jonathanc-n"&gt;@jonathanc-n&lt;/a&gt; and others helped with the migration,
then &lt;a href="https://github.com/comphead"&gt;@comphead&lt;/a&gt; lead a project to &lt;a href="https://github.com/apache/datafusion/pull/12822"&gt;create a doc macro&lt;/a&gt; to allow for an even easier way to write 
function documentation. A special thanks to &lt;a href="https://github.com/Chen-Yuan-Lai"&gt;@Chen-Yuan-Lai&lt;/a&gt; for migrating many functions to 
the new syntax.&lt;/p&gt;
&lt;p&gt;Additionally, the &lt;a href="https://github.com/apache/datafusion/pull/13877"&gt;examples&lt;/a&gt; were &lt;a href="https://github.com/apache/datafusion/pull/13905"&gt;refactored&lt;/a&gt; and &lt;a href="https://github.com/apache/datafusion/pull/13950"&gt;cleaned up&lt;/a&gt; to improve their usefulness.&lt;/p&gt;
&lt;h2 id="new-features"&gt;New Features ✨&lt;a class="headerlink" href="#new-features" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;There are too many new features in the last 6 months to list them all, but here
are some highlights:&lt;/p&gt;
&lt;h3 id="functions"&gt;Functions&lt;a class="headerlink" href="#functions" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;ul&gt;
&lt;li&gt;Uniform Window Functions:  &lt;code&gt;BuiltInWindowFunctions&lt;/code&gt; was removed and all now use UDFs (&lt;a href="https://github.com/jcsherin"&gt;@jcsherin&lt;/a&gt;)&lt;/li&gt;
&lt;li&gt;Uniform Aggregate Functions: &lt;code&gt;BuiltInAggregateFunctions&lt;/code&gt; was removed and all now use UDFs&lt;/li&gt;
&lt;li&gt;As mentioned above function documentation was extracted from the markdown files&lt;/li&gt;
&lt;li&gt;Some new functions and sql support were added including '&lt;a href="https://github.com/apache/datafusion/pull/13799"&gt;show functions&lt;/a&gt;', '&lt;a href="https://github.com/apache/datafusion/pull/11347"&gt;to_local_time&lt;/a&gt;',
  '&lt;a href="https://github.com/apache/datafusion/pull/12970"&gt;regexp_count&lt;/a&gt;', '&lt;a href="https://github.com/apache/datafusion/pull/11969"&gt;map_extract&lt;/a&gt;', '&lt;a href="https://github.com/apache/datafusion/pull/12211"&gt;array_distance&lt;/a&gt;', '&lt;a href="https://github.com/apache/datafusion/pull/12329"&gt;array_any_value&lt;/a&gt;', '&lt;a href="https://github.com/apache/datafusion/pull/12474"&gt;greatest&lt;/a&gt;',
  '&lt;a href="https://github.com/apache/datafusion/pull/13786"&gt;least&lt;/a&gt;', '&lt;a href="https://github.com/apache/datafusion/pull/14217"&gt;arrays_overlap&lt;/a&gt;'&lt;/li&gt;
&lt;/ul&gt;
&lt;h3 id="ffi"&gt;FFI&lt;a class="headerlink" href="#ffi" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;ul&gt;
&lt;li&gt;Foreign Function Interface work has started. This should allow for 
  &lt;a href="https://github.com/apache/datafusion/pull/12920"&gt;using table providers&lt;/a&gt; across languages and versions of DataFusion. This 
  is especially pertinent for integration with &lt;a href="https://delta-io.github.io/delta-rs/"&gt;delta-rs&lt;/a&gt; and other table formats.&lt;/li&gt;
&lt;/ul&gt;
&lt;h3 id="materialized-views"&gt;Materialized Views&lt;a class="headerlink" href="#materialized-views" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;ul&gt;
&lt;li&gt;&lt;a href="https://github.com/suremarc"&gt;@suremarc&lt;/a&gt; has added a &lt;a href="https://github.com/datafusion-contrib/datafusion-materialized-views"&gt;materialized view implementation&lt;/a&gt; in datafusion-contrib 🚀&lt;/li&gt;
&lt;/ul&gt;
&lt;h3 id="substrait"&gt;Substrait&lt;a class="headerlink" href="#substrait" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;ul&gt;
&lt;li&gt;A lot of work was put into improving and enhancing substrait support (&lt;a href="https://github.com/Blizzara"&gt;@Blizzara&lt;/a&gt;, &lt;a href="https://github.com/westonpace"&gt;@westonpace&lt;/a&gt;, &lt;a href="https://github.com/tokoko"&gt;@tokoko&lt;/a&gt;, &lt;a href="https://github.com/vbarua"&gt;@vbarua&lt;/a&gt;, &lt;a href="https://github.com/LatrecheYasser"&gt;@LatrecheYasser&lt;/a&gt;, &lt;a href="https://github.com/notfilippo"&gt;@notfilippo&lt;/a&gt; and others)&lt;/li&gt;
&lt;/ul&gt;
&lt;h2 id="looking-ahead-the-next-six-months"&gt;Looking Ahead: The Next Six Months 🔭&lt;a class="headerlink" href="#looking-ahead-the-next-six-months" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;One of the long term goals of &lt;a href="https://github.com/alamb"&gt;@alamb&lt;/a&gt;, DataFusion's PMC chair, has been to have 
&lt;a href="https://www.influxdata.com/blog/datafusion-2025-influxdb/"&gt;1000 DataFusion based projects&lt;/a&gt;. This may be the year that happens!&lt;/p&gt;
&lt;p&gt;The community has been &lt;a href="https://github.com/apache/datafusion/issues/14580"&gt;discussing what we will work on in the next six months&lt;/a&gt;.
Some major initiatives are likely to be:&lt;/p&gt;
&lt;ol&gt;
&lt;li&gt;&lt;em&gt;Performance&lt;/em&gt;: A &lt;a href="https://github.com/apache/datafusion/issues/14482"&gt;number of items have been identified&lt;/a&gt; as areas that could use additional work&lt;/li&gt;
&lt;li&gt;&lt;em&gt;Memory usage&lt;/em&gt;: Tracking and improving memory usage, statistics and spilling to disk &lt;/li&gt;
&lt;li&gt;&lt;em&gt;&lt;a href="https://summerofcode.withgoogle.com/"&gt;Google Summer of Code&lt;/a&gt; (GSOC)&lt;/em&gt;: DataFusion is hopefully selected as a project and we start accepting and supporting student projects &lt;/li&gt;
&lt;li&gt;&lt;em&gt;FFI&lt;/em&gt;: Extending the FFI implementation to support to all types of UDF's and SessionContext&lt;/li&gt;
&lt;li&gt;&lt;em&gt;Spark Functions&lt;/em&gt;: A &lt;a href="https://github.com/apache/datafusion/issues/5600"&gt;proposal has been made to add a crate&lt;/a&gt; covering spark compatible builtin functions &lt;/li&gt;
&lt;/ol&gt;
&lt;h2 id="how-to-get-involved"&gt;How to Get Involved&lt;a class="headerlink" href="#how-to-get-involved" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;DataFusion is not a project built or driven by a single person, company, or
foundation. Rather, our community of users and contributors work together to
build a shared technology that none of us could have built alone.&lt;/p&gt;
&lt;p&gt;If you are interested in joining us we would love to have you. You can try out
DataFusion on some of your own data and projects and let us know how it goes,
contribute suggestions, documentation, bug reports, or a PR with documentation,
tests or code. A list of open issues suitable for beginners is &lt;a href="https://github.com/apache/arrow-datafusion/issues?q=is%3Aissue+is%3Aopen+label%3A%22good+first+issue%22"&gt;here&lt;/a&gt; and you
can find how to reach us on the &lt;a href="https://datafusion.apache.org/contributor-guide/communication.html"&gt;communication doc&lt;/a&gt;.&lt;/p&gt;</content><category term="blog"/></entry><entry><title>Apache DataFusion Comet 0.6.0 Release</title><link href="https://datafusion.apache.org/blog/2025/02/17/datafusion-comet-0.6.0" rel="alternate"/><published>2025-02-17T00:00:00+00:00</published><updated>2025-02-17T00:00:00+00:00</updated><author><name>pmc</name></author><id>tag:datafusion.apache.org,2025-02-17:/blog/2025/02/17/datafusion-comet-0.6.0</id><summary type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;p&gt;The Apache DataFusion PMC is pleased to announce version 0.6.0 of the &lt;a href="https://datafusion.apache.org/comet/"&gt;Comet&lt;/a&gt; subproject.&lt;/p&gt;
&lt;p&gt;Comet is an accelerator for Apache Spark that translates Spark physical plans to DataFusion physical plans for
improved performance and efficiency without requiring any code changes.&lt;/p&gt;
&lt;p&gt;Comet runs on commodity hardware and aims to …&lt;/p&gt;</summary><content type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;p&gt;The Apache DataFusion PMC is pleased to announce version 0.6.0 of the &lt;a href="https://datafusion.apache.org/comet/"&gt;Comet&lt;/a&gt; subproject.&lt;/p&gt;
&lt;p&gt;Comet is an accelerator for Apache Spark that translates Spark physical plans to DataFusion physical plans for
improved performance and efficiency without requiring any code changes.&lt;/p&gt;
&lt;p&gt;Comet runs on commodity hardware and aims to provide 100% compatibility with Apache Spark. Any operators or
expressions that are not fully compatible will fall back to Spark unless explicitly enabled by the user. Refer
to the &lt;a href="https://datafusion.apache.org/comet/user-guide/compatibility.html"&gt;compatibility guide&lt;/a&gt; for more information.&lt;/p&gt;
&lt;p&gt;This release covers approximately four weeks of development work and is the result of merging 39 PRs from 12
contributors. See the &lt;a href="https://github.com/apache/datafusion-comet/blob/main/dev/changelog/0.6.0.md"&gt;change log&lt;/a&gt; for more information.&lt;/p&gt;
&lt;p&gt;Starting with this release, we now plan on releasing new versions of Comet more frequently, typically within 1-2 weeks
of each major DataFusion release. The main motivation for this change is to better support downstream Rust projects 
that depend on the &lt;a href="https://docs.rs/datafusion-comet-spark-expr/latest/datafusion_comet_spark_expr/"&gt;datafusion_comet_spark_expr&lt;/a&gt; crate.&lt;/p&gt;
&lt;h2 id="release-highlights"&gt;Release Highlights&lt;a class="headerlink" href="#release-highlights" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;h3 id="datafusion-upgrade"&gt;DataFusion Upgrade&lt;a class="headerlink" href="#datafusion-upgrade" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;ul&gt;
&lt;li&gt;Comet 0.6.0 uses DataFusion 45.0.0&lt;/li&gt;
&lt;/ul&gt;
&lt;h3 id="new-features"&gt;New Features&lt;a class="headerlink" href="#new-features" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;ul&gt;
&lt;li&gt;Comet now supports &lt;code&gt;array_join&lt;/code&gt;, &lt;code&gt;array_intersect&lt;/code&gt;, and &lt;code&gt;arrays_overlap&lt;/code&gt;. Note that these expressions are not 
  yet guaranteed to be 100% compatible with Spark for all input data types, so these expressions are only enabled 
  with the configuration setting &lt;code&gt;spark.comet.expression.allowIncompatible=true&lt;/code&gt;. &lt;/li&gt;
&lt;/ul&gt;
&lt;h3 id="performance-stability"&gt;Performance &amp;amp; Stability&lt;a class="headerlink" href="#performance-stability" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;ul&gt;
&lt;li&gt;Metrics from native execution are now updated in Spark every 3 seconds by default, rather than for each
  batch being processed. The mechanism for passing the metrics via JNI is also more efficient.&lt;/li&gt;
&lt;li&gt;New memory pool options "fair unified" and "unbounded" have been added. See the &lt;a href="https://datafusion.apache.org/comet/user-guide/tuning.html"&gt;Comet Tuning Guide&lt;/a&gt; for more information.&lt;/li&gt;
&lt;/ul&gt;
&lt;h2 id="bug-fixes"&gt;Bug Fixes&lt;a class="headerlink" href="#bug-fixes" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;ul&gt;
&lt;li&gt;Hashing of decimal values with precision &amp;lt;= 18 is now compatible with Spark&lt;/li&gt;
&lt;li&gt;Comet falls back to Spark when hashing decimals with precision &amp;gt; 18&lt;/li&gt;
&lt;/ul&gt;
&lt;h2 id="getting-involved"&gt;Getting Involved&lt;a class="headerlink" href="#getting-involved" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;The Comet project welcomes new contributors. We use the same &lt;a href="https://datafusion.apache.org/contributor-guide/communication.html#slack-and-discord"&gt;Slack and Discord&lt;/a&gt; channels as the main DataFusion
project and have a weekly &lt;a href="https://docs.google.com/document/d/1NBpkIAuU7O9h8Br5CbFksDhX-L9TyO9wmGLPMe0Plc8/edit?usp=sharing"&gt;DataFusion video call&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;The easiest way to get involved is to test Comet with your current Spark jobs and file issues for any bugs or
performance regressions that you find. See the &lt;a href="https://datafusion.apache.org/comet/user-guide/installation.html"&gt;Getting Started&lt;/a&gt; guide for instructions on downloading and installing
Comet.&lt;/p&gt;
&lt;p&gt;There are also many &lt;a href="https://github.com/apache/datafusion-comet/contribute"&gt;good first issues&lt;/a&gt; waiting for contributions.&lt;/p&gt;</content><category term="blog"/></entry><entry><title>Apache DataFusion Ballista 43.0.0 Released</title><link href="https://datafusion.apache.org/blog/2025/02/02/datafusion-ballista-43.0.0" rel="alternate"/><published>2025-02-02T00:00:00+00:00</published><updated>2025-02-02T00:00:00+00:00</updated><author><name>milenkovicm</name></author><id>tag:datafusion.apache.org,2025-02-02:/blog/2025/02/02/datafusion-ballista-43.0.0</id><summary type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;p&gt;We are  pleased to announce version &lt;a href="https://github.com/apache/datafusion-ballista/blob/main/CHANGELOG.md#4300-2025-01-07"&gt;43.0.0&lt;/a&gt; of the &lt;a href="https://datafusion.apache.org/ballista/"&gt;DataFusion Ballista&lt;/a&gt;. Ballista allows existing &lt;a href="https://datafusion.apache.org"&gt;DataFusion&lt;/a&gt; applications to be scaled out on a cluster for use cases that are not practical to run on a single node.&lt;/p&gt;
&lt;h2 id="highlights-of-this-release"&gt;Highlights of this release&lt;a class="headerlink" href="#highlights-of-this-release" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;h3 id="seamless-integration-with-datafusion"&gt;Seamless Integration with DataFusion&lt;a class="headerlink" href="#seamless-integration-with-datafusion" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;The primary objective of …&lt;/p&gt;</summary><content type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;p&gt;We are  pleased to announce version &lt;a href="https://github.com/apache/datafusion-ballista/blob/main/CHANGELOG.md#4300-2025-01-07"&gt;43.0.0&lt;/a&gt; of the &lt;a href="https://datafusion.apache.org/ballista/"&gt;DataFusion Ballista&lt;/a&gt;. Ballista allows existing &lt;a href="https://datafusion.apache.org"&gt;DataFusion&lt;/a&gt; applications to be scaled out on a cluster for use cases that are not practical to run on a single node.&lt;/p&gt;
&lt;h2 id="highlights-of-this-release"&gt;Highlights of this release&lt;a class="headerlink" href="#highlights-of-this-release" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;h3 id="seamless-integration-with-datafusion"&gt;Seamless Integration with DataFusion&lt;a class="headerlink" href="#seamless-integration-with-datafusion" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;The primary objective of this release has been to achieve a more seamless integration with the DataFusion ecosystem and try to achieve the same level of flexibility as DataFusion.&lt;/p&gt;
&lt;p&gt;In recent months, our development efforts have been directed toward providing a robust and extensible Ballista API. This new API empowers end-users to tailor Ballista's core functionality to their specific use cases. As a result, we have deprecated several experimental features from the Ballista core, allowing users to reintroduce them as custom extensions outside the core framework. This shift reduces the maintenance burden on Ballista's core maintainers and paves the way for optional features, such as &lt;a href="https://github.com/delta-io/delta-rs"&gt;delta-rs&lt;/a&gt; support, to be added externally when needed.&lt;/p&gt;
&lt;p&gt;The most significant enhancement in this release is the deprecation of &lt;code&gt;BallistaContext&lt;/code&gt;, which has been superseded by the DataFusion &lt;code&gt;SessionContext&lt;/code&gt;. This change enables DataFusion applications written in Rust to execute on a Ballista cluster with minimal modifications. Beyond simplifying migration and reducing maintenance overhead, this update introduces distributed write functionality to Ballista for the first time, significantly enhancing its capabilities.&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-rust"&gt;use ballista::prelude::*;
use datafusion::prelude::*;

#[tokio::main]
async fn main() -&amp;gt; datafusion::error::Result&amp;lt;()&amp;gt; {

  // Instead of creating classic SessionContext
  // let ctx = SessionContext::new();

  // create DataFusion SessionContext with ballista standalone cluster started
  // let ctx = SessionContext::standalone().await;

  // create DataFusion SessionContext with ballista remote cluster started
  let ctx = SessionContext::remote("df://localhost:50050").await;

  // register the table
  ctx.register_csv("example", "tests/data/example.csv", CsvReadOptions::new()).await?;

  // create a plan to run a SQL query
  let df = ctx.sql("SELECT a, MIN(b) FROM example WHERE a &amp;lt;= b GROUP BY a LIMIT 100").await?;

  // execute and print results
  df.show().await?;
  Ok(())
}
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;Additionally, Ballista’s versioning scheme has been aligned with that of DataFusion, ensuring that Ballista's version number reflects the compatible DataFusion version.&lt;/p&gt;
&lt;p&gt;At the moment there is a gap between DataFusion and Ballista, which we will try to bridge in the future.&lt;/p&gt;
&lt;h3 id="removal-of-experimental-features"&gt;Removal of Experimental Features&lt;a class="headerlink" href="#removal-of-experimental-features" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;Ballista had grown in scope to include several experimental features in various states of completeness. Some features have been removed from this release in an effort to strip Ballista back to its core and make it easier to maintain and extend.&lt;/p&gt;
&lt;p&gt;Specifically, the caching subsystem, predefined object store registry, plugin subsystem, key-value stores for persistent scheduler state, and the UI have been removed.&lt;/p&gt;
&lt;h3 id="performance-scalability"&gt;Performance &amp;amp; Scalability&lt;a class="headerlink" href="#performance-scalability" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;Ballista has significantly leveraged the advancements made in the DataFusion project over the past year. Benchmark results demonstrate notable improvements in performance, highlighting the impact of these enhancements:&lt;/p&gt;
&lt;p&gt;Per query comparison:&lt;/p&gt;
&lt;p&gt;&lt;img alt="Per query comparison" class="img-fluid" src="/blog/images/datafusion-ballista-43.0.0/tpch_queries_compare.png" width="100%"/&gt;&lt;/p&gt;
&lt;p&gt;Relative speedup:&lt;/p&gt;
&lt;p&gt;&lt;img alt="Relative speedup graph" class="img-fluid" src="/blog/images/datafusion-ballista-43.0.0/tpch_queries_speedup_rel.png" width="100%"/&gt;&lt;/p&gt;
&lt;p&gt;The overall speedup is 2.9x&lt;/p&gt;
&lt;p&gt;&lt;img alt="Overall speedup" class="img-fluid" src="/blog/images/datafusion-ballista-43.0.0/tpch_allqueries.png" width="50%"/&gt;&lt;/p&gt;
&lt;h3 id="new-logo"&gt;New Logo&lt;a class="headerlink" href="#new-logo" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;Ballista now has a new logo, which is visually similar to other DataFusion projects.  &lt;/p&gt;
&lt;p&gt;&lt;img alt="New logo" class="img-fluid" src="/blog/images/datafusion-ballista-43.0.0/ballista-logo.png" width="50%"/&gt;&lt;/p&gt;
&lt;h2 id="roadmap"&gt;Roadmap&lt;a class="headerlink" href="#roadmap" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;Moving forward, Ballista will adopt the same release cadence as DataFusion, providing synchronized updates across the ecosystem.
Currently, there is no established long-term roadmap for Ballista. A plan will be formulated in the coming months based on community feedback and the availability of additional maintainers.&lt;/p&gt;
&lt;p&gt;In the short term, development efforts will concentrate on closing the feature gap between DataFusion and Ballista. Key priorities include implementing support for &lt;code&gt;INSERT INTO&lt;/code&gt;, enabling table &lt;code&gt;URL&lt;/code&gt; functionality, and achieving deeper integration with the Python ecosystem.&lt;/p&gt;</content><category term="blog"/></entry><entry><title>Apache DataFusion Comet 0.5.0 Release</title><link href="https://datafusion.apache.org/blog/2025/01/17/datafusion-comet-0.5.0" rel="alternate"/><published>2025-01-17T00:00:00+00:00</published><updated>2025-01-17T00:00:00+00:00</updated><author><name>pmc</name></author><id>tag:datafusion.apache.org,2025-01-17:/blog/2025/01/17/datafusion-comet-0.5.0</id><summary type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;p&gt;The Apache DataFusion PMC is pleased to announce version 0.5.0 of the &lt;a href="https://datafusion.apache.org/comet/"&gt;Comet&lt;/a&gt; subproject.&lt;/p&gt;
&lt;p&gt;Comet is an accelerator for Apache Spark that translates Spark physical plans to DataFusion physical plans for
improved performance and efficiency without requiring any code changes.&lt;/p&gt;
&lt;p&gt;Comet runs on commodity hardware and aims to …&lt;/p&gt;</summary><content type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;p&gt;The Apache DataFusion PMC is pleased to announce version 0.5.0 of the &lt;a href="https://datafusion.apache.org/comet/"&gt;Comet&lt;/a&gt; subproject.&lt;/p&gt;
&lt;p&gt;Comet is an accelerator for Apache Spark that translates Spark physical plans to DataFusion physical plans for
improved performance and efficiency without requiring any code changes.&lt;/p&gt;
&lt;p&gt;Comet runs on commodity hardware and aims to provide 100% compatibility with Apache Spark. Any operators or
expressions that are not fully compatible will fall back to Spark unless explicitly enabled by the user. Refer
to the &lt;a href="https://datafusion.apache.org/comet/user-guide/compatibility.html"&gt;compatibility guide&lt;/a&gt; for more information.&lt;/p&gt;
&lt;p&gt;This release covers approximately 8 weeks of development work and is the result of merging 69 PRs from 15
contributors. See the &lt;a href="https://github.com/apache/datafusion-comet/blob/main/dev/changelog/0.5.0.md"&gt;change log&lt;/a&gt; for more information.&lt;/p&gt;
&lt;h2 id="release-highlights"&gt;Release Highlights&lt;a class="headerlink" href="#release-highlights" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;h3 id="performance"&gt;Performance&lt;a class="headerlink" href="#performance" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;Comet 0.5.0 achieves a 1.9x speedup for single-node TPC-H @ 100 GB, an improvement from 1.7x in the previous release.&lt;/p&gt;
&lt;p&gt;&lt;img alt="Chart showing TPC-H benchmark results for Comet 0.5.0" class="img-fluid" src="/blog/images/comet-0.5.0/tpch_allqueries.png" width="100%"/&gt;&lt;/p&gt;
&lt;p&gt;&lt;img alt="Chart showing TPC-H benchmark results for Comet 0.5.0" class="img-fluid" src="/blog/images/comet-0.5.0/tpch_queries_compare.png" width="100%"/&gt;&lt;/p&gt;
&lt;p&gt;More benchmarking results can be found in the &lt;a href="https://datafusion.apache.org/comet/contributor-guide/benchmarking.html"&gt;Comet Benchmarking Guide&lt;/a&gt;.&lt;/p&gt;
&lt;h3 id="shuffle-improvements"&gt;Shuffle Improvements&lt;a class="headerlink" href="#shuffle-improvements" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;Comet now supports multiple compression algorithms for compressing shuffle files. Previously, only ZSTD was supported
but Comet now also supports LZ4 and Snappy. The default is now LZ4, which matches the default in Spark. ZSTD may be
a better choice when the compression ratio is more important than CPU overhead.&lt;/p&gt;
&lt;p&gt;Previously, Comet used Arrow IPC to encode record batches into shuffle files. Although Arrow IPC is a good
general-purpose framework for serializing Arrow record batches, we found that we could get better performance using
a custom serialization approach optimized for Comet. One optimization is that the schema is encoded once per shuffle
operation rather than once per batch. There are some planned performance improvements in the Rust implementation of
Arrow IPC and Comet may switch back to Arrow IPC in the future.&lt;/p&gt;
&lt;p&gt;Comet provides two shuffle implementations. Comet native shuffle is the fastest and performs repartitioning in
native code. Comet columnar shuffle delegates to Spark to perform repartitioning and is used in cases where native
shuffle is not supported, such as with &lt;code&gt;RangePartitioning&lt;/code&gt;. Comet generally tries to use native shuffle first, then
columnar shuffle, and finally falls back to Spark if neither is supported. There was a bug in previous releases
where Comet would sometimes fall back to Spark shuffle if native shuffle was not supported and missed opportunities
to use columnar shuffle. This bug was fixed in this release but currently requires the configuration setting
&lt;code&gt;spark.comet.exec.shuffle.fallbackToColumnar=true&lt;/code&gt;. This will be enabled by default in the next release.&lt;/p&gt;
&lt;h3 id="memory-management"&gt;Memory Management&lt;a class="headerlink" href="#memory-management" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;Comet 0.4.0 required Spark to be configured to use off-heap memory. In this release it is no longer required and
there are multiple options for configuring Comet to use on-heap memory instead. More details are available in the
&lt;a href="https://datafusion.apache.org/comet/user-guide/tuning.html"&gt;Comet Tuning Guide&lt;/a&gt;.&lt;/p&gt;
&lt;h3 id="spark-sql-metrics"&gt;Spark SQL Metrics&lt;a class="headerlink" href="#spark-sql-metrics" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;Comet now provides detailed metrics for native shuffle, showing time for repartitioning, encoding and compressing,
and writing to disk.&lt;/p&gt;
&lt;h3 id="crate-reorganization"&gt;Crate Reorganization&lt;a class="headerlink" href="#crate-reorganization" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;One of the goals of the Comet project is to make Spark-compatible functionality available to other projects that
are based on DataFusion. In this release, many implementations of Spark-compatible expressions were moved from the
unpublished &lt;code&gt;datafusion-comet&lt;/code&gt; crate, which provides the native part of the Spark plugin, into the
&lt;code&gt;datafusion-comet-spark-expr&lt;/code&gt; crate. There is also ongoing work to reorganize this crate to move expressions into
subfolders named after the group name that Spark uses to organize expressions. For example, there are now subfolders
named &lt;code&gt;agg_funcs&lt;/code&gt;, &lt;code&gt;datetime_funcs&lt;/code&gt;, &lt;code&gt;hash_funcs&lt;/code&gt;, and so on.&lt;/p&gt;
&lt;h2 id="update-on-complex-type-support"&gt;Update on Complex Type Support&lt;a class="headerlink" href="#update-on-complex-type-support" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;Good progress has been made with proof-of-concept work using DataFusion’s &lt;code&gt;ParquetExec&lt;/code&gt;, which has the advantage of
supporting complex types. This work is available on the &lt;code&gt;comet-parquet-exec&lt;/code&gt; branch, and the current focus is on
fixing test regressions, particularly regarding timestamp conversion issues.&lt;/p&gt;
&lt;h2 id="getting-involved"&gt;Getting Involved&lt;a class="headerlink" href="#getting-involved" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;The Comet project welcomes new contributors. We use the same &lt;a href="https://datafusion.apache.org/contributor-guide/communication.html#slack-and-discord"&gt;Slack and Discord&lt;/a&gt; channels as the main DataFusion
project and have a weekly &lt;a href="https://docs.google.com/document/d/1NBpkIAuU7O9h8Br5CbFksDhX-L9TyO9wmGLPMe0Plc8/edit?usp=sharing"&gt;DataFusion video call&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;The easiest way to get involved is to test Comet with your current Spark jobs and file issues for any bugs or
performance regressions that you find. See the &lt;a href="https://datafusion.apache.org/comet/user-guide/installation.html"&gt;Getting Started&lt;/a&gt; guide for instructions on downloading and installing
Comet.&lt;/p&gt;
&lt;p&gt;There are also many &lt;a href="https://github.com/apache/datafusion-comet/contribute"&gt;good first issues&lt;/a&gt; waiting for contributions.&lt;/p&gt;</content><category term="blog"/></entry><entry><title>Apache DataFusion Python 43.1.0 Released</title><link href="https://datafusion.apache.org/blog/2024/12/14/datafusion-python-43.1.0" rel="alternate"/><published>2024-12-14T00:00:00+00:00</published><updated>2024-12-14T00:00:00+00:00</updated><author><name>timsaucer</name></author><id>tag:datafusion.apache.org,2024-12-14:/blog/2024/12/14/datafusion-python-43.1.0</id><summary type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;p&gt;We are happy to announce that &lt;a href="https://pypi.org/project/datafusion/43.1.0/"&gt;datafusion-python 43.1.0&lt;/a&gt; has been released. This release
brings in all of the new features of the core &lt;a href="https://github.com/apache/datafusion/blob/main/dev/changelog/43.0.0.md"&gt;DataFusion 43.0.0&lt;/a&gt; library. Since the last
blog post for &lt;a href="https://datafusion.apache.org/blog/2024/08/20/python-datafusion-40.0.0/"&gt;datafusion-python 40.1.0&lt;/a&gt;, a large number of improvements have been made
that can …&lt;/p&gt;</summary><content type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;p&gt;We are happy to announce that &lt;a href="https://pypi.org/project/datafusion/43.1.0/"&gt;datafusion-python 43.1.0&lt;/a&gt; has been released. This release
brings in all of the new features of the core &lt;a href="https://github.com/apache/datafusion/blob/main/dev/changelog/43.0.0.md"&gt;DataFusion 43.0.0&lt;/a&gt; library. Since the last
blog post for &lt;a href="https://datafusion.apache.org/blog/2024/08/20/python-datafusion-40.0.0/"&gt;datafusion-python 40.1.0&lt;/a&gt;, a large number of improvements have been made
that can be found in the &lt;a href="https://github.com/apache/datafusion-python/tree/main/dev/changelog"&gt;changelogs&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;We would like to point out four features that are particularly noteworthy.&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;Arrow PyCapsule import and export&lt;/li&gt;
&lt;li&gt;User-Defined Window Functions&lt;/li&gt;
&lt;li&gt;Foreign Table Providers&lt;/li&gt;
&lt;li&gt;String View performance enhancements&lt;/li&gt;
&lt;/ul&gt;
&lt;h2 id="arrow-pycapsule-import-and-export"&gt;Arrow PyCapsule import and export&lt;a class="headerlink" href="#arrow-pycapsule-import-and-export" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;Arrow has stable C interface for moving data between different libraries, but difficulties
sometimes arise when different Python libraries expose this interface through different
methods, requiring developers to write function calls for each library they are attempting
to work with. A better approach is to use the &lt;a href="https://arrow.apache.org/docs/format/CDataInterface/PyCapsuleInterface.html"&gt;Arrow PyCapsule Interface&lt;/a&gt; which gives a
consistent method for exposing these data structures across libraries.&lt;/p&gt;
&lt;p&gt;In &lt;a href="https://github.com/apache/datafusion-python/pull/825"&gt;PR #825&lt;/a&gt;, we introduced support for both importing and exporting Arrow data in
&lt;code&gt;datafusion-python&lt;/code&gt;. With this improvement, you can now use a single function call to import
a table from &lt;strong&gt;any&lt;/strong&gt; Python library that implements the &lt;a href="https://arrow.apache.org/docs/format/CDataInterface/PyCapsuleInterface.html"&gt;Arrow PyCapsule Interface&lt;/a&gt;.
Many popular libraries, such as &lt;a href="https://pandas.pydata.org/"&gt;Pandas&lt;/a&gt; and &lt;a href="https://pola.rs/"&gt;Polars&lt;/a&gt;
already support these interfaces.&lt;/p&gt;
&lt;p&gt;Suppose you have a Pandas and Polars DataFrames named &lt;code&gt;df_pandas&lt;/code&gt; or &lt;code&gt;df_polars&lt;/code&gt;, respectively:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-python"&gt;ctx = SessionContext()
df_dfn1 = ctx.from_arrow(df_pandas)
df_dfn1.show()

df_dfn2 = ctx.from_arrow(df_polars)
df_dfn2.show()
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;One great thing about using this interface is that as any new library is developed and
uses these stable interfaces, they will work out of the box with DataFusion!&lt;/p&gt;
&lt;p&gt;Additionally, DataFusion DataFrames allow for exporting via the PyCapsule interface. For example,
to convert a DataFrame to a PyArrow table, it is simply&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-python"&gt;import pyarrow as pa
table = pa.table(df)
&lt;/code&gt;&lt;/pre&gt;
&lt;h2 id="user-defined-window-functions"&gt;User-Defined Window Functions&lt;a class="headerlink" href="#user-defined-window-functions" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;In &lt;code&gt;datafusion-python 42.0.0&lt;/code&gt; we released User-Defined Window Support in &lt;a href="https://github.com/apache/datafusion-python/pull/880"&gt;PR #880&lt;/a&gt;.
For a detailed description of how these work please see the online documentation for
all &lt;a href="https://datafusion.apache.org/python/user-guide/common-operations/udf-and-udfa.html"&gt;user-defined functions&lt;/a&gt;. Additionally the &lt;a href="https://github.com/apache/datafusion-python/tree/main/examples"&gt;examples folder&lt;/a&gt; contains a complete
example demonstrating the four different modes of operation of window functions
within DataFusion.&lt;/p&gt;
&lt;h2 id="foreign-table-providers"&gt;Foreign Table Providers&lt;a class="headerlink" href="#foreign-table-providers" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;In the core &lt;a href="https://github.com/apache/datafusion/blob/main/dev/changelog/43.0.0.md"&gt;DataFusion 43.0.0&lt;/a&gt; release, support was added for a Foreign Function
Interface to table providers. This creates a stable way for sharing functionality
across different libraries, similar to the &lt;a href="https://arrow.apache.org/docs/format/CDataInterface.html"&gt;Arrow C data interface&lt;/a&gt; operates. This
enables libraries, such as &lt;a href="https://delta.io/docs/"&gt;delta lake&lt;/a&gt; and &lt;a href="https://github.com/datafusion-contrib/datafusion-table-providers"&gt;datafusion-contrib&lt;/a&gt; to write their own
table providers in Rust and expose them in Python without requiring a Rust dependency
on &lt;code&gt;datafusion-python&lt;/code&gt;. This is important because it allows these libraries to
operate with &lt;code&gt;datafusion-python&lt;/code&gt; regardless of which version of &lt;code&gt;datafusion&lt;/code&gt; they
were built against.&lt;/p&gt;
&lt;p&gt;To implement this feature in a table provider is quite simple. There is a complete
example in the &lt;a href="https://github.com/apache/datafusion-python/tree/main/examples"&gt;examples folder&lt;/a&gt;, but the relevant code is here, exposed as a
Python function via &lt;a href="https://pyo3.rs/"&gt;pyo3&lt;/a&gt;:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-rust"&gt;    fn __datafusion_table_provider__&amp;lt;'py&amp;gt;(
        &amp;amp;self,
        py: Python&amp;lt;'py&amp;gt;,
    ) -&amp;gt; PyResult&amp;lt;Bound&amp;lt;'py, PyCapsule&amp;gt;&amp;gt; {
        let name = CString::new("datafusion_table_provider").unwrap();

        let provider = self
            .create_table()
            .map_err(|e| PyRuntimeError::new_err(e.to_string()))?;
        let provider = FFI_TableProvider::new(Arc::new(provider), false);

        PyCapsule::new_bound(py, provider, Some(name.clone()))
    }
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;That's it! All of the work of converting the table provider to use the FFI interface
is performed by the core library.&lt;/p&gt;
&lt;h2 id="string-view-performance-enhancements"&gt;String View performance enhancements&lt;a class="headerlink" href="#string-view-performance-enhancements" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;In the core &lt;a href="https://github.com/apache/datafusion/blob/main/dev/changelog/43.0.0.md"&gt;DataFusion 43.0.0&lt;/a&gt; release, the option to enable StringView by default
was turned on. This leads to some significant performance enhancements, but it &lt;em&gt;may&lt;/em&gt;
require some changes to users of &lt;code&gt;datafusion-python&lt;/code&gt;.&lt;/p&gt;
&lt;p&gt;To learn more about the excellent work on this feature please read &lt;a href="https://datafusion.apache.org/blog/2024/09/13/string-view-german-style-strings-part-1/"&gt;part 1&lt;/a&gt; and &lt;a href="https://datafusion.apache.org/blog/2024/09/13/string-view-german-style-strings-part-2/"&gt;part 2&lt;/a&gt;
of the blog post describing how these enhancements can lead to 20-200% performance
gains in some tests.&lt;/p&gt;
&lt;p&gt;During our testing we identified some cases where we needed to adjust workflows to
account for the fact that StringView is now the default type for string based operations.
First, when performing manipulations on string objects there is a performance loss when
needing to cast from string to string view or vice versa. To reap the best performance,
ideally all of your string type data will use StringView. For most users this should be
transparent. However if you specify a schema for reading or creating data, then you
likely need to change from &lt;code&gt;pa.string()&lt;/code&gt; to &lt;code&gt;pa.string_view()&lt;/code&gt;. For our testing, this
primarily happens during data loading operations and in unit tests.&lt;/p&gt;
&lt;p&gt;If you wish to disable StringView as the default type to retain the old approach,
you can do so following this example:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-python"&gt;from datafusion import SessionContext
from datafusion import SessionConfig
config = SessionConfig({"datafusion.execution.parquet.schema_force_view_types": "false"})
ctx = SessionContext(config=config)
&lt;/code&gt;&lt;/pre&gt;
&lt;h2 id="appreciation"&gt;Appreciation&lt;a class="headerlink" href="#appreciation" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;We would like to thank everyone who has helped with these releases through their helpful
conversations, code review, issue descriptions, and code authoring. We would especially
like to thank the following authors of PRs who made these releases possible, listed in
alphabetical order by username: &lt;a href="https://github.com/andygrove"&gt;@andygrove&lt;/a&gt;, &lt;a href="https://github.com/drauschenbach"&gt;@drauschenbach&lt;/a&gt;, &lt;a href="https://github.com/emgeee"&gt;@emgeee&lt;/a&gt;, &lt;a href="https://github.com/ion-elgreco"&gt;@ion-elgreco&lt;/a&gt;,
&lt;a href="https://github.com/jcrist"&gt;@jcrist&lt;/a&gt;, &lt;a href="https://github.com/kosiew"&gt;@kosiew&lt;/a&gt;, &lt;a href="https://github.com/mesejo"&gt;@mesejo&lt;/a&gt;, &lt;a href="https://github.com/Michael-J-Ward"&gt;@Michael-J-Ward&lt;/a&gt;, and &lt;a href="https://github.com/sir-sigurd"&gt;@sir-sigurd&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;Thank you!&lt;/p&gt;
&lt;h2 id="get-involved"&gt;Get Involved&lt;a class="headerlink" href="#get-involved" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;The DataFusion Python team is an active and engaging community and we would love
to have you join us and help the project.&lt;/p&gt;
&lt;p&gt;Here are some ways to get involved:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;
&lt;p&gt;Learn more by visiting the &lt;a href="https://datafusion.apache.org/python/index.html"&gt;DataFusion Python project&lt;/a&gt;
page.&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;
&lt;p&gt;Try out the project and provide feedback, file issues, and contribute code.&lt;/p&gt;
&lt;/li&gt;
&lt;/ul&gt;</content><category term="blog"/></entry><entry><title>Apache DataFusion Comet 0.4.0 Release</title><link href="https://datafusion.apache.org/blog/2024/11/20/datafusion-comet-0.4.0" rel="alternate"/><published>2024-11-20T00:00:00+00:00</published><updated>2024-11-20T00:00:00+00:00</updated><author><name>pmc</name></author><id>tag:datafusion.apache.org,2024-11-20:/blog/2024/11/20/datafusion-comet-0.4.0</id><summary type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;p&gt;The Apache DataFusion PMC is pleased to announce version 0.4.0 of the &lt;a href="https://datafusion.apache.org/comet/"&gt;Comet&lt;/a&gt; subproject.&lt;/p&gt;
&lt;p&gt;Comet is an accelerator for Apache Spark that translates Spark physical plans to DataFusion physical plans for
improved performance and efficiency without requiring any code changes.&lt;/p&gt;
&lt;p&gt;Comet runs on commodity hardware and aims to …&lt;/p&gt;</summary><content type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;p&gt;The Apache DataFusion PMC is pleased to announce version 0.4.0 of the &lt;a href="https://datafusion.apache.org/comet/"&gt;Comet&lt;/a&gt; subproject.&lt;/p&gt;
&lt;p&gt;Comet is an accelerator for Apache Spark that translates Spark physical plans to DataFusion physical plans for
improved performance and efficiency without requiring any code changes.&lt;/p&gt;
&lt;p&gt;Comet runs on commodity hardware and aims to provide 100% compatibility with Apache Spark. Any operators or
expressions that are not fully compatible will fall back to Spark unless explicitly enabled by the user. Refer
to the &lt;a href="https://datafusion.apache.org/comet/user-guide/compatibility.html"&gt;compatibility guide&lt;/a&gt; for more information.&lt;/p&gt;
&lt;p&gt;This release covers approximately six weeks of development work and is the result of merging 51 PRs from 10
contributors. See the &lt;a href="https://github.com/apache/datafusion-comet/blob/main/dev/changelog/0.4.0.md"&gt;change log&lt;/a&gt; for more information.&lt;/p&gt;
&lt;h2 id="release-highlights"&gt;Release Highlights&lt;a class="headerlink" href="#release-highlights" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;h3 id="performance-stability"&gt;Performance &amp;amp; Stability&lt;a class="headerlink" href="#performance-stability" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;There are a number of performance and stability improvements in this release. Here is a summary of some of the
larger changes. Current benchmarking results can be found in the &lt;a href="https://datafusion.apache.org/comet/contributor-guide/benchmarking.html"&gt;Comet Benchmarking Guide&lt;/a&gt;.&lt;/p&gt;
&lt;h4 id="unified-memory-management"&gt;Unified Memory Management&lt;a class="headerlink" href="#unified-memory-management" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h4&gt;
&lt;p&gt;Comet now uses a unified memory management approach that shares an off-heap memory pool with Apache Spark, resulting
in a much simpler configuration. Comet now requires &lt;code&gt;spark.memory.offHeap.enabled=true&lt;/code&gt;. This approach provides a
holistic view of memory usage in Spark and Comet and makes it easier to optimize system performance.&lt;/p&gt;
&lt;h4 id="faster-joins"&gt;Faster Joins&lt;a class="headerlink" href="#faster-joins" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h4&gt;
&lt;p&gt;Apache Spark supports sort-merge and hash joins, which have similar performance characteristics. Spark defaults to
using sort-merge joins because they are less likely to result in OutOfMemory exceptions. In vectorized query
engines such as DataFusion, hash joins outperform sort-merge joins. Comet now has an experimental feature to
replace Spark sort-merge joins with hash joins for improved performance. This feature is experimental because
there is currently no spill-to-disk support in the hash join implementation. This feature can be enabled by
setting &lt;code&gt;spark.comet.exec.replaceSortMergeJoin=true&lt;/code&gt;.&lt;/p&gt;
&lt;h4 id="bloom-filter-aggregates"&gt;Bloom Filter Aggregates&lt;a class="headerlink" href="#bloom-filter-aggregates" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h4&gt;
&lt;p&gt;Spark’s optimizer can insert Bloom filter aggregations and filters to prune large result sets before a shuffle. However,
Comet would fall back to Spark for the aggregation. Comet now has native support for Bloom filter aggregations
after previously supporting Bloom filter testing. Users no longer need to set
&lt;code&gt;spark.sql.optimizer.runtime.bloomFilter.enabled=false&lt;/code&gt; when using Comet.&lt;/p&gt;
&lt;h4 id="complex-type-support"&gt;Complex Type support&lt;a class="headerlink" href="#complex-type-support" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h4&gt;
&lt;p&gt;This release has the following improvements to complex type support:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;Implemented &lt;code&gt;ArrayAppend&lt;/code&gt; and &lt;code&gt;GetArrayStructFields&lt;/code&gt;.&lt;/li&gt;
&lt;li&gt;Implemented native cast between structs&lt;/li&gt;
&lt;li&gt;Implemented native cast from structs to string&lt;/li&gt;
&lt;/ul&gt;
&lt;h2 id="roadmap"&gt;Roadmap&lt;a class="headerlink" href="#roadmap" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;One of the highest priority items on the roadmap is to add support for reading complex types (maps, structs, and arrays)
from Parquet sources, both when reading Parquet directly and from Iceberg.&lt;/p&gt;
&lt;p&gt;Comet currently has proprietary native code for decoding Parquet pages, native column readers for all of Spark’s
primitive types, and special handling for Spark-specific use cases such as timestamp rebasing and decimal type
promotion. This implementation does not yet support complex types. File IO, decryption, and decompression are handled
in JVM code, and Parquet pages are passed on to native code for decoding.&lt;/p&gt;
&lt;p&gt;Rather than add complex type support to this existing code, we are exploring two main options to allow us to
leverage more of the upstream Arrow and DataFusion code.&lt;/p&gt;
&lt;h3 id="use-datafusions-parquetexec"&gt;Use DataFusion’s ParquetExec&lt;a class="headerlink" href="#use-datafusions-parquetexec" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;For use cases where DataFusion can support reading a Parquet source, Comet could create a native plan that uses
DataFusion’s ParquetExec. We are investigating using DataFusion’s SchemaAdapter to handle some Spark-specific
handling of timestamps and decimals.&lt;/p&gt;
&lt;h3 id="use-arrows-parquet-batch-reader"&gt;Use Arrow’s Parquet Batch Reader&lt;a class="headerlink" href="#use-arrows-parquet-batch-reader" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;For use cases not supported by DataFusion’s ParquetExec, such as integrating with Iceberg, we are exploring
replacing our current native Parquet decoding logic with the Arrow readers provided by the Parquet crate.&lt;/p&gt;
&lt;p&gt;Iceberg already provides a vectorized Spark reader for Parquet. A &lt;a href="https://github.com/apache/iceberg/pull/9841"&gt;PR&lt;/a&gt; is open against Iceberg for adding a native
version based on Comet, and we hope to update this to leverage the improvements outlined above.&lt;/p&gt;
&lt;h2 id="getting-involved"&gt;Getting Involved&lt;a class="headerlink" href="#getting-involved" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;The Comet project welcomes new contributors. We use the same &lt;a href="https://datafusion.apache.org/contributor-guide/communication.html#slack-and-discord"&gt;Slack and Discord&lt;/a&gt; channels as the main DataFusion
project and have a weekly &lt;a href="https://docs.google.com/document/d/1NBpkIAuU7O9h8Br5CbFksDhX-L9TyO9wmGLPMe0Plc8/edit?usp=sharing"&gt;DataFusion video call&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;The easiest way to get involved is to test Comet with your current Spark jobs and file issues for any bugs or
performance regressions that you find. See the &lt;a href="https://datafusion.apache.org/comet/user-guide/installation.html"&gt;Getting Started&lt;/a&gt; guide for instructions on downloading and installing
Comet.&lt;/p&gt;
&lt;p&gt;There are also many &lt;a href="https://github.com/apache/datafusion-comet/contribute"&gt;good first issues&lt;/a&gt; waiting for contributions.&lt;/p&gt;</content><category term="blog"/></entry><entry><title>Comparing approaches to User Defined Functions in Apache DataFusion using Python</title><link href="https://datafusion.apache.org/blog/2024/11/19/datafusion-python-udf-comparisons" rel="alternate"/><published>2024-11-19T00:00:00+00:00</published><updated>2024-11-19T00:00:00+00:00</updated><author><name>timsaucer</name></author><id>tag:datafusion.apache.org,2024-11-19:/blog/2024/11/19/datafusion-python-udf-comparisons</id><summary type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;h2 id="personal-context"&gt;Personal Context&lt;a class="headerlink" href="#personal-context" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;For a few months now I’ve been working with &lt;a href="https://datafusion.apache.org/"&gt;Apache DataFusion&lt;/a&gt;, a
fast query engine written in Rust. From my experience the language that nearly all data scientists
are working in is Python. In general, data scientists often use &lt;a href="https://pandas.pydata.org/"&gt;Pandas&lt;/a&gt;
for in-memory tasks and &lt;a href="https://spark.apache.org/"&gt;PySpark&lt;/a&gt; for larger …&lt;/p&gt;</summary><content type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;h2 id="personal-context"&gt;Personal Context&lt;a class="headerlink" href="#personal-context" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;For a few months now I’ve been working with &lt;a href="https://datafusion.apache.org/"&gt;Apache DataFusion&lt;/a&gt;, a
fast query engine written in Rust. From my experience the language that nearly all data scientists
are working in is Python. In general, data scientists often use &lt;a href="https://pandas.pydata.org/"&gt;Pandas&lt;/a&gt;
for in-memory tasks and &lt;a href="https://spark.apache.org/"&gt;PySpark&lt;/a&gt; for larger tasks that require
distributed processing.&lt;/p&gt;
&lt;p&gt;In addition to DataFusion, there is another Rust based newcomer to the DataFrame world,
&lt;a href="https://pola.rs/"&gt;Polars&lt;/a&gt;. The latter is growing extremely fast, and it serves many of the same
use cases as DataFusion. For my use cases, I'm interested in DataFusion because I want to be able
to build small scale tests rapidly and then scale them up to larger distributed systems with ease.
I do recommend evaluating Polars for in-memory work.&lt;/p&gt;
&lt;p&gt;Personally, I would love a single query approach that is fast for both in-memory usage and can
extend to large batch processing to exploit parallelization. I think DataFusion, coupled with
&lt;a href="https://datafusion.apache.org/ballista/"&gt;Ballista&lt;/a&gt; or
&lt;a href="https://github.com/apache/datafusion-ray"&gt;DataFusion-Ray&lt;/a&gt;, may provide this solution.&lt;/p&gt;
&lt;p&gt;As I’m testing, I’m primarily limiting my work to the
&lt;a href="https://datafusion.apache.org/python/"&gt;datafusion-python&lt;/a&gt; project, a wrapper around the Rust
DataFusion library. This wrapper gives you the speed advantages of keeping all of the data in the
Rust implementation and the ergonomics of working in Python. Personally, I would prefer to work
purely in Rust, but I also recognize that since the industry works in Python we should meet the
people where they are.&lt;/p&gt;
&lt;h2 id="user-defined-functions"&gt;User-Defined Functions&lt;a class="headerlink" href="#user-defined-functions" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;The focus of this post is User-Defined Functions (UDFs). The DataFusion library gives a lot of
useful functions already for doing DataFrame manipulation. These are going to be similar to those
you find in other DataFrame libraries. You’ll be able to do simple arithmetic, create substrings of
columns, or find the average value across a group of rows. These cover most of the use cases
you’ll need in a DataFrame.&lt;/p&gt;
&lt;p&gt;However, there will always arise times when you want a custom function. With UDFs you open a
world of possibilities in your code. Sometimes there simply isn’t an easy way to use built-in
functions to achieve your goals.&lt;/p&gt;
&lt;p&gt;In the following, I’m going to demonstrate two example use cases. These are based on real world
problems I’ve encountered. Also I want to demonstrate the approach of “make it work, make it work
well, make it work fast” that is a motto I’ve seen thrown around in data science.&lt;/p&gt;
&lt;p&gt;I will demonstrate three approaches to writing UDFs. In order of increasing performance they are&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;Writing a pure Python function to do your computation&lt;/li&gt;
&lt;li&gt;Using the PyArrow libraries in Python to accelerate your processing&lt;/li&gt;
&lt;li&gt;Writing a UDF in Rust and exposing it to Python&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;Additionally I will demonstrate two variants of this. The first will be nearly identical to the
PyArrow library approach to simplify understanding how to connect the Rust code to Python. In the
second version we will do the iteration through the input arrays ourselves to give even greater
flexibility to the user.&lt;/p&gt;
&lt;p&gt;Here are the two example use cases, taken from my own work but generalized.&lt;/p&gt;
&lt;h3 id="use-case-1-scalar-function"&gt;Use Case 1: Scalar Function&lt;a class="headerlink" href="#use-case-1-scalar-function" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;I have a DataFrame and a list of tuples that I’m interested in. I want to filter out the DataFrame
to only have values that match those tuples from certain columns in the DataFrame.&lt;/p&gt;
&lt;p&gt;To give a concrete example, we will use data generated for the &lt;a href="https://www.tpc.org/tpch/"&gt;TPC-H benchmarks&lt;/a&gt;.
Suppose I have a table of sales line items. There are many columns, but I am interested in three: a
part key (&lt;code&gt;p_partkey&lt;/code&gt;), supplier key (&lt;code&gt;p_suppkey&lt;/code&gt;), and return status (&lt;code&gt;p_returnflag&lt;/code&gt;). I want
only to return a DataFrame with a specific combination of these three values. That is, I want
to know if part number 1530 from supplier 4031 was sold (not returned), so I want a specific
combination of &lt;code&gt;p_partkey = 1530&lt;/code&gt;, &lt;code&gt;p_suppkey = 4031&lt;/code&gt;, and &lt;code&gt;p_returnflag = 'N'&lt;/code&gt;. I have a small
handful of these combinations I want to return.&lt;/p&gt;
&lt;p&gt;Probably the most ergonomic way to do this without UDF is to turn that list of tuples into a
DataFrame itself, perform a join, and select the columns from the original DataFrame. If we were
working in PySpark we would probably broadcast join the DataFrame created from the tuple list since
it is tiny. In practice, I have found that with some DataFrame libraries performing a filter rather
than a join can be significantly faster. This is worth profiling for your specific use case.&lt;/p&gt;
&lt;h3 id="use-case-2-aggregate-function"&gt;Use Case 2: Aggregate Function&lt;a class="headerlink" href="#use-case-2-aggregate-function" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;I have a DataFrame with many values that I want to aggregate. I have already analyzed it and
determined there is a noise level below which I do not want to include in my analysis. I want to
compute a sum of only values that are above my noise threshold.&lt;/p&gt;
&lt;p&gt;This can be done fairly easy without leaning on a User Defined Aggregate Function (UDAF). You can
simply filter the DataFrame and then aggregate using the built-in &lt;code&gt;sum&lt;/code&gt; function. Here, we
demonstrate doing this as a UDF primarily as an example of how to write UDAFs. We will use the
PyArrow compute approach.&lt;/p&gt;
&lt;h2 id="pure-python-approach"&gt;Pure Python approach&lt;a class="headerlink" href="#pure-python-approach" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;The fastest way (developer time, not code time) for me to implement the scalar problem solution
was to do something along the lines of “for each row, check the values of interest contains that
tuple”. I’ve published this as
&lt;a href="https://github.com/apache/datafusion-python/blob/main/examples/python-udf-comparisons.py"&gt;an example&lt;/a&gt;
in the &lt;a href="https://github.com/apache/datafusion-python"&gt;datafusion-python repository&lt;/a&gt;. Here is an
example of how this can be done:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-python"&gt;values_of_interest = [
    (1530, 4031, "N"),
    (6530, 1531, "N"),
    (5618, 619, "N"),
    (8118, 8119, "N"),
]

def is_of_interest_impl(
    partkey_arr: pa.Array,
    suppkey_arr: pa.Array,
    returnflag_arr: pa.Array,
) -&amp;gt; pa.Array:
    result = []
    for idx, partkey in enumerate(partkey_arr):
        partkey = partkey.as_py()
        suppkey = suppkey_arr[idx].as_py()
        returnflag = returnflag_arr[idx].as_py()
        value = (partkey, suppkey, returnflag)
        result.append(value in values_of_interest)

    return pa.array(result)

# Wrap our custom function with `datafusion.udf`, annotating expected 
# parameter and return types
is_of_interest = udf(
    is_of_interest_impl,
    [pa.int64(), pa.int64(), pa.utf8()],
    pa.bool_(),
    "stable",
)

df_udf_filter = df_lineitem.filter(
    is_of_interest(col("l_partkey"), col("l_suppkey"), col("l_returnflag"))
)
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;When working with a DataFusion UDF in Python, you define your function to take in some number of
expressions. During the evaluation, these will get computed into their corresponding values and
passed to your UDF as a PyArrow Array. We must return an Array also with the same number of
elements (rows). So the UDF example just iterates through all of the arrays and checks to see if
the tuple created from these columns matches any of those that we’re looking for.&lt;/p&gt;
&lt;p&gt;I’ll repeat because this is something that tripped me up the first time I wrote a UDF for
datafusion: &lt;strong&gt;DataFusion UDFs, even scalar UDFs, process an array of values at a time not a single
row.&lt;/strong&gt; This is different from some other DataFrame libraries and you may need to recognize a slight
change in mentality.&lt;/p&gt;
&lt;p&gt;Some important lines here are the lines like &lt;code&gt;partkey = partkey.as_py()&lt;/code&gt;. When we do this, we pay a
heavy cost. Now instead of keeping the analysis in the Rust code, we have to take the values in the
array and convert them over to Python objects. In this case we end up getting two numbers and a
string as real Python objects, complete with reference counting and all. Also we are iterating
through the array in Python rather than Rust native. These will &lt;strong&gt;significantly&lt;/strong&gt; slow down your
code. Any time you have to cross the barrier where you change values inside the Rust arrays into
Python objects or vice versa you will pay &lt;strong&gt;heavy&lt;/strong&gt; cost in that transformation. You will want to
design your UDFs to avoid this as much as possible.&lt;/p&gt;
&lt;h2 id="python-approach-using-pyarrow-compute"&gt;Python approach using PyArrow compute&lt;a class="headerlink" href="#python-approach-using-pyarrow-compute" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;DataFusion uses &lt;a href="https://arrow.apache.org/"&gt;Apache Arrow&lt;/a&gt; as its in-memory data format. This can
be seen in the way that Arrow Arrays are passed into the UDFs. We can take advantage of the fact
that &lt;a href="https://arrow.apache.org/docs/python/"&gt;PyArrow&lt;/a&gt;, the canonical Python Arrow implementation,
provides a variety of
useful functions. In the example below, we are only using a few of the boolean functions and the
equality function. Each of these functions takes two arrays and analyzes them row by row. In the
below example, we shift the logic around a little since we are now operating on an entire array of
values instead of checking a single row ourselves.&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-python"&gt;import pyarrow.compute as pc

def udf_using_pyarrow_compute_impl(
    partkey_arr: pa.Array,
    suppkey_arr: pa.Array,
    returnflag_arr: pa.Array,
) -&amp;gt; pa.Array:
    results = None
    for partkey, suppkey, returnflag in values_of_interest:
        filtered_partkey_arr = pc.equal(partkey_arr, partkey)
        filtered_suppkey_arr = pc.equal(suppkey_arr, suppkey)
        filtered_returnflag_arr = pc.equal(returnflag_arr, returnflag)

        resultant_arr = pc.and_(filtered_partkey_arr, filtered_suppkey_arr)
        resultant_arr = pc.and_(resultant_arr, filtered_returnflag_arr)

        if results is None:
            results = resultant_arr
        else:
            results = pc.or_(results, resultant_arr)

    return results


udf_using_pyarrow_compute = udf(
    udf_using_pyarrow_compute_impl,
    [pa.int64(), pa.int64(), pa.utf8()],
    pa.bool_(),
    "stable",
)

df_udf_pyarrow_compute = df_lineitem.filter(
    udf_using_pyarrow_compute(col("l_partkey"), col("l_suppkey"), col("l_returnflag"))
)
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;The idea in the code above is that we will iterate through each of the values of interest, which we
expect to be small. For each of the columns, we compare the value of interest to it’s corresponding
array using &lt;code&gt;pyarrow.compute.equal&lt;/code&gt;. This will give use three boolean arrays. We have a match to
the tuple if we have a row in all three arrays that is true, so we use &lt;code&gt;pyarrow.compute.and_&lt;/code&gt;. Now
our return value from the UDF needs to include arrays for which any of the values of interest list
of tuples exists, so we take the result from the current loop and perform a &lt;code&gt;pyarrow.compute.or_&lt;/code&gt;
on it.&lt;/p&gt;
&lt;p&gt;From my benchmarking, switching from approach of converting values into Python objects to this
approach of using the PyArrow built-in functions leads to about a 10x speed improvement in this
simple problem.&lt;/p&gt;
&lt;p&gt;It’s worth noting that almost all of the PyArrow compute functions expect to take one or two arrays
as their arguments. If you need to write a UDF that is evaluating three or more columns, you’ll
need to do something akin to what we’ve shown here.&lt;/p&gt;
&lt;h2 id="rust-udf-with-python-wrapper"&gt;Rust UDF with Python wrapper&lt;a class="headerlink" href="#rust-udf-with-python-wrapper" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;This is the most complicated approach, but has the potential to be the most performant. What we
will do here is write a Rust function to perform our computation and then expose that function to
Python. I know of two use cases where I would recommend this approach. The first is the case when
the PyArrow compute functions are insufficient for your needs. Perhaps your code is too complex or
could be greatly simplified if you pulled in some outside dependency. The second use case is when
you have written a UDF that you’re sharing across multiple projects and have hardened the approach.
It is possible that you can implement your function in Rust to give a speed improvement and then
every project that is using this shared UDF will benefit from those updates.&lt;/p&gt;
&lt;p&gt;When deciding to use this approach, it’s worth considering how much you think you’ll actually
benefit from the Rust implementation to decide if it’s worth the additional effort to maintain and
deploy the Python wheels you generate. It is certainly not necessary for every use case.&lt;/p&gt;
&lt;p&gt;Due to the excellent work by the Python arrow team, we can simplify our work to needing only two
dependencies on the Rust side, &lt;a href="https://github.com/apache/arrow-rs"&gt;arrow-rs&lt;/a&gt; and
&lt;a href="https://pyo3.rs/"&gt;pyo3&lt;/a&gt;. I have posted a &lt;a href="https://github.com/timsaucer/tuple_filter_example"&gt;minimal example&lt;/a&gt;.
You’ll need &lt;a href="https://github.com/PyO3/maturin"&gt;maturin&lt;/a&gt; to build the project, and you must use
release mode when building to get the expected performance.&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-bash"&gt;maturin develop --release
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;When you write your UDF in Rust you generally will need to take these steps&lt;/p&gt;
&lt;ol&gt;
&lt;li&gt;Write a function description that takes in some number of Python generic objects.&lt;/li&gt;
&lt;li&gt;Convert these objects to Arrow Arrays of the appropriate type(s).&lt;/li&gt;
&lt;li&gt;Perform your computation and create a resultant Array.&lt;/li&gt;
&lt;li&gt;Convert the array into a Python generic object.&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;For the conversion to and from Python objects, we can take advantage of the
&lt;code&gt;ArrayData::from_pyarrow_bound&lt;/code&gt; and &lt;code&gt;ArrayData::to_pyarrow&lt;/code&gt; functions.  All that remains is to
perform your computation.&lt;/p&gt;
&lt;p&gt;We are going to demonstrate doing this computation in two ways. The first is to mimic what we’ve
done in the above approach using PyArrow. In the second we demonstrate iterating through the three
arrays ourselves.&lt;/p&gt;
&lt;p&gt;In our first approach, we can expect the performance to be nearly identical to when we used the
PyArrow compute functions. On the Rust side we will have slightly less overhead but the heavy
lifting portions of the code are essentially the same between this Rust implementation and the
PyArrow approach above.&lt;/p&gt;
&lt;p&gt;The reason for demonstrating this, even though it doesn’t provide a significant speedup over
Python, is to primarily demonstrate how to make the Python to Rust with Python wrapper
transition. In the second implementation you can see how we can iterate through all of the arrays
ourselves.&lt;/p&gt;
&lt;p&gt;In this first example, we are hard coding the values of interest, but in the following section
we demonstrate passing these in during initialization.&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-rust"&gt;#[pyfunction]
pub fn tuple_filter_fn(
    py: Python&amp;lt;'_&amp;gt;,
    partkey_expr: &amp;amp;Bound&amp;lt;'_, PyAny&amp;gt;,
    suppkey_expr: &amp;amp;Bound&amp;lt;'_, PyAny&amp;gt;,
    returnflag_expr: &amp;amp;Bound&amp;lt;'_, PyAny&amp;gt;,
) -&amp;gt; PyResult&amp;lt;Py&amp;lt;PyAny&amp;gt;&amp;gt; {
    let partkey_arr: PrimitiveArray&amp;lt;Int64Type&amp;gt; =
        ArrayData::from_pyarrow_bound(partkey_expr)?.into();
    let suppkey_arr: PrimitiveArray&amp;lt;Int64Type&amp;gt; =
        ArrayData::from_pyarrow_bound(suppkey_expr)?.into();
    let returnflag_arr: StringArray = ArrayData::from_pyarrow_bound(returnflag_expr)?.into();

    let values_of_interest = vec![
        (1530, 4031, "N".to_string()),
        (6530, 1531, "N".to_string()),
        (5618, 619, "N".to_string()),
        (8118, 8119, "N".to_string()),
    ];

    let mut res: Option&amp;lt;BooleanArray&amp;gt; = None;

    for (partkey, suppkey, returnflag) in &amp;amp;values_of_interest {
        let filtered_partkey_arr = BooleanArray::from_unary(&amp;amp;partkey_arr, |p| p == *partkey);
        let filtered_suppkey_arr = BooleanArray::from_unary(&amp;amp;suppkey_arr, |s| s == *suppkey);
        let filtered_returnflag_arr =
            BooleanArray::from_unary(&amp;amp;returnflag_arr, |s| s == returnflag);

        let part_and_supp = compute::and(&amp;amp;filtered_partkey_arr, &amp;amp;filtered_suppkey_arr)
            .map_err(|e| PyValueError::new_err(e.to_string()))?;
        let resultant_arr = compute::and(&amp;amp;part_and_supp, &amp;amp;filtered_returnflag_arr)
            .map_err(|e| PyValueError::new_err(e.to_string()))?;

        res = match res {
            Some(r) =&amp;gt; compute::or(&amp;amp;r, &amp;amp;resultant_arr).ok(),
            None =&amp;gt; Some(resultant_arr),
        };
    }

    res.unwrap().into_data().to_pyarrow(py)
}


#[pymodule]
fn tuple_filter_example(module: &amp;amp;Bound&amp;lt;'_, PyModule&amp;gt;) -&amp;gt; PyResult&amp;lt;()&amp;gt; {
    module.add_function(wrap_pyfunction!(tuple_filter_fn, module)?)?;
    Ok(())
}
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;To use this we use the &lt;code&gt;udf&lt;/code&gt; function in &lt;code&gt;datafusion-python&lt;/code&gt; just as before.&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-python"&gt;from datafusion import udf
import pyarrow as pa
from tuple_filter_example import tuple_filter_fn

udf_using_custom_rust_fn = udf(
    tuple_filter_fn,
    [pa.int64(), pa.int64(), pa.utf8()],
    pa.bool_(),
    "stable",
)
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;That's it! We've now got a third party Rust UDF with Python wrappers working with DataFusion's
Python bindings!&lt;/p&gt;
&lt;h3 id="rust-udf-with-initialization"&gt;Rust UDF with initialization&lt;a class="headerlink" href="#rust-udf-with-initialization" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;Looking at the code above, you can see that it is hard coding the values we're interested in. There
are many types of UDFs that don't require any additional data provided to them before they start
the computation. The code above is sloppy, so let's clean it up.&lt;/p&gt;
&lt;p&gt;We want to write the function to take some additional data. A limitation of the UDFs we create is
that they expect to operate on entire arrays of data at a time. We can get around this problem by
creating an initializer for our UDF. We do this by defining a Rust struct that contains the data we
need and implement two methods on this struct, &lt;code&gt;new&lt;/code&gt; and &lt;code&gt;__call__&lt;/code&gt;. By doing this we will create a
Python object that is callable, so it can be the function we provide to &lt;code&gt;udf&lt;/code&gt;.&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-rust"&gt;#[pyclass]
pub struct TupleFilterClass {
    values_of_interest: Vec&amp;lt;(i64, i64, String)&amp;gt;,
}

#[pymethods]
impl TupleFilterClass {
    #[new]
    fn new(values_of_interest: Vec&amp;lt;(i64, i64, String)&amp;gt;) -&amp;gt; Self {
        Self {
            values_of_interest,
        }
    }

    fn __call__(
        &amp;amp;self,
        py: Python&amp;lt;'_&amp;gt;,
        partkey_expr: &amp;amp;Bound&amp;lt;'_, PyAny&amp;gt;,
        suppkey_expr: &amp;amp;Bound&amp;lt;'_, PyAny&amp;gt;,
        returnflag_expr: &amp;amp;Bound&amp;lt;'_, PyAny&amp;gt;,
    ) -&amp;gt; PyResult&amp;lt;Py&amp;lt;PyAny&amp;gt;&amp;gt; {
        let partkey_arr: PrimitiveArray&amp;lt;Int64Type&amp;gt; =
            ArrayData::from_pyarrow_bound(partkey_expr)?.into();
        let suppkey_arr: PrimitiveArray&amp;lt;Int64Type&amp;gt; =
            ArrayData::from_pyarrow_bound(suppkey_expr)?.into();
        let returnflag_arr: StringArray = ArrayData::from_pyarrow_bound(returnflag_expr)?.into();

        let mut res: Option&amp;lt;BooleanArray&amp;gt; = None;

        for (partkey, suppkey, returnflag) in &amp;amp;self.values_of_interest {
            let filtered_partkey_arr = BooleanArray::from_unary(&amp;amp;partkey_arr, |p| p == *partkey);
            let filtered_suppkey_arr = BooleanArray::from_unary(&amp;amp;suppkey_arr, |s| s == *suppkey);
            let filtered_returnflag_arr =
                BooleanArray::from_unary(&amp;amp;returnflag_arr, |s| s == returnflag);

            let part_and_supp = compute::and(&amp;amp;filtered_partkey_arr, &amp;amp;filtered_suppkey_arr)
                .map_err(|e| PyValueError::new_err(e.to_string()))?;
            let resultant_arr = compute::and(&amp;amp;part_and_supp, &amp;amp;filtered_returnflag_arr)
                .map_err(|e| PyValueError::new_err(e.to_string()))?;

            res = match res {
                Some(r) =&amp;gt; compute::or(&amp;amp;r, &amp;amp;resultant_arr).ok(),
                None =&amp;gt; Some(resultant_arr),
            };
        }

        res.unwrap().into_data().to_pyarrow(py)
    }
}

#[pymodule]
fn tuple_filter_example(module: &amp;amp;Bound&amp;lt;'_, PyModule&amp;gt;) -&amp;gt; PyResult&amp;lt;()&amp;gt; {
    module.add_class::&amp;lt;TupleFilterClass&amp;gt;()?;
    Ok(())
}
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;When you write this, you don't have to call your constructor &lt;code&gt;new&lt;/code&gt;. The more important part is that
you have &lt;code&gt;#[new]&lt;/code&gt; designated on the function. With this you can provide any kinds of data you need
during processing. Using this initializer in Python is fairly straightforward.&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-python"&gt;from datafusion import udf
import pyarrow as pa
from tuple_filter_example import TupleFilterClass

tuple_filter_class = TupleFilterClass(values_of_interest)

udf_using_custom_rust_fn_with_data = udf(
    tuple_filter_class,
    [pa.int64(), pa.int64(), pa.utf8()],
    pa.bool_(),
    "stable",
    name="tuple_filter_with_data"
)
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;When you use this approach you will need to provide a &lt;code&gt;name&lt;/code&gt; argument to &lt;code&gt;udf&lt;/code&gt;. This is because our
class/struct does not get the &lt;code&gt;__qualname__&lt;/code&gt; attribute that the &lt;code&gt;udf&lt;/code&gt; function is looking for. You
can give this udf any name you choose.&lt;/p&gt;
&lt;h3 id="rust-udf-with-direct-iteration"&gt;Rust UDF with direct iteration&lt;a class="headerlink" href="#rust-udf-with-direct-iteration" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;The final version of our scalar UDF is one where we implement it in Rust and iterate through all of
the arrays ourselves. If you are iterating through more than 3 arrays at a time I recommend looking
at &lt;a href="https://docs.rs/itertools/latest/itertools/macro.izip.html"&gt;izip&lt;/a&gt; in the
&lt;a href="https://crates.io/crates/itertools"&gt;itertools crate&lt;/a&gt;. For ease of understanding and since we only
have 3 arrays here I will just explicitly create my own tuple here.&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-rust"&gt;#[pyclass]
pub struct TupleFilterDirectIterationClass {
    values_of_interest: Vec&amp;lt;(i64, i64, String)&amp;gt;,
}

#[pymethods]
impl TupleFilterDirectIterationClass {
    #[new]
    fn new(values_of_interest: Vec&amp;lt;(i64, i64, String)&amp;gt;) -&amp;gt; Self {
        Self { values_of_interest }
    }

    fn __call__(
        &amp;amp;self,
        py: Python&amp;lt;'_&amp;gt;,
        partkey_expr: &amp;amp;Bound&amp;lt;'_, PyAny&amp;gt;,
        suppkey_expr: &amp;amp;Bound&amp;lt;'_, PyAny&amp;gt;,
        returnflag_expr: &amp;amp;Bound&amp;lt;'_, PyAny&amp;gt;,
    ) -&amp;gt; PyResult&amp;lt;Py&amp;lt;PyAny&amp;gt;&amp;gt; {
        let partkey_arr: PrimitiveArray&amp;lt;Int64Type&amp;gt; =
            ArrayData::from_pyarrow_bound(partkey_expr)?.into();
        let suppkey_arr: PrimitiveArray&amp;lt;Int64Type&amp;gt; =
            ArrayData::from_pyarrow_bound(suppkey_expr)?.into();
        let returnflag_arr: StringArray = ArrayData::from_pyarrow_bound(returnflag_expr)?.into();

        let values_to_search: Vec&amp;lt;(&amp;amp;i64, &amp;amp;i64, &amp;amp;str)&amp;gt; = (&amp;amp;self.values_of_interest)
            .iter()
            .map(|(a, b, c)| (a, b, c.as_str()))
            .collect();

        let values = partkey_arr
            .values()
            .iter()
            .zip(suppkey_arr.values().iter())
            .zip(returnflag_arr.iter())
            .map(|((a, b), c)| (a, b, c.unwrap_or_default()))
            .map(|v| values_to_search.contains(&amp;amp;v));

        let res: BooleanArray = BooleanBuffer::from_iter(values).into();

        res.into_data().to_pyarrow(py)
    }
}
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;We convert the &lt;code&gt;values_of_interest&lt;/code&gt; into a vector of borrowed types so that we can do a fast search
without creating additional memory. The other option is to turn the &lt;code&gt;returnflag&lt;/code&gt; into a &lt;code&gt;String&lt;/code&gt;
but that memory allocation is unnecessary. After that we use two &lt;code&gt;zip&lt;/code&gt; operations so that we can
iterate over all three columns in a single pass. Since each &lt;code&gt;zip&lt;/code&gt; will return a tuple of two
elements, a quick &lt;code&gt;map&lt;/code&gt; turns them into the tuple format we need. Also, &lt;code&gt;StringArray&lt;/code&gt; is a little
different in the buffer it uses, so it is treated slightly differently from the others.&lt;/p&gt;
&lt;h2 id="user-defined-aggregate-function"&gt;User Defined Aggregate Function&lt;a class="headerlink" href="#user-defined-aggregate-function" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;Writing a user defined aggregate function or user defined window function is slightly more complex
than scalar functions. This is because we must accumulate values and there is no guarantee that one
batch will contain all the values we are aggregating over. For this we need to define an
&lt;code&gt;Accumulator&lt;/code&gt; which will do a few things.&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;Process a batch and compute an internal state&lt;/li&gt;
&lt;li&gt;Share the state so that we can combine multiple batches&lt;/li&gt;
&lt;li&gt;Merge the results across multiple batches&lt;/li&gt;
&lt;li&gt;Return the final result&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;In the example below, we're going to look at customer orders and we want to know per customer ID,
how much they have ordered total. We want to ignore small orders, which we define as anything under
5000.&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-python"&gt;from datafusion import Accumulator, udaf
import pyarrow as pa
import pyarrow.compute as pc

IGNORE_THRESHOLD = 5000.0
class AboveThresholdAccum(Accumulator):
    def __init__(self) -&amp;gt; None:
        self._sum = 0.0

    def update(self, values: pa.Array) -&amp;gt; None:
        over_threshold = pc.greater(values, pa.scalar(IGNORE_THRESHOLD))
        sum_above = pc.sum(values.filter(over_threshold)).as_py()
        if sum_above is None:
            sum_above = 0.0
        self._sum = self._sum + sum_above

    def merge(self, states: List[pa.Array]) -&amp;gt; None:
        self._sum = self._sum + pc.sum(states[0]).as_py()

    def state(self) -&amp;gt; List[pa.Scalar]:
        return [pa.scalar(self._sum)]

    def evaluate(self) -&amp;gt; pa.Scalar:
        return pa.scalar(self._sum)

sum_above_threshold = udaf(AboveThresholdAccum, [pa.float64()], pa.float64(), [pa.float64()], 'stable')

df_orders.aggregate([col("o_custkey")],[sum_above_threshold(col("o_totalprice")).alias("sales")]).show()
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;Since we are doing a &lt;code&gt;sum&lt;/code&gt; we can keep a single value as our internal state. When we call &lt;code&gt;update()&lt;/code&gt;
we will process a single array and update the internal state, which we share with the &lt;code&gt;state()&lt;/code&gt;
function. For larger batches we may &lt;code&gt;merge()&lt;/code&gt; these states. It is important to note that the
&lt;code&gt;states&lt;/code&gt; in the &lt;code&gt;merge()&lt;/code&gt; function are an array of the values returned from &lt;code&gt;state()&lt;/code&gt;. It is
entirely possible that the &lt;code&gt;merge&lt;/code&gt; function is significantly different than the &lt;code&gt;update&lt;/code&gt;, though in
our example they are very similar.&lt;/p&gt;
&lt;p&gt;One example of implementing a user defined aggregate function where the &lt;code&gt;update()&lt;/code&gt; and &lt;code&gt;merge()&lt;/code&gt;
operations are different is computing an average. In &lt;code&gt;update()&lt;/code&gt; we would create a state that is both
a sum and a count. &lt;code&gt;state()&lt;/code&gt; would return a list of these two values, and &lt;code&gt;merge()&lt;/code&gt; would compute
the final result.&lt;/p&gt;
&lt;h2 id="user-defined-window-functions"&gt;User Defined Window Functions&lt;a class="headerlink" href="#user-defined-window-functions" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;Writing a user defined window function is slightly more complex than an aggregate function due
to the variety of ways that window functions are called. I recommend reviewing the
&lt;a href="https://datafusion.apache.org/python/user-guide/common-operations/udf-and-udfa.html"&gt;online documentation&lt;/a&gt;
for a description of which functions need to be implemented. The details of how to implement
these generally follow the same patterns as described above for aggregate functions.&lt;/p&gt;
&lt;h2 id="performance-comparison"&gt;Performance Comparison&lt;a class="headerlink" href="#performance-comparison" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;For the scalar functions above, we performed a timing evaluation, repeating the operation 100
times. For this simple example these are our results.&lt;/p&gt;
&lt;pre&gt;&lt;code&gt;+-----------------------------+--------------+---------+
| approach                    | Average Time | Std Dev |
+-----------------------------+--------------+---------+
| python udf                  | 4.969        | 0.062   |
| simple filter               | 1.075        | 0.022   |
| explicit filter             | 0.685        | 0.063   |
| pyarrow compute             | 0.529        | 0.017   |
| arrow rust compute          | 0.511        | 0.034   |
| arrow rust compute as class | 0.502        | 0.011   |
| rust custom iterator        | 0.478        | 0.009   |
+-----------------------------+--------------+---------+
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;As expected, the conversion to Python objects is by far the worst performance. As soon as we drop
into using any functions that keep the data entirely on the Native (Rust or C/C++) side we see a
near 10x speed improvement. Then as we increase our complexity from using PyArrow compute functions
to implementing the UDF in Rust we see incremental improvements. Our fastest approach - iterating
through the arrays ourselves does operate nearly 10% faster than the PyArrow compute approach.&lt;/p&gt;
&lt;h2 id="final-thoughts-and-recommendations"&gt;Final Thoughts and Recommendations&lt;a class="headerlink" href="#final-thoughts-and-recommendations" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;For anyone who is curious about &lt;a href="https://datafusion.apache.org/"&gt;DataFusion&lt;/a&gt; I highly recommend
giving it a try. This post was designed to make it easier for new users to the Python implementation
to work with User Defined Functions by giving a few examples of how one might implement these.&lt;/p&gt;
&lt;p&gt;When it comes to designing UDFs, I strongly recommend seeing if you can write your UDF using
&lt;a href="https://arrow.apache.org/docs/python/api/compute.html"&gt;PyArrow functions&lt;/a&gt; rather than pure Python
objects. As shown in the scalar example above, you can achieve a 10x speedup by using PyArrow
functions. If you must do something that isn't well represented by the PyArrow compute functions,
then I would consider using a Rust based UDF in the manner shown above.&lt;/p&gt;
&lt;p&gt;I would like to thank &lt;a href="https://github.com/alamb"&gt;@alamb&lt;/a&gt;, &lt;a href="https://github.com/andygrove"&gt;@andygrove&lt;/a&gt;, &lt;a href="https://github.com/comphead"&gt;@comphead&lt;/a&gt;, &lt;a href="https://github.com/emgeee"&gt;@emgeee&lt;/a&gt;, &lt;a href="https://github.com/kylebarron"&gt;@kylebarron&lt;/a&gt;, and &lt;a href="https://github.com/Omega359"&gt;@Omega359&lt;/a&gt;
for their helpful reviews and feedback.&lt;/p&gt;
&lt;p&gt;Lastly, the Apache Arrow and DataFusion community is an active group of very helpful people working
to make a great tool. If you want to get involved, please take a look at the
&lt;a href="https://datafusion.apache.org/python/"&gt;online documentation&lt;/a&gt; and jump in to help with one of the
&lt;a href="https://github.com/apache/datafusion-python/issues"&gt;open issues&lt;/a&gt;.&lt;/p&gt;</content><category term="blog"/></entry><entry><title>Apache DataFusion is now the fastest single node engine for querying Apache Parquet files</title><link href="https://datafusion.apache.org/blog/2024/11/18/datafusion-fastest-single-node-parquet-clickbench" rel="alternate"/><published>2024-11-18T00:00:00+00:00</published><updated>2024-11-18T00:00:00+00:00</updated><author><name>Andrew Lamb, Staff Engineer at InfluxData</name></author><id>tag:datafusion.apache.org,2024-11-18:/blog/2024/11/18/datafusion-fastest-single-node-parquet-clickbench</id><summary type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;p&gt;I am extremely excited to announce that &lt;a href="https://crates.io/crates/datafusion"&gt;Apache DataFusion&lt;/a&gt;  is the
fastest engine for querying Apache Parquet files in &lt;a href="https://benchmark.clickhouse.com/"&gt;ClickBench&lt;/a&gt;. It is faster
than &lt;a href="https://duckdb.org/"&gt;DuckDB&lt;/a&gt;, &lt;a href="https://clickhouse.com/chdb"&gt;chDB&lt;/a&gt; and &lt;a href="https://clickhouse.com/"&gt;Clickhouse&lt;/a&gt; using the same hardware. It also marks
the first time a &lt;a href="https://www.rust-lang.org/"&gt;Rust&lt;/a&gt;-based engine holds the top spot, which has previously
been …&lt;/p&gt;</summary><content type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;p&gt;I am extremely excited to announce that &lt;a href="https://crates.io/crates/datafusion"&gt;Apache DataFusion&lt;/a&gt;  is the
fastest engine for querying Apache Parquet files in &lt;a href="https://benchmark.clickhouse.com/"&gt;ClickBench&lt;/a&gt;. It is faster
than &lt;a href="https://duckdb.org/"&gt;DuckDB&lt;/a&gt;, &lt;a href="https://clickhouse.com/chdb"&gt;chDB&lt;/a&gt; and &lt;a href="https://clickhouse.com/"&gt;Clickhouse&lt;/a&gt; using the same hardware. It also marks
the first time a &lt;a href="https://www.rust-lang.org/"&gt;Rust&lt;/a&gt;-based engine holds the top spot, which has previously
been held by traditional C/C++-based engines.&lt;/p&gt;
&lt;p&gt;&lt;img alt="Apache DataFusion Logo" class="img-fluid" src="/blog/images/2x_bgwhite_original.png" width="80%"/&gt;&lt;/p&gt;
&lt;p&gt;&lt;img alt="ClickBench performance for DataFusion 43.0.0" class="img-fluid" src="/blog/images/clickbench-datafusion-43/perf.png" width="100%"/&gt;&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Figure 1&lt;/strong&gt;: 2024-11-16 &lt;a href="https://benchmark.clickhouse.com/#eyJzeXN0ZW0iOnsiQWxsb3lEQiI6ZmFsc2UsIkFsbG95REIgKHR1bmVkKSI6ZmFsc2UsIkF0aGVuYSAocGFydGl0aW9uZWQpIjpmYWxzZSwiQXRoZW5hIChzaW5nbGUpIjpmYWxzZSwiQXVyb3JhIGZvciBNeVNRTCI6ZmFsc2UsIkF1cm9yYSBmb3IgUG9zdGdyZVNRTCI6ZmFsc2UsIkJ5Q29uaXR5IjpmYWxzZSwiQnl0ZUhvdXNlIjpmYWxzZSwiY2hEQiAoRGF0YUZyYW1lKSI6ZmFsc2UsImNoREIgKFBhcnF1ZXQsIHBhcnRpdGlvbmVkKSI6dHJ1ZSwiY2hEQiI6ZmFsc2UsIkNpdHVzIjpmYWxzZSwiQ2xpY2tIb3VzZSBDbG91ZCAoYXdzKSI6ZmFsc2UsIkNsaWNrSG91c2UgQ2xvdWQgKGF6dXJlKSI6ZmFsc2UsIkNsaWNrSG91c2UgQ2xvdWQgKGdjcCkiOmZhbHNlLCJDbGlja0hvdXNlIChkYXRhIGxha2UsIHBhcnRpdGlvbmVkKSI6ZmFsc2UsIkNsaWNrSG91c2UgKGRhdGEgbGFrZSwgc2luZ2xlKSI6ZmFsc2UsIkNsaWNrSG91c2UgKFBhcnF1ZXQsIHBhcnRpdGlvbmVkKSI6dHJ1ZSwiQ2xpY2tIb3VzZSAoUGFycXVldCwgc2luZ2xlKSI6ZmFsc2UsIkNsaWNrSG91c2UgKHdlYikiOmZhbHNlLCJDbGlja0hvdXNlIjpmYWxzZSwiQ2xpY2tIb3VzZSAodHVuZWQpIjpmYWxzZSwiQ2xpY2tIb3VzZSAodHVuZWQsIG1lbW9yeSkiOmZhbHNlLCJDbG91ZGJlcnJ5IjpmYWxzZSwiQ3JhdGVEQiI6ZmFsc2UsIkNydW5jaHkgQnJpZGdlIGZvciBBbmFseXRpY3MgKFBhcnF1ZXQpIjpmYWxzZSwiRGF0YWJlbmQiOmZhbHNlLCJEYXRhRnVzaW9uIChQYXJxdWV0LCBwYXJ0aXRpb25lZCkiOnRydWUsIkRhdGFGdXNpb24gKFBhcnF1ZXQsIHNpbmdsZSkiOmZhbHNlLCJBcGFjaGUgRG9yaXMiOmZhbHNlLCJEcnVpZCI6ZmFsc2UsIkR1Y2tEQiAoRGF0YUZyYW1lKSI6ZmFsc2UsIkR1Y2tEQiAoUGFycXVldCwgcGFydGl0aW9uZWQpIjp0cnVlLCJEdWNrREIiOmZhbHNlLCJFbGFzdGljc2VhcmNoIjpmYWxzZSwiRWxhc3RpY3NlYXJjaCAodHVuZWQpIjpmYWxzZSwiR2xhcmVEQiI6ZmFsc2UsIkdyZWVucGx1bSI6ZmFsc2UsIkhlYXZ5QUkiOmZhbHNlLCJIeWRyYSI6ZmFsc2UsIkluZm9icmlnaHQiOmZhbHNlLCJLaW5ldGljYSI6ZmFsc2UsIk1hcmlhREIgQ29sdW1uU3RvcmUiOmZhbHNlLCJNYXJpYURCIjpmYWxzZSwiTW9uZXREQiI6ZmFsc2UsIk1vbmdvREIiOmZhbHNlLCJNb3RoZXJEdWNrIjpmYWxzZSwiTXlTUUwgKE15SVNBTSkiOmZhbHNlLCJNeVNRTCI6ZmFsc2UsIk94bGEiOmZhbHNlLCJQYW5kYXMgKERhdGFGcmFtZSkiOmZhbHNlLCJQYXJhZGVEQiAoUGFycXVldCwgcGFydGl0aW9uZWQpIjp0cnVlLCJQYXJhZGVEQiAoUGFycXVldCwgc2luZ2xlKSI6ZmFsc2UsIlBpbm90IjpmYWxzZSwiUG9sYXJzIChEYXRhRnJhbWUpIjpmYWxzZSwiUG9zdGdyZVNRTCAodHVuZWQpIjpmYWxzZSwiUG9zdGdyZVNRTCI6ZmFsc2UsIlF1ZXN0REIgKHBhcnRpdGlvbmVkKSI6ZmFsc2UsIlF1ZXN0REIiOmZhbHNlLCJSZWRzaGlmdCI6ZmFsc2UsIlNpbmdsZVN0b3JlIjpmYWxzZSwiU25vd2ZsYWtlIjpmYWxzZSwiU1FMaXRlIjpmYWxzZSwiU3RhclJvY2tzIjpmYWxzZSwiVGFibGVzcGFjZSI6ZmFsc2UsIlRlbWJvIE9MQVAgKGNvbHVtbmFyKSI6ZmFsc2UsIlRpbWVzY2FsZURCIChubyBjb2x1bW5zdG9yZSkiOmZhbHNlLCJUaW1lc2NhbGVEQiI6ZmFsc2UsIlRpbnliaXJkIChGcmVlIFRyaWFsKSI6ZmFsc2UsIlVtYnJhIjpmYWxzZX0sInR5cGUiOnsiQyI6dHJ1ZSwiY29sdW1uLW9yaWVudGVkIjp0cnVlLCJQb3N0Z3JlU1FMIGNvbXBhdGlibGUiOnRydWUsIm1hbmFnZWQiOnRydWUsImdjcCI6dHJ1ZSwic3RhdGVsZXNzIjp0cnVlLCJKYXZhIjp0cnVlLCJDKysiOnRydWUsIk15U1FMIGNvbXBhdGlibGUiOnRydWUsInJvdy1vcmllbnRlZCI6dHJ1ZSwiQ2xpY2tIb3VzZSBkZXJpdmF0aXZlIjp0cnVlLCJlbWJlZGRlZCI6dHJ1ZSwic2VydmVybGVzcyI6dHJ1ZSwiZGF0YWZyYW1lIjp0cnVlLCJhd3MiOnRydWUsImF6dXJlIjp0cnVlLCJhbmFseXRpY2FsIjp0cnVlLCJSdXN0Ijp0cnVlLCJzZWFyY2giOnRydWUsImRvY3VtZW50Ijp0cnVlLCJzb21ld2hhdCBQb3N0Z3JlU1FMIGNvbXBhdGlibGUiOnRydWUsInRpbWUtc2VyaWVzIjp0cnVlfSwibWFjaGluZSI6eyIxNiB2Q1BVIDEyOEdCIjp0cnVlLCI4IHZDUFUgNjRHQiI6dHJ1ZSwic2VydmVybGVzcyI6dHJ1ZSwiMTZhY3UiOnRydWUsImM2YS40eGxhcmdlLCA1MDBnYiBncDIiOnRydWUsIkwiOnRydWUsIk0iOnRydWUsIlMiOnRydWUsIlhTIjp0cnVlLCJjNmEubWV0YWwsIDUwMGdiIGdwMiI6ZmFsc2UsIjE5MkdCIjp0cnVlLCIyNEdCIjp0cnVlLCIzNjBHQiI6dHJ1ZSwiNDhHQiI6dHJ1ZSwiNzIwR0IiOnRydWUsIjk2R0IiOnRydWUsImRldiI6dHJ1ZSwiNzA4R0IiOnRydWUsImM1bi40eGxhcmdlLCA1MDBnYiBncDIiOnRydWUsIkFuYWx5dGljcy0yNTZHQiAoNjQgdkNvcmVzLCAyNTYgR0IpIjp0cnVlLCJjNS40eGxhcmdlLCA1MDBnYiBncDIiOnRydWUsImM2YS40eGxhcmdlLCAxNTAwZ2IgZ3AyIjp0cnVlLCJjbG91ZCI6dHJ1ZSwiZGMyLjh4bGFyZ2UiOnRydWUsInJhMy4xNnhsYXJnZSI6dHJ1ZSwicmEzLjR4bGFyZ2UiOnRydWUsInJhMy54bHBsdXMiOnRydWUsIlMyIjp0cnVlLCJTMjQiOnRydWUsIjJYTCI6dHJ1ZSwiM1hMIjp0cnVlLCI0WEwiOnRydWUsIlhMIjp0cnVlLCJMMSAtIDE2Q1BVIDMyR0IiOnRydWUsImM2YS40eGxhcmdlLCA1MDBnYiBncDMiOnRydWV9LCJjbHVzdGVyX3NpemUiOnsiMSI6dHJ1ZSwiMiI6dHJ1ZSwiNCI6dHJ1ZSwiOCI6dHJ1ZSwiMTYiOnRydWUsIjMyIjp0cnVlLCI2NCI6dHJ1ZSwiMTI4Ijp0cnVlLCJzZXJ2ZXJsZXNzIjp0cnVlfSwibWV0cmljIjoiaG90IiwicXVlcmllcyI6W3RydWUsdHJ1ZSx0cnVlLHRydWUsdHJ1ZSx0cnVlLHRydWUsdHJ1ZSx0cnVlLHRydWUsdHJ1ZSx0cnVlLHRydWUsdHJ1ZSx0cnVlLHRydWUsdHJ1ZSx0cnVlLHRydWUsdHJ1ZSx0cnVlLHRydWUsdHJ1ZSx0cnVlLHRydWUsdHJ1ZSx0cnVlLHRydWUsdHJ1ZSx0cnVlLHRydWUsdHJ1ZSx0cnVlLHRydWUsdHJ1ZSx0cnVlLHRydWUsdHJ1ZSx0cnVlLHRydWUsdHJ1ZSx0cnVlLHRydWVdfQ=="&gt;ClickBench Results&lt;/a&gt; for the  ‘hot’[^1] run against the
partitioned 14 GB Parquet dataset (100 files, each ~140MB) on a &lt;code&gt;c6a.4xlarge&lt;/code&gt; (16
CPU / 32 GB  RAM) VM. Measurements are relative (&lt;code&gt;1.x&lt;/code&gt;) to results using
different hardware.&lt;/p&gt;
&lt;p&gt;Best in class performance on Parquet is now available to anyone. DataFusion’s
open design lets you start quickly with a full featured Query Engine, including
SQL, data formats, catalogs, and more, and then customize any behavior you need.
I predict the continued emergence of new classes of data systems now that
creators can focus the bulk of their innovation on areas such as query
languages, system integrations, and data formats rather than trying to play
catchup with core engine performance.&lt;/p&gt;
&lt;p&gt;ClickBench also includes results for proprietary storage formats, which require
costly load / export steps, making them useful in fewer use cases and thus much
less important than open formats (though the idea of use case specific formats
is interesting[^2]).&lt;/p&gt;
&lt;p&gt;This blog post highlights some of the techniques we used to achieve this
performance, and celebrates the teamwork involved.&lt;/p&gt;
&lt;h1 id="a-strong-history-of-performance-improvements"&gt;A Strong History of Performance Improvements&lt;a class="headerlink" href="#a-strong-history-of-performance-improvements" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h1&gt;
&lt;p&gt;Performance has long been a core focus for DataFusion's community, and 
speed attracts users and contributors. Recently, we seem to have been
even more focused on performance, including in July, 2024 when &lt;a href="https://www.linkedin.com/in/mehmet-ozan-kabak/"&gt;Mehmet Ozan
Kabak&lt;/a&gt;, CEO of &lt;a href="https://www.synnada.ai/"&gt;Synnada&lt;/a&gt;, again &lt;a href="https://github.com/apache/datafusion/issues/11442#issuecomment-2226834443"&gt;suggested focusing on performance&lt;/a&gt;. This
got many of us excited (who doesn’t love a challenge!), and we have subsequently
rallied to steadily improve the performance release on release as shown in
Figure 2.&lt;/p&gt;
&lt;p&gt;&lt;img alt="ClickBench performance results over time for DataFusion" class="img-fluid" src="/blog/images/clickbench-datafusion-43/perf-over-time.png" width="100%"/&gt;&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Figure 2&lt;/strong&gt;: ClickBench performance improved over 30% between DataFusion 34
(released Dec. 2023) and DataFusion 43 (released Nov. 2024).&lt;/p&gt;
&lt;p&gt;Like all good optimization efforts, ours took sustained effort as DataFusion ran
out of &lt;a href="https://www.influxdata.com/blog/aggregating-millions-groups-fast-apache-arrow-datafusion"&gt;single 2x performance improvements&lt;/a&gt; several years ago. Working together our
community of engineers from around the world[^3] and all experience levels[^4]
pulled it off (check out &lt;a href="https://github.com/apache/datafusion/issues/12821"&gt;this discussion&lt;/a&gt; to get a sense). It may be a "&lt;a href="https://db.cs.cmu.edu/seminar2024/"&gt;hobo
sandwich&lt;/a&gt;" [^5], but it is a tasty one!&lt;/p&gt;
&lt;p&gt;Of course, most of these techniques have been implemented and described before,
but until now they were only available in proprietary systems such as
&lt;a href="https://www.vertica.com/"&gt;Vertica&lt;/a&gt;, &lt;a href="https://www.databricks.com/product/photon"&gt;DataBricks
Photon&lt;/a&gt;, or
&lt;a href="https://www.snowflake.com/en/"&gt;Snowflake&lt;/a&gt; or in tightly integrated open source
systems such as &lt;a href="https://duckdb.org/"&gt;DuckDB&lt;/a&gt; or
&lt;a href="https://clickhouse.com/"&gt;ClickHouse&lt;/a&gt; which were not designed to be extended.&lt;/p&gt;
&lt;h2 id="stringview"&gt;StringView&lt;a class="headerlink" href="#stringview" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;Performance improved for all queries when DataFusion switched to using Arrow
&lt;code&gt;StringView&lt;/code&gt;. Using &lt;code&gt;StringView&lt;/code&gt; “just” saves some copies and avoids one memory
access for certain comparisons. However, these copies and comparisons happen to
occur in many of the hottest loops during query processing, so optimizing them
resulted in measurable performance improvements.&lt;/p&gt;
&lt;p&gt;&lt;img alt="Illustration of how take works with StringView" class="img-fluid" src="/blog/images/clickbench-datafusion-43/string-view-take.png" width="80%"/&gt;&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Figure 3:&lt;/strong&gt; Figure from &lt;a href="https://www.influxdata.com/blog/faster-queries-with-stringview-part-one-influxdb/"&gt;Using StringView / German Style Strings to Make
Queries Faster: Part 1&lt;/a&gt; showing how &lt;code&gt;StringView&lt;/code&gt; saves copying data in many cases.&lt;/p&gt;
&lt;p&gt;Using StringView to make DataFusion faster for ClickBench required substantial
careful, low level optimization work described in &lt;a href="https://www.influxdata.com/blog/faster-queries-with-stringview-part-one-influxdb/"&gt;Using StringView / German
Style Strings to Make Queries Faster: Part 1&lt;/a&gt; and &lt;a href="https://www.influxdata.com/blog/faster-queries-with-stringview-part-two-influxdb/"&gt;Part 2&lt;/a&gt;. However, it &lt;em&gt;also&lt;/em&gt;
required extending the rest of DataFusion’s operations to support the new type.
You can get a sense of the magnitude of the work required by looking at the 100+
pull requests linked to the epic in arrow-rs
(&lt;a href="https://github.com/apache/arrow-rs/issues/5374"&gt;here&lt;/a&gt;) and three major epics
(&lt;a href="https://github.com/apache/datafusion/issues/10918"&gt;here&lt;/a&gt;,
&lt;a href="https://github.com/apache/datafusion/issues/11790"&gt;here&lt;/a&gt; and
&lt;a href="https://github.com/apache/datafusion/issues/11752"&gt;here&lt;/a&gt;) in DataFusion.&lt;/p&gt;
&lt;p&gt;Here is a partial list of people involved in the project (I am sorry to those whom I forgot)&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;strong&gt;Arrow&lt;/strong&gt;:  &lt;a href="https://github.com/XiangpengHao"&gt;Xiangpeng Hao&lt;/a&gt; (InfluxData’s amazing 2024 summer intern and UW Madison PhD), &lt;a href="https://github.com/ariesdevil"&gt;Yijun Zhao&lt;/a&gt; from DataBend Labs, and &lt;a href="https://github.com/tustvold"&gt;Raphael Taylor-Davies&lt;/a&gt; laid the foundation.  &lt;a href="https://github.com/RinChanNOWWW"&gt;RinChanNOW&lt;/a&gt; from Tencent and &lt;a href="https://github.com/a10y"&gt;Andrew Duffy&lt;/a&gt; from SpiralDB helped push it along in the early days, and &lt;a href="https://github.com/viirya"&gt;Liang-Chi Hsieh&lt;/a&gt;, &lt;a href="https://github.com/Dandandan"&gt;Daniël Heres&lt;/a&gt; reviewed and provided guidance.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;DataFusion&lt;/strong&gt;:  &lt;a href="https://github.com/XiangpengHao"&gt;Xiangpeng Hao&lt;/a&gt;, again charted the initial path and &lt;a href="https://github.com/Weijun-H"&gt;Weijun Huang&lt;/a&gt;, &lt;a href="https://github.com/dharanad"&gt;Dharan Aditya&lt;/a&gt; &lt;a href="https://github.com/Lordworms"&gt;Lordworms&lt;/a&gt;, &lt;a href="https://github.com/goldmedal"&gt;Jax Liu&lt;/a&gt;,  &lt;a href="https://github.com/wiedld"&gt;wiedld&lt;/a&gt;, &lt;a href="https://github.com/tlm365"&gt;Tai Le Manh&lt;/a&gt;, &lt;a href="https://github.com/my-vegetable-has-exploded"&gt;yi wang&lt;/a&gt;, &lt;a href="https://github.com/doupache"&gt;doupache&lt;/a&gt;, &lt;a href="https://github.com/jayzhan211"&gt;Jay Zhan&lt;/a&gt; , &lt;a href="https://github.com/xinlifoobar"&gt;Xin Li&lt;/a&gt;  and &lt;a href="https://github.com/Kev1n8"&gt;Kaifeng Zheng&lt;/a&gt; made it real.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;DataFusion String Function Migration&lt;/strong&gt;:  &lt;a href="https://github.com/tshauck"&gt;Trent Hauck&lt;/a&gt; organized the effort and set the patterns, &lt;a href="https://github.com/goldmedal"&gt;Jax Liu&lt;/a&gt; made a clever testing framework, and &lt;a href="https://github.com/austin362667"&gt;Austin Liu&lt;/a&gt;, &lt;a href="https://github.com/demetribu"&gt;Dmitrii Bu&lt;/a&gt;, &lt;a href="https://github.com/tlm365"&gt;Tai Le Manh&lt;/a&gt;, &lt;a href="https://github.com/PsiACE"&gt;Chojan Shang&lt;/a&gt;, &lt;a href="https://github.com/devanbenz"&gt;WeblWabl&lt;/a&gt;, &lt;a href="https://github.com/Lordworms"&gt;Lordworms&lt;/a&gt;, &lt;a href="https://github.com/thinh2"&gt;iamthinh&lt;/a&gt;, &lt;a href="https://github.com/Omega359"&gt;Bruce Ritchie&lt;/a&gt;, &lt;a href="https://github.com/Kev1n8"&gt;Kaifeng Zheng&lt;/a&gt;, and &lt;a href="https://github.com/xinlifoobar"&gt;Xin Li&lt;/a&gt; bashed out the conversions.&lt;/li&gt;
&lt;/ul&gt;
&lt;h2 id="parquet"&gt;Parquet&lt;a class="headerlink" href="#parquet" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;Part of the reason for DataFusion's speed in ClickBench is reading Parquet files (really) quickly,
which reflects invested effort in the Parquet reading system (see &lt;a href="https://www.influxdata.com/blog/querying-parquet-millisecond-latency/"&gt;Querying
Parquet with Millisecond Latency&lt;/a&gt; )&lt;/p&gt;
&lt;p&gt;The &lt;a href="https://docs.rs/datafusion/latest/datafusion/datasource/physical_plan/parquet/struct.ParquetExec.html"&gt;DataFusion ParquetExec&lt;/a&gt; (built on the &lt;a href="https://crates.io/crates/parquet"&gt;Rust Parquet Implementation&lt;/a&gt;) is now the most
sophisticated open source Parquet reader I know of. It has every optimization we
can think of for reading Parquet, including projection pushdown, predicate
pushdown (row group metadata, page index, and bloom filters), limit pushdown,
parallel reading, interleaved I/O, and late materialized filtering (coming soon ™️
by default). Some recent work from &lt;a href="https://github.com/itsjunetime"&gt;June&lt;/a&gt;
&lt;a href="https://github.com/apache/datafusion/pull/12135"&gt;recently unblocked a remaining hurdle&lt;/a&gt; for enabling late materialized
filtering, and conveniently &lt;a href="https://github.com/XiangpengHao"&gt;Xiangpeng Hao&lt;/a&gt; is
working on the &lt;a href="https://github.com/apache/arrow-datafusion/issues/3463"&gt;final piece&lt;/a&gt; (no pressure😅)&lt;/p&gt;
&lt;h2 id="skipping-partial-aggregation-when-it-doesnt-help"&gt;Skipping Partial Aggregation When It Doesn't Help&lt;a class="headerlink" href="#skipping-partial-aggregation-when-it-doesnt-help" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;Many ClickBench queries are aggregations that summarize millions of rows, a
common task for reporting and dashboarding. DataFusion uses state of the art
&lt;a href="https://docs.rs/datafusion/latest/datafusion/physical_plan/trait.Accumulator.html#tymethod.state"&gt;two phase aggregation&lt;/a&gt; plans. Normally, two phase aggregation works well as the
first phase consolidates many rows immediately after reading, while the data is
still in cache. However, for certain “high cardinality” aggregate queries (that
have large numbers of groups), &lt;a href="https://github.com/apache/datafusion/issues/6937"&gt;the two phase aggregation strategy used in
DataFusion was inefficient&lt;/a&gt;,
manifesting in relatively slower performance compared to other engines for
ClickBench queries such as&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-sql"&gt;SELECT "WatchID", "ClientIP", COUNT(*) AS c, ... 
FROM hits 
GROUP BY "WatchID", "ClientIP" /* &amp;lt;----- 13M Distinct Groups!!! */
ORDER BY c DESC 
LIMIT 10;
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;For such queries, the first aggregation phase does not significantly
reduce the number of rows, which wastes significant effort. &lt;a href="https://github.com/korowa"&gt;Eduard
Karacharov&lt;/a&gt; contributed a &lt;a href="https://github.com/apache/datafusion/pull/11627"&gt;dynamic strategy&lt;/a&gt; to
bypass the first phase when it is not working efficiently, shown in Figure 4.&lt;/p&gt;
&lt;p&gt;&lt;img alt="Two phase aggregation diagram from DataFusion API docs annotated to show first phase not helping" class="img-fluid" src="/blog/images/clickbench-datafusion-43/skipping-partial-aggregation.png" width="100%"/&gt;&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Figure 4&lt;/strong&gt;: Diagram from &lt;a href="https://docs.rs/datafusion/latest/datafusion/physical_plan/trait.Accumulator.html#tymethod.state"&gt;DataFusion API docs&lt;/a&gt; showing when the multi-phase
grouping is not effective&lt;/p&gt;
&lt;h2 id="optimized-multi-column-grouping"&gt;Optimized Multi-Column Grouping&lt;a class="headerlink" href="#optimized-multi-column-grouping" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;Another method for improving analytic database performance is specialized (aka
highly optimized) versions of operations for different data types, which the
system picks at runtime based on the query. Like other systems, DataFusion has
specialized code for handling different types of group columns. For example,
there is &lt;a href="https://github.com/apache/datafusion/blob/73507c307487708deb321e1ba4e0d302084ca27e/datafusion/physical-plan/src/aggregates/group_values/single_group_by/primitive.rs"&gt;special code&lt;/a&gt; that handles &lt;code&gt;GROUP BY int_id&lt;/code&gt;  and &lt;a href="https://github.com/apache/datafusion/blob/73507c307487708deb321e1ba4e0d302084ca27e/datafusion/physical-plan/src/aggregates/group_values/single_group_by/bytes.rs"&gt;different special
code&lt;/a&gt; that handles &lt;code&gt;GROUP BY string_id&lt;/code&gt; .&lt;/p&gt;
&lt;p&gt;When a query groups by multiple columns, it is tricker to apply this technique.
For example &lt;code&gt;GROUP BY string_id, int_id&lt;/code&gt; and &lt;code&gt;GROUP BY int_id, string_id&lt;/code&gt; have
different optimal structures, but it is not possible to include specialized
versions for all possible combinations of group column types.&lt;/p&gt;
&lt;p&gt;DataFusion includes &lt;a href="https://github.com/apache/datafusion/blob/73507c307487708deb321e1ba4e0d302084ca27e/datafusion/physical-plan/src/aggregates/group_values/row.rs#L33-L39"&gt;a general Row based mechanism&lt;/a&gt; that works for any
combination of column types, but this general mechanism copies each value twice
as shown in Figure 5. The cost of this copy &lt;a href="https://github.com/apache/datafusion/issues/9403"&gt;is especially high for variable
length strings and binary data&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;&lt;img alt="Row based storage for multiple group columns" class="img-fluid" src="/blog/images/clickbench-datafusion-43/row-based-storage.png" width="100%"/&gt;&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Figure 5&lt;/strong&gt;: Prior to DataFusion 43.0.0, queries with multiple group columns
used Row based group storage and copied each group value twice. This copy
consumes a substantial amount of the query time for queries with many distinct
groups, such as several of the queries in ClickBench.&lt;/p&gt;
&lt;p&gt;Many optimizations in Databases boil down to simply avoiding copies, and this
was no exception. The trick was to figure out how to avoid copies without
causing per-column comparison overhead to dominate or complexity to get out of
hand. In a great example of diligent and disciplined engineering, &lt;a href="https://github.com/jayzhan211"&gt;Jay
Zhan&lt;/a&gt; tried &lt;a href="https://github.com/apache/datafusion/pull/10937"&gt;several&lt;/a&gt;, &lt;a href="https://github.com/apache/datafusion/pull/10976"&gt;different&lt;/a&gt; approaches until arriving
at the [one shipped in DataFusion &lt;code&gt;43.0.0&lt;/code&gt;], shown in Figure 6.&lt;/p&gt;
&lt;p&gt;&lt;img alt="Column based storage for multiple group columns" class="img-fluid" src="/blog/images/clickbench-datafusion-43/column-based-storage.png" width="100%"/&gt;&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Figure 6&lt;/strong&gt;: DataFusion 43.0.0’s new columnar group storage copies each group
value exactly once, which is significantly faster when grouping by multiple
columns.&lt;/p&gt;
&lt;p&gt;Huge thanks as well to &lt;a href="https://github.com/eejbyfeldt"&gt;Emil Ejbyfeldt&lt;/a&gt; and
&lt;a href="https://github.com/Dandandan"&gt;Daniël Heres&lt;/a&gt; for their help reviewing and to
&lt;a href="https://github.com/Rachelint"&gt;Rachelint (kamille&lt;/a&gt;) for reviewing and
contributing a faster &lt;a href="https://github.com/apache/datafusion/pull/12996"&gt;vectorized append and compare for multiple groups&lt;/a&gt; which
will be released in DataFusion 44. The discussion on &lt;a href="https://github.com/apache/datafusion/issues/9403"&gt;the ticket&lt;/a&gt; is another
great example of the power of the DataFusion community working together to build
great software.&lt;/p&gt;
&lt;h1 id="whats-next"&gt;What’s Next 🚀&lt;a class="headerlink" href="#whats-next" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h1&gt;
&lt;p&gt;Just as I expect the performance of other engines to improve, DataFusion has
several more performance improvements lined up itself:&lt;/p&gt;
&lt;ol&gt;
&lt;li&gt;&lt;a href="https://github.com/apache/datafusion/pull/11943#top"&gt;Intermediate results blocked management&lt;/a&gt; (thanks again &lt;a href="https://github.com/Rachelint"&gt;Rachelint (kamille&lt;/a&gt;)&lt;/li&gt;
&lt;li&gt;&lt;a href="https://github.com/apache/datafusion/issues/3463"&gt;Enable parquet filter pushdown by default&lt;/a&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;We are also talking about what to focus on over the &lt;a href="https://github.com/apache/datafusion/issues/13274"&gt;next three
months&lt;/a&gt; and are always
looking for people to help! If you want to geek out (obsess??) about performance
and other features with engineers from around the world, &lt;a href="https://datafusion.apache.org/contributor-guide/communication.html"&gt;we would love you to
join us&lt;/a&gt;.&lt;/p&gt;
&lt;h1 id="additional-thanks"&gt;Additional Thanks&lt;a class="headerlink" href="#additional-thanks" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h1&gt;
&lt;p&gt;In addition to the people called out above, thanks:&lt;/p&gt;
&lt;ol&gt;
&lt;li&gt;&lt;a href="https://github.com/pmcgleenon"&gt;Patrick McGleenon&lt;/a&gt; for running ClickBench and gathering this data (&lt;a href="https://github.com/apache/datafusion/issues/13099#issuecomment-2478314793"&gt;source&lt;/a&gt;).&lt;/li&gt;
&lt;li&gt;Everyone I missed in the shoutouts – there are so many of you. We appreciate everyone.&lt;/li&gt;
&lt;/ol&gt;
&lt;h1 id="conclusion"&gt;Conclusion&lt;a class="headerlink" href="#conclusion" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h1&gt;
&lt;p&gt;I have dreamed about DataFusion being on top of the ClickBench leaderboard for
several years. I often watched with envy improvements in systems backed by large
VC investments, internet companies, or world class research institutions, and
doubted that we could pull off something similar in an open source project with
always limited time.&lt;/p&gt;
&lt;p&gt;The fact that we have now surpassed those other systems in query performance I
think speaks to the power and possibility of focusing on community and aligning
our collective enthusiasm and skills towards a common goal. Of course, being on
the top in any particular benchmark is likely fleeting as other engines will
improve, but so will DataFusion!&lt;/p&gt;
&lt;p&gt;I love working on DataFusion – the people, the quality of the code, my
interactions and the results we have achieved together far surpass my
expectations as well as most of my other software development experiences. I
can’t wait to see what people will build next, and hope to &lt;a href="https://github.com/apache/datafusion"&gt;see you
online&lt;/a&gt;.&lt;/p&gt;
&lt;h2 id="notes"&gt;Notes&lt;a class="headerlink" href="#notes" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;[^1]: Note that DuckDB is slightly faster on the ‘cold’ run.&lt;/p&gt;
&lt;p&gt;[^2]: Want to try your hand at a custom format for ClickBench fame / glory?: &lt;a href="https://github.com/apache/datafusion/issues/13448"&gt;Make DataFusion the fastest engine in ClickBench with custom file format&lt;/a&gt;&lt;/p&gt;
&lt;p&gt;[^3]: We have contributors from North America, South American, Europe, Asia, Africa and Australia&lt;/p&gt;
&lt;p&gt;[^4]: Undergraduates, PhD, Junior engineers, and getting-kind-of-crotchety experienced engineers&lt;/p&gt;
&lt;p&gt;[^5]: Thanks to Andy Pavlo, I love that nomenclature&lt;/p&gt;</content><category term="blog"/></entry><entry><title>Apache DataFusion Comet 0.3.0 Release</title><link href="https://datafusion.apache.org/blog/2024/09/27/datafusion-comet-0.3.0" rel="alternate"/><published>2024-09-27T00:00:00+00:00</published><updated>2024-09-27T00:00:00+00:00</updated><author><name>pmc</name></author><id>tag:datafusion.apache.org,2024-09-27:/blog/2024/09/27/datafusion-comet-0.3.0</id><summary type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;p&gt;The Apache DataFusion PMC is pleased to announce version 0.3.0 of the &lt;a href="https://datafusion.apache.org/comet/"&gt;Comet&lt;/a&gt; subproject.&lt;/p&gt;
&lt;p&gt;Comet is an accelerator for Apache Spark that translates Spark physical plans to DataFusion physical plans for
improved performance and efficiency without requiring any code changes.&lt;/p&gt;
&lt;p&gt;Comet runs on commodity hardware and aims to …&lt;/p&gt;</summary><content type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;p&gt;The Apache DataFusion PMC is pleased to announce version 0.3.0 of the &lt;a href="https://datafusion.apache.org/comet/"&gt;Comet&lt;/a&gt; subproject.&lt;/p&gt;
&lt;p&gt;Comet is an accelerator for Apache Spark that translates Spark physical plans to DataFusion physical plans for
improved performance and efficiency without requiring any code changes.&lt;/p&gt;
&lt;p&gt;Comet runs on commodity hardware and aims to provide 100% compatibility with Apache Spark. Any operators or
expressions that are not fully compatible will fall back to Spark unless explicitly enabled by the user. Refer
to the &lt;a href="https://datafusion.apache.org/comet/user-guide/compatibility.html"&gt;compatibility guide&lt;/a&gt; for more information.&lt;/p&gt;
&lt;p&gt;This release covers approximately four weeks of development work and is the result of merging 57 PRs from 12 
contributors. See the &lt;a href="https://github.com/apache/datafusion-comet/blob/main/dev/changelog/0.3.0.md"&gt;change log&lt;/a&gt; for more information.&lt;/p&gt;
&lt;h2 id="release-highlights"&gt;Release Highlights&lt;a class="headerlink" href="#release-highlights" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;h3 id="binary-releases"&gt;Binary Releases&lt;a class="headerlink" href="#binary-releases" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;Comet jar files are now published to Maven central for amd64 and arm64 architectures (Linux only).&lt;/p&gt;
&lt;p&gt;Files can be found at https://central.sonatype.com/search?q=org.apache.datafusion&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;Spark versions 3.3, 3.4, and 3.5 are supported.&lt;/li&gt;
&lt;li&gt;Scala versions 2.12 and 2.13 are supported.&lt;/li&gt;
&lt;/ul&gt;
&lt;h3 id="new-features"&gt;New Features&lt;a class="headerlink" href="#new-features" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;The following expressions are now supported natively:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;code&gt;DateAdd&lt;/code&gt;&lt;/li&gt;
&lt;li&gt;&lt;code&gt;DateSub&lt;/code&gt;&lt;/li&gt;
&lt;li&gt;&lt;code&gt;ElementAt&lt;/code&gt;&lt;/li&gt;
&lt;li&gt;&lt;code&gt;GetArrayElement&lt;/code&gt;&lt;/li&gt;
&lt;li&gt;&lt;code&gt;ToJson&lt;/code&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;h3 id="performance-stability"&gt;Performance &amp;amp; Stability&lt;a class="headerlink" href="#performance-stability" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;ul&gt;
&lt;li&gt;Upgraded to DataFusion 42.0.0&lt;/li&gt;
&lt;li&gt;Reduced memory overhead due to some memory leaks being fixed&lt;/li&gt;
&lt;li&gt;Comet will now fall back to Spark for queries that use DPP, to avoid performance regressions because Comet does 
  not have native support for DPP yet&lt;/li&gt;
&lt;li&gt;Improved performance when converting Spark columnar data to Arrow format&lt;/li&gt;
&lt;li&gt;Faster decimal sum and avg functions &lt;/li&gt;
&lt;/ul&gt;
&lt;h3 id="documentation-updates"&gt;Documentation Updates&lt;a class="headerlink" href="#documentation-updates" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;ul&gt;
&lt;li&gt;Improved documentation for deploying Comet with Kubernetes and Helm in the &lt;a href="https://datafusion.apache.org/comet/user-guide/kubernetes.html"&gt;Comet Kubernetes Guide&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;More detailed architectural overview of Comet scan and execution in the &lt;a href="https://datafusion.apache.org/comet/contributor-guide/plugin_overview.html"&gt;Comet Plugin Overview&lt;/a&gt; in the contributor guide&lt;/li&gt;
&lt;/ul&gt;
&lt;h2 id="getting-involved"&gt;Getting Involved&lt;a class="headerlink" href="#getting-involved" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;The Comet project welcomes new contributors. We use the same &lt;a href="https://datafusion.apache.org/contributor-guide/communication.html#slack-and-discord"&gt;Slack and Discord&lt;/a&gt; channels as the main DataFusion
project.&lt;/p&gt;
&lt;p&gt;The easiest way to get involved is to test Comet with your current Spark jobs and file issues for any bugs or
performance regressions that you find. See the &lt;a href="https://datafusion.apache.org/comet/user-guide/installation.html"&gt;Getting Started&lt;/a&gt; guide for instructions on downloading and installing
Comet.&lt;/p&gt;
&lt;p&gt;There are also many &lt;a href="https://github.com/apache/datafusion-comet/contribute"&gt;good first issues&lt;/a&gt; waiting for contributions.&lt;/p&gt;</content><category term="blog"/></entry><entry><title>Using StringView / German Style Strings to Make Queries Faster: Part 1- Reading Parquet</title><link href="https://datafusion.apache.org/blog/2024/09/13/string-view-german-style-strings-part-1" rel="alternate"/><published>2024-09-13T00:00:00+00:00</published><updated>2024-09-13T00:00:00+00:00</updated><author><name>Xiangpeng Hao, Andrew Lamb</name></author><id>tag:datafusion.apache.org,2024-09-13:/blog/2024/09/13/string-view-german-style-strings-part-1</id><summary type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;p&gt;&lt;em&gt;Editor's Note: This is the first of a &lt;a href="../string-view-german-style-strings-part-2/"&gt;two part&lt;/a&gt; blog series that was first published on the &lt;a href="https://www.influxdata.com/blog/faster-queries-with-stringview-part-one-influxdb/"&gt;InfluxData blog&lt;/a&gt;. Thanks to InfluxData for sponsoring this work as &lt;a href="https://haoxp.xyz/"&gt;Xiangpeng Hao&lt;/a&gt;'s summer intern project&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;This blog describes our experience implementing &lt;a href="https://arrow.apache.org/docs/format/Columnar.html#variable-size-binary-view-layout"&gt;StringView&lt;/a&gt; in the &lt;a href="https://github.com/apache/arrow-rs"&gt;Rust implementation&lt;/a&gt; of &lt;a href="https://arrow.apache.org/"&gt;Apache Arrow&lt;/a&gt;, and integrating …&lt;/p&gt;</summary><content type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;p&gt;&lt;em&gt;Editor's Note: This is the first of a &lt;a href="../string-view-german-style-strings-part-2/"&gt;two part&lt;/a&gt; blog series that was first published on the &lt;a href="https://www.influxdata.com/blog/faster-queries-with-stringview-part-one-influxdb/"&gt;InfluxData blog&lt;/a&gt;. Thanks to InfluxData for sponsoring this work as &lt;a href="https://haoxp.xyz/"&gt;Xiangpeng Hao&lt;/a&gt;'s summer intern project&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;This blog describes our experience implementing &lt;a href="https://arrow.apache.org/docs/format/Columnar.html#variable-size-binary-view-layout"&gt;StringView&lt;/a&gt; in the &lt;a href="https://github.com/apache/arrow-rs"&gt;Rust implementation&lt;/a&gt; of &lt;a href="https://arrow.apache.org/"&gt;Apache Arrow&lt;/a&gt;, and integrating it into &lt;a href="https://datafusion.apache.org/"&gt;Apache DataFusion&lt;/a&gt;, significantly accelerating string-intensive queries in the &lt;a href="https://benchmark.clickhouse.com/"&gt;ClickBench&lt;/a&gt; benchmark by 20%- 200% (Figure 1[^1]).&lt;/p&gt;
&lt;p&gt;Getting significant end-to-end performance improvements was non-trivial. Implementing StringView itself was only a fraction of the effort required. Among other things, we had to optimize UTF-8 validation, implement unintuitive compiler optimizations, tune block sizes, and time GC to realize the &lt;a href="https://www.influxdata.com/blog/flight-datafusion-arrow-parquet-fdap-architecture-influxdb/"&gt;FDAP ecosystem&lt;/a&gt;’s benefit. With other members of the open source community, we were able to overcome performance bottlenecks that could have killed the project. We would like to contribute by explaining the challenges and solutions in more detail so that more of the community can learn from our experience.&lt;/p&gt;
&lt;p&gt;StringView is based on a simple idea: avoid some string copies and accelerate comparisons with inlined prefixes. Like most great ideas, it is “obvious” only after &lt;a href="https://db.in.tum.de/~freitag/papers/p29-neumann-cidr20.pdf"&gt;someone describes it clearly&lt;/a&gt;. Although simple, straightforward implementation actually &lt;em&gt;slows down performance for almost every query&lt;/em&gt;. We must, therefore, apply astute observations and diligent engineering to realize the actual benefits from StringView.&lt;/p&gt;
&lt;p&gt;Although this journey was successful, not all research ideas are as lucky. To accelerate the adoption of research into industry, it is valuable to integrate research prototypes with practical systems. Understanding the nuances of real-world systems makes it more likely that research designs[^2] will lead to practical system improvements.&lt;/p&gt;
&lt;p&gt;StringView support was released as part of &lt;a href="https://crates.io/crates/arrow/52.2.0"&gt;arrow-rs v52.2.0&lt;/a&gt; and &lt;a href="https://crates.io/crates/datafusion/41.0.0"&gt;DataFusion v41.0.0&lt;/a&gt;. You can try it by setting the &lt;code&gt;schema_force_view_types&lt;/code&gt; &lt;a href="https://datafusion.apache.org/user-guide/configs.html"&gt;DataFusion configuration option&lt;/a&gt;, and we are&lt;a href="https://github.com/apache/datafusion/issues/11682"&gt; hard at work with the community to &lt;/a&gt;make it the default. We invite everyone to try it out, take advantage of the effort invested so far, and contribute to making it better.&lt;/p&gt;
&lt;p&gt;&lt;img alt="End to end performance improvements for ClickBench queries" class="img-fluid" src="/blog/images/string-view-1/figure1-performance.png" width="100%"/&gt;&lt;/p&gt;
&lt;p&gt;Figure 1: StringView improves string-intensive ClickBench query performance by 20% - 200%&lt;/p&gt;
&lt;h2 id="what-is-stringview"&gt;What is StringView?&lt;a class="headerlink" href="#what-is-stringview" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;&lt;img alt="Diagram of using StringArray and StringViewArray to represent the same string content" class="img-fluid" src="/blog/images/string-view-1/figure2-string-view.png" width="100%"/&gt;&lt;/p&gt;
&lt;p&gt;Figure 2: Use StringArray and StringViewArray to represent the same string content.&lt;/p&gt;
&lt;p&gt;The concept of inlined strings with prefixes (called “German Strings” &lt;a href="https://x.com/andy_pavlo/status/1813258735965643203"&gt;by Andy Pavlo&lt;/a&gt;, in homage to &lt;a href="https://www.tum.de/"&gt;TUM&lt;/a&gt;, where the &lt;a href="https://db.in.tum.de/~freitag/papers/p29-neumann-cidr20.pdf"&gt;Umbra paper that describes&lt;/a&gt; them originated) 
has been used in many recent database systems (&lt;a href="https://engineering.fb.com/2024/02/20/developer-tools/velox-apache-arrow-15-composable-data-management/"&gt;Velox&lt;/a&gt;, &lt;a href="https://pola.rs/posts/polars-string-type/"&gt;Polars&lt;/a&gt;, &lt;a href="https://duckdb.org/2021/12/03/duck-arrow.html"&gt;DuckDB&lt;/a&gt;, &lt;a href="https://cedardb.com/blog/german_strings/"&gt;CedarDB&lt;/a&gt;, etc.) 
and was introduced to Arrow as a new &lt;a href="https://arrow.apache.org/docs/format/Columnar.html#variable-size-binary-view-layout"&gt;StringViewArray&lt;/a&gt;[^3] type. Arrow’s original &lt;a href="https://arrow.apache.org/docs/format/Columnar.html#variable-size-binary-layout"&gt;StringArray&lt;/a&gt; is very memory efficient but less effective for certain operations. 
StringViewArray accelerates string-intensive operations via prefix inlining and a more flexible and compact string representation.&lt;/p&gt;
&lt;p&gt;A StringViewArray consists of three components:&lt;/p&gt;
&lt;ol&gt;
&lt;li&gt;The &lt;code&gt;&lt;em&gt;view&lt;/em&gt;&lt;/code&gt; array&lt;/li&gt;
&lt;li&gt;The buffers&lt;/li&gt;
&lt;li&gt;The buffer pointers (IDs) that map buffer offsets to their physical locations&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;Each &lt;code&gt;view&lt;/code&gt; is 16 bytes long, and its contents differ based on the string’s length:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;string length &amp;lt; 12 bytes: the first four bytes store the string length, and the remaining 12 bytes store the inlined string.&lt;/li&gt;
&lt;li&gt;string length &amp;gt; 12 bytes: the string is stored in a separate buffer. The length is again stored in the first 4 bytes, followed by the buffer id (4 bytes), the buffer offset (4 bytes), and the prefix (first 4 bytes) of the string.&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;Figure 2 shows an example of the same logical content (left) using StringArray (middle) and StringViewArray (right):&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;The first string – &lt;code&gt;"Apache DataFusion"&lt;/code&gt; – is 17 bytes long, and both StringArray and StringViewArray store the string’s bytes at the beginning of the buffer. The StringViewArray also inlines the first 4 bytes – &lt;code&gt;"Apac"&lt;/code&gt; – in the view.&lt;/li&gt;
&lt;li&gt;The second string, &lt;code&gt;"InfluxDB"&lt;/code&gt; is only 8 bytes long, so StringViewArray completely inlines the string content in the &lt;code&gt;view&lt;/code&gt; struct while StringArray stores the string in the buffer as well.&lt;/li&gt;
&lt;li&gt;The third string &lt;code&gt;"Arrow Rust Impl"&lt;/code&gt; is 15 bytes long and cannot be fully inlined. StringViewArray stores this in the same form as the first string.&lt;/li&gt;
&lt;li&gt;The last string &lt;code&gt;"Apache DataFusion"&lt;/code&gt; has the same content as the first string. It’s possible to use StringViewArray to avoid this duplication and reuse the bytes by pointing the view to the previous location.&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;StringViewArray provides three opportunities for outperforming StringArray:&lt;/p&gt;
&lt;ol&gt;
&lt;li&gt;Less copying via the offset + buffer format&lt;/li&gt;
&lt;li&gt;Faster comparisons using the inlined string prefix&lt;/li&gt;
&lt;li&gt;Reusing repeated string values with the flexible &lt;code&gt;view&lt;/code&gt; layout&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;The rest of this blog post discusses how to apply these opportunities in real query scenarios to improve performance, what challenges we encountered along the way, and how we solved them.&lt;/p&gt;
&lt;h2 id="faster-parquet-loading"&gt;Faster Parquet Loading&lt;a class="headerlink" href="#faster-parquet-loading" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;&lt;a href="https://parquet.apache.org/"&gt;Apache Parquet&lt;/a&gt; is the de facto format for storing large-scale analytical data commonly stored LakeHouse-style, such as &lt;a href="https://iceberg.apache.org"&gt;Apache Iceberg&lt;/a&gt; and &lt;a href="https://delta.io"&gt;Delta Lake&lt;/a&gt;. Efficiently loading data from Parquet is thus critical to query performance in many important real-world workloads.&lt;/p&gt;
&lt;p&gt;Parquet encodes strings (i.e., &lt;a href="https://docs.rs/parquet/latest/parquet/data_type/struct.ByteArray.html"&gt;byte array&lt;/a&gt;) in a slightly different format than required for the original Arrow StringArray. The string length is encoded inline with the actual string data (as shown in Figure 4 left). As mentioned previously, StringArray requires the data buffer to be continuous and compact—the strings have to follow one after another. This requirement means that reading Parquet string data into an Arrow StringArray requires copying and consolidating the string bytes to a new buffer and tracking offsets in a separate array. Copying these strings is often wasteful. Typical queries filter out most data immediately after loading, so most of the copied data is quickly discarded.&lt;/p&gt;
&lt;p&gt;On the other hand, reading Parquet data as a StringViewArray can re-use the same data buffer as storing the Parquet pages because StringViewArray does not require strings to be contiguous. For example, in Figure 4, the StringViewArray directly references the buffer with the decoded Parquet page. The string &lt;code&gt;"Arrow Rust Impl"&lt;/code&gt; is represented by a &lt;code&gt;view&lt;/code&gt; with offset 37 and length 15 into that buffer.&lt;/p&gt;
&lt;p&gt;&lt;img alt="Diagram showing how StringViewArray can avoid copying by reusing decoded Parquet pages." class="img-fluid" src="/blog/images/string-view-1/figure4-copying.png" width="100%"/&gt;&lt;/p&gt;
&lt;p&gt;Figure 4: StringViewArray avoids copying by reusing decoded Parquet pages.&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Mini benchmark&lt;/strong&gt;&lt;/p&gt;
&lt;p&gt;Reusing Parquet buffers is great in theory, but how much does saving a copy actually matter? We can run the following benchmark in arrow-rs to find out:&lt;/p&gt;
&lt;p&gt;Our benchmarking machine shows that loading &lt;em&gt;BinaryViewArray&lt;/em&gt; is almost 2x faster than loading BinaryArray (see next section about why this isn’t &lt;em&gt;String&lt;/em&gt; ViewArray).&lt;/p&gt;
&lt;p&gt;You can read more on this arrow-rs issue: &lt;a href="https://github.com/apache/arrow-rs/issues/5904"&gt;https://github.com/apache/arrow-rs/issues/5904&lt;/a&gt;&lt;/p&gt;
&lt;h1 id="from-binary-to-strings"&gt;From Binary to Strings&lt;a class="headerlink" href="#from-binary-to-strings" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h1&gt;
&lt;p&gt;You may wonder why we reported performance for BinaryViewArray when this post is about StringViewArray. Surprisingly, initially, our implementation to read StringViewArray from Parquet was much &lt;em&gt;slower&lt;/em&gt; than StringArray. Why? TLDR: Although reading StringViewArray copied less data, the initial implementation also spent much more time validating &lt;a href="https://en.wikipedia.org/wiki/UTF-8#:~:text=UTF%2D8%20is%20a%20variable,Unicode%20Standard"&gt;UTF-8&lt;/a&gt; (as shown in Figure 5).&lt;/p&gt;
&lt;p&gt;Strings are stored as byte sequences. When reading data from (potentially untrusted) Parquet files, a Parquet decoder must ensure those byte sequences are valid UTF-8 strings, and most programming languages, including Rust, include highly&lt;a href="https://doc.rust-lang.org/std/str/fn.from_utf8.html"&gt; optimized routines&lt;/a&gt; for doing so.&lt;/p&gt;
&lt;p&gt;&lt;img alt="Figure showing time to load strings from Parquet and the effect of optimized UTF-8 validation." class="img-fluid" src="/blog/images/string-view-1/figure5-loading-strings.png" width="100%"/&gt;&lt;/p&gt;
&lt;p&gt;Figure 5: Time to load strings from Parquet. The UTF-8 validation advantage initially eliminates the advantage of reduced copying for StringViewArray.&lt;/p&gt;
&lt;p&gt;A StringArray can be validated in a single call to the UTF-8 validation function as it has a continuous string buffer. As long as the underlying buffer is UTF-8[^4], all strings in the array must be UTF-8. The Rust parquet reader makes a single function call to validate the entire buffer.&lt;/p&gt;
&lt;p&gt;However, validating an arbitrary StringViewArray requires validating each string with a separate call to the validation function, as the underlying buffer may also contain non-string data (for example, the lengths in Parquet pages).&lt;/p&gt;
&lt;p&gt;UTF-8 validation in Rust is highly optimized and favors longer strings (as shown in Figure 6), likely because it leverages SIMD instructions to perform parallel validation. The benefit of a single function call to validate UTF-8 over a function call for each string more than eliminates the advantage of avoiding the copy for StringViewArray.&lt;/p&gt;
&lt;p&gt;&lt;img alt="Figure showing UTF-8 validation throughput vs string length." class="img-fluid" src="/blog/images/string-view-1/figure6-utf8-validation.png" width="100%"/&gt;&lt;/p&gt;
&lt;p&gt;Figure 6: UTF-8 validation throughput vs string length—StringArray’s contiguous buffer can be validated much faster than StringViewArray’s buffer.&lt;/p&gt;
&lt;p&gt;Does this mean we should only use StringArray? No! Thankfully, there’s a clever way out. The key observation is that in many real-world datasets,&lt;a href="https://www.vldb.org/pvldb/vol17/p148-zeng.pdf"&gt; 99% of strings are shorter than 128 bytes&lt;/a&gt;, meaning the encoded length values are smaller than 128, &lt;strong&gt;in which case the length itself is also valid UTF-8&lt;/strong&gt; (in fact, it is &lt;a href="https://en.wikipedia.org/wiki/ASCII"&gt;ASCII&lt;/a&gt;).&lt;/p&gt;
&lt;p&gt;This observation means we can optimize validating UTF-8 strings in Parquet pages by treating the length bytes as part of a single large string as long as the length &lt;em&gt;value&lt;/em&gt; is less than 128. Put another way, prior to this optimization, the length bytes act as string boundaries, which require a UTF-8 validation on each string. After this optimization, only those strings with lengths larger than 128 bytes (less than 1% of the strings in the ClickBench dataset) are string boundaries, significantly increasing the UTF-8 validation chunk size and thus improving performance.&lt;/p&gt;
&lt;p&gt;The &lt;a href="https://github.com/apache/arrow-rs/pull/6009/files"&gt;actual implementation&lt;/a&gt; is only nine lines of Rust (with 30 lines of comments). You can find more details in the related arrow-rs issue:&lt;a href="https://github.com/apache/arrow-rs/issues/5995"&gt; https://github.com/apache/arrow-rs/issues/5995&lt;/a&gt;. As expected, with this optimization, loading StringViewArray is almost 2x faster than loading StringArray.&lt;/p&gt;
&lt;h1 id="be-careful-about-implicit-copies"&gt;Be Careful About Implicit Copies&lt;a class="headerlink" href="#be-careful-about-implicit-copies" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h1&gt;
&lt;p&gt;After all the work to avoid copying strings when loading from Parquet, performance was still not as good as expected. We tracked the problem to a few implicit data copies that we weren't aware of, as described in&lt;a href="https://github.com/apache/arrow-rs/issues/6033"&gt; this issue&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;The copies we eventually identified come from the following innocent-looking line of Rust code, where &lt;code&gt;self.buf&lt;/code&gt; is a &lt;a href="https://en.wikipedia.org/wiki/Reference_counting"&gt;reference counted&lt;/a&gt; pointer that should transform without copying into a buffer for use in StringViewArray.&lt;/p&gt;
&lt;p&gt;However, Rust-type coercion rules favored a blanket implementation that &lt;em&gt;did&lt;/em&gt; copy data. This implementation is shown in the following code block where the &lt;code&gt;impl&amp;lt;T: AsRef&amp;lt;[u8]&amp;gt;&amp;gt;&lt;/code&gt; will accept any type that implements &lt;code&gt;AsRef&amp;lt;[u8]&amp;gt;&lt;/code&gt; and copies the data to create a new buffer. To avoid copying, users need to explicitly call &lt;code&gt;from_vec&lt;/code&gt;, which consumes the &lt;code&gt;Vec&lt;/code&gt; and transforms it into a buffer.&lt;/p&gt;
&lt;p&gt;Diagnosing this implicit copy was time-consuming as it relied on subtle Rust language semantics. We needed to track every step of the data flow to ensure every copy was necessary. To help other users and prevent future mistakes, we also &lt;a href="https://github.com/apache/arrow-rs/pull/6043"&gt;removed&lt;/a&gt; the implicit API from arrow-rs in favor of an explicit API. Using this approach, we found and fixed several &lt;a href="https://github.com/apache/arrow-rs/pull/6039"&gt;other unintentional copies&lt;/a&gt; in the code base—hopefully, the change will help other &lt;a href="https://github.com/spiraldb/vortex/pull/504"&gt;downstream users&lt;/a&gt; avoid unnecessary copies.&lt;/p&gt;
&lt;h1 id="help-the-compiler-by-giving-it-more-information"&gt;Help the Compiler by Giving it More Information&lt;a class="headerlink" href="#help-the-compiler-by-giving-it-more-information" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h1&gt;
&lt;p&gt;The Rust compiler’s automatic optimizations mostly work very well for a wide variety of use cases, but sometimes, it needs additional hints to generate the most efficient code. When profiling the performance of &lt;code&gt;view&lt;/code&gt; construction, we found, counterintuitively, that constructing &lt;strong&gt;long&lt;/strong&gt; strings was 10x faster than constructing &lt;strong&gt;short&lt;/strong&gt; strings, which made short strings slower on StringViewArray than on StringArray!&lt;/p&gt;
&lt;p&gt;As described in the first section, StringViewArray treats long and short strings differently. Short strings (&amp;lt;12 bytes) directly inline to the &lt;code&gt;view&lt;/code&gt; struct, while long strings only inline the first 4 bytes. The code to construct a &lt;code&gt;view&lt;/code&gt; looks something like this:&lt;/p&gt;
&lt;p&gt;It appears that both branches of the code should be fast: they both involve copying at most 16 bytes of data and some memory shift/store operations. How could the branch for short strings be 10x slower?&lt;/p&gt;
&lt;p&gt;Looking at the assembly code using &lt;a href="https://godbolt.org/"&gt;Compiler Explorer&lt;/a&gt;, we (with help from &lt;a href="https://github.com/aoli-al"&gt;Ao Li&lt;/a&gt;) found the compiler used CPU &lt;strong&gt;load instructions&lt;/strong&gt; to copy the fixed-sized 4 bytes to the &lt;code&gt;view&lt;/code&gt; for long strings, but it calls a function, &lt;a href="https://doc.rust-lang.org/std/ptr/fn.copy_nonoverlapping.html"&gt;&lt;code&gt;ptr::copy_non_overlapping&lt;/code&gt;&lt;/a&gt;, to copy the inlined bytes to the &lt;code&gt;view&lt;/code&gt; for short strings. The difference is that long strings have a prefix size (4 bytes) known at compile time, so the compiler directly uses efficient CPU instructions. But, since the size of the short string is unknown to the compiler, it has to call the general-purpose function &lt;code&gt;ptr::copy_non_coverlapping&lt;/code&gt;. Making a function call is significant unnecessary overhead compared to a CPU copy instruction.&lt;/p&gt;
&lt;p&gt;However, we know something the compiler doesn’t know: the short string size is not arbitrary—it must be between 0 and 12 bytes, and we can leverage this information to avoid the function call. Our solution generates 13 copies of the function using generics, one for each of the possible prefix lengths. The code looks as follows, and &lt;a href="https://godbolt.org/z/685YPsd5G"&gt;checking the assembly code&lt;/a&gt;, we confirmed there are no calls to &lt;code&gt;ptr::copy_non_overlapping&lt;/code&gt;, and only native CPU instructions are used. For more details, see &lt;a href="https://github.com/apache/arrow-rs/issues/6034"&gt;the ticket&lt;/a&gt;.&lt;/p&gt;
&lt;h1 id="end-to-end-query-performance"&gt;End-to-End Query Performance&lt;a class="headerlink" href="#end-to-end-query-performance" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h1&gt;
&lt;p&gt;In the previous sections, we went out of our way to make sure loading StringViewArray is faster than StringArray. Before going further, we wanted to verify if obsessing about reducing copies and function calls has actually improved end-to-end performance in real-life queries. To do this, we evaluated a ClickBench query (Q20) in DataFusion that counts how many URLs contain the word &lt;code&gt;"google"&lt;/code&gt;:&lt;/p&gt;
&lt;p&gt;This is a relatively simple query; most of the time is spent on loading the “URL” column to find matching rows. The query plan looks like this:&lt;/p&gt;
&lt;p&gt;We ran the benchmark in the DataFusion repo like this:&lt;/p&gt;
&lt;p&gt;With StringViewArray we saw a 24% end-to-end performance improvement, as shown in Figure 7. With the &lt;code&gt;--string-view&lt;/code&gt; argument, the end-to-end query time is &lt;code&gt;944.3 ms, 869.6 ms, 861.9 ms&lt;/code&gt; (three iterations). Without &lt;code&gt;--string-view&lt;/code&gt;, the end-to-end query time is &lt;code&gt;1186.1 ms, 1126.1 ms, 1138.3 ms&lt;/code&gt;.&lt;/p&gt;
&lt;p&gt;&lt;img alt="Figure showing StringView improves end to end performance by 24 percent." class="img-fluid" src="/blog/images/string-view-1/figure7-end-to-end.png" width="100%"/&gt;&lt;/p&gt;
&lt;p&gt;Figure 7: StringView reduces end-to-end query time by 24% on ClickBench Q20.&lt;/p&gt;
&lt;p&gt;We also double-checked with detailed profiling and verified that the time reduction is indeed due to faster Parquet loading.&lt;/p&gt;
&lt;h2 id="conclusion"&gt;Conclusion&lt;a class="headerlink" href="#conclusion" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;In this first blog post, we have described what it took to improve the
performance of simply reading strings from Parquet files using StringView. While
this resulted in real end-to-end query performance improvements, in our &lt;a href="https://datafusion.apache.org/blog/2024/09/13//string-view-german-style-strings-part-2"&gt;next
post&lt;/a&gt;, we explore additional optimizations enabled by StringView in DataFusion,
along with some of the pitfalls we encountered while implementing them.&lt;/p&gt;
&lt;h1 id="footnotes"&gt;Footnotes&lt;a class="headerlink" href="#footnotes" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h1&gt;
&lt;p&gt;[^1]: Benchmarked with AMD Ryzen 7600x (12 core, 24 threads, 32 MiB L3), WD Black SN770 NVMe SSD (5150MB/4950MB seq RW bandwidth)&lt;/p&gt;
&lt;p&gt;[^2]: Xiangpeng is a PhD student at the University of Wisconsin-Madison&lt;/p&gt;
&lt;p&gt;[^3]: There is also a corresponding &lt;em&gt;BinaryViewArray&lt;/em&gt; which is similar except that the data is not constrained to be UTF-8 encoded strings.&lt;/p&gt;
&lt;p&gt;[^4]: We also make sure that offsets do not break a UTF-8 code point, which is &lt;a href="https://github.com/apache/arrow-rs/blob/master/parquet/src/arrow/buffer/offset_buffer.rs#L62-L71"&gt;cheaply validated&lt;/a&gt;.&lt;/p&gt;</content><category term="blog"/></entry><entry><title>Using StringView / German Style Strings to make Queries Faster: Part 2 - String Operations</title><link href="https://datafusion.apache.org/blog/2024/09/13/string-view-german-style-strings-part-2" rel="alternate"/><published>2024-09-13T00:00:00+00:00</published><updated>2024-09-13T00:00:00+00:00</updated><author><name>Xiangpeng Hao, Andrew Lamb</name></author><id>tag:datafusion.apache.org,2024-09-13:/blog/2024/09/13/string-view-german-style-strings-part-2</id><summary type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;p&gt;&lt;em&gt;Editor's Note: This blog series was first published on the &lt;a href="https://www.influxdata.com/blog/faster-queries-with-stringview-part-two-influxdb/"&gt;InfluxData blog&lt;/a&gt;. Thanks to InfluxData for sponsoring this work as &lt;a href="https://haoxp.xyz/"&gt;Xiangpeng Hao&lt;/a&gt;'s summer intern project&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;In the &lt;a href="https://datafusion.apache.org/blog/2024/09/13/string-view-german-style-strings-part-1"&gt;first post&lt;/a&gt;, we discussed the nuances required to accelerate Parquet loading using StringViewArray by reusing buffers and reducing copies. 
In this second …&lt;/p&gt;</summary><content type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;p&gt;&lt;em&gt;Editor's Note: This blog series was first published on the &lt;a href="https://www.influxdata.com/blog/faster-queries-with-stringview-part-two-influxdb/"&gt;InfluxData blog&lt;/a&gt;. Thanks to InfluxData for sponsoring this work as &lt;a href="https://haoxp.xyz/"&gt;Xiangpeng Hao&lt;/a&gt;'s summer intern project&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;In the &lt;a href="https://datafusion.apache.org/blog/2024/09/13/string-view-german-style-strings-part-1"&gt;first post&lt;/a&gt;, we discussed the nuances required to accelerate Parquet loading using StringViewArray by reusing buffers and reducing copies. 
In this second part of the post, we describe the rest of the journey: implementing additional efficient operations for real query processing.&lt;/p&gt;
&lt;h2 id="faster-string-operations"&gt;Faster String Operations&lt;a class="headerlink" href="#faster-string-operations" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;h1 id="faster-comparison"&gt;Faster comparison&lt;a class="headerlink" href="#faster-comparison" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h1&gt;
&lt;p&gt;String comparison is ubiquitous; it is the core of 
&lt;a href="https://docs.rs/arrow/latest/arrow/compute/kernels/cmp/index.html"&gt;&lt;code&gt;cmp&lt;/code&gt;&lt;/a&gt;, 
&lt;a href="https://docs.rs/arrow/latest/arrow/compute/fn.min.html"&gt;&lt;code&gt;min&lt;/code&gt;&lt;/a&gt;/&lt;a href="https://docs.rs/arrow/latest/arrow/compute/fn.max.html"&gt;&lt;code&gt;max&lt;/code&gt;&lt;/a&gt;, 
and &lt;a href="https://docs.rs/arrow/latest/arrow/compute/kernels/comparison/fn.like.html"&gt;&lt;code&gt;like&lt;/code&gt;&lt;/a&gt;/&lt;a href="https://docs.rs/arrow/latest/arrow/compute/kernels/comparison/fn.ilike.html"&gt;&lt;code&gt;ilike&lt;/code&gt;&lt;/a&gt; kernels. StringViewArray is designed to accelerate such comparisons using the inlined prefix—the key observation is that, in many cases, only the first few bytes of the string determine the string comparison results.&lt;/p&gt;
&lt;p&gt;For example, to compare the strings &lt;code&gt;InfluxDB&lt;/code&gt; with &lt;code&gt;Apache DataFusion&lt;/code&gt;, we only need to look at the first byte to determine the string ordering or equality. In this case, since &lt;code&gt;A&lt;/code&gt; is earlier in the alphabet than &lt;code&gt;I,&lt;/code&gt; &lt;code&gt;Apache DataFusion&lt;/code&gt; sorts first, and we know the strings are not equal. Despite only needing the first byte, comparing these strings when stored as a StringArray requires two memory accesses: 1) load the string offset and 2) use the offset to locate the string bytes. For low-level operations such as &lt;code&gt;cmp&lt;/code&gt; that are invoked millions of times in the very hot paths of queries, avoiding this extra memory access can make a measurable difference in query performance.&lt;/p&gt;
&lt;p&gt;For StringViewArray, typically, only one memory access is needed to load the view struct. Only if the result can not be determined from the prefix is the second memory access required. For the example above, there is no need for the second access. This technique is very effective in practice: the second access is never necessary for the more than &lt;a href="https://www.vldb.org/pvldb/vol17/p148-zeng.pdf"&gt;60% of real-world strings which are shorter than 12 bytes&lt;/a&gt;, as they are stored completely in the prefix.&lt;/p&gt;
&lt;p&gt;However, functions that operate on strings must be specialized to take advantage of the inlined prefix. In addition to low-level comparison kernels, we implemented &lt;a href="https://github.com/apache/arrow-rs/issues/5374"&gt;a wide range&lt;/a&gt; of other StringViewArray operations that cover the functions and operations seen in ClickBench queries. Supporting StringViewArray in all string operations takes quite a bit of effort, and thankfully the Arrow and DataFusion communities are already hard at work doing so (see &lt;a href="https://github.com/apache/datafusion/issues/11752"&gt;https://github.com/apache/datafusion/issues/11752&lt;/a&gt; if you want to help out).&lt;/p&gt;
&lt;h1 id="faster-takeandfilter"&gt;Faster &lt;code&gt;take&lt;/code&gt;and&lt;code&gt;filter&lt;/code&gt;&lt;a class="headerlink" href="#faster-takeandfilter" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h1&gt;
&lt;p&gt;After a filter operation such as &lt;code&gt;WHERE url &amp;lt;&amp;gt; ''&lt;/code&gt; to avoid processing empty urls, DataFusion will often &lt;em&gt;coalesce&lt;/em&gt; results to form a new array with only the passing elements. 
This coalescing ensures the batches are sufficiently sized to benefit from &lt;a href="https://www.vldb.org/pvldb/vol11/p2209-kersten.pdf"&gt;vectorized processing&lt;/a&gt; in subsequent steps.&lt;/p&gt;
&lt;p&gt;The coalescing operation is implemented using the &lt;a href="https://docs.rs/arrow/latest/arrow/compute/fn.take.html"&gt;take&lt;/a&gt; and &lt;a href="https://arrow.apache.org/rust/arrow/compute/kernels/filter/fn.filter.html"&gt;filter&lt;/a&gt; kernels in arrow-rs. For StringArray, these kernels require copying the string contents to a new buffer without “holes” in between. This copy can be expensive especially when the new array is large.&lt;/p&gt;
&lt;p&gt;However, &lt;code&gt;take&lt;/code&gt; and &lt;code&gt;filter&lt;/code&gt; for StringViewArray can avoid the copy by reusing buffers from the old array. The kernels only need to create a new list of  &lt;code&gt;view&lt;/code&gt;s that point at the same strings within the old buffers. 
Figure 1 illustrates the difference between the output of both string representations. StringArray creates two new strings at offsets 0-17 and 17-32, while StringViewArray simply points to the original buffer at offsets 0 and 25.&lt;/p&gt;
&lt;p&gt;&lt;img alt="Diagram showing Zero-copy &amp;lt;code&amp;gt;take&amp;lt;/code&amp;gt;/&amp;lt;code&amp;gt;filter&amp;lt;/code&amp;gt; for StringViewArray" class="img-fluid" src="/blog/images/string-view-2/figure1-zero-copy-take.png" width="100%"/&gt;&lt;/p&gt;
&lt;p&gt;Figure 1: Zero-copy &lt;code&gt;take&lt;/code&gt;/&lt;code&gt;filter&lt;/code&gt; for StringViewArray&lt;/p&gt;
&lt;h1 id="when-to-gc"&gt;When to GC?&lt;a class="headerlink" href="#when-to-gc" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h1&gt;
&lt;p&gt;Zero-copy &lt;code&gt;take/filter&lt;/code&gt; is great for generating large arrays quickly, but it is suboptimal for highly selective filters, where most of the strings are filtered out. When the cardinality drops, StringViewArray buffers become sparse—only a small subset of the bytes in the buffer’s memory are referred to by any &lt;code&gt;view&lt;/code&gt;. This leads to excessive memory usage, especially in a &lt;a href="https://github.com/apache/datafusion/issues/11628"&gt;filter-then-coalesce scenario&lt;/a&gt;. For example, a StringViewArray with 10M strings may only refer to 1M strings after some filter operations; however, due to zero-copy take/filter, the (reused) 10M buffers can not be released/reused.&lt;/p&gt;
&lt;p&gt;To release unused memory, we implemented a &lt;a href="https://docs.rs/arrow/latest/arrow/array/struct.GenericByteViewArray.html#method.gc"&gt;garbage collection (GC)&lt;/a&gt; routine to consolidate the data into a new buffer to release the old sparse buffer(s). As the GC operation copies strings, similarly to StringArray, we must be careful about when to call it. If we call GC too early, we cause unnecessary copying, losing much of the benefit of StringViewArray. If we call GC too late, we hold large buffers for too long, increasing memory use and decreasing cache efficiency. The &lt;a href="https://pola.rs/posts/polars-string-type/"&gt;Polars blog&lt;/a&gt; on StringView also refers to the challenge presented by garbage collection timing.&lt;/p&gt;
&lt;p&gt;&lt;code&gt;arrow-rs&lt;/code&gt; implements the GC process, but it is up to users to decide when to call it. We leverage the semantics of the query engine and observed that the &lt;a href="https://docs.rs/datafusion/latest/datafusion/physical_plan/coalesce_batches/struct.CoalesceBatchesExec.html"&gt;&lt;code&gt;CoalesceBatchesExec&lt;/code&gt;&lt;/a&gt; operator, which merge smaller batches to a larger batch, is often used after the record cardinality is expected to shrink, which aligns perfectly with the scenario of GC in StringViewArray. 
We, therefore,&lt;a href="https://github.com/apache/datafusion/pull/11587"&gt; implemented the GC procedure&lt;/a&gt; inside &lt;code&gt;CoalesceBatchesExec&lt;/code&gt;[^5] with a heuristic that estimates when the buffers are too sparse.&lt;/p&gt;
&lt;h2 id="the-art-of-function-inlining-not-too-much-not-too-little"&gt;The art of function inlining: not too much, not too little&lt;a class="headerlink" href="#the-art-of-function-inlining-not-too-much-not-too-little" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;Like string inlining, &lt;em&gt;function&lt;/em&gt; inlining is the process of embedding a short function into the caller to avoid the overhead of function calls (caller/callee save). 
Usually, the Rust compiler does a good job of deciding when to inline. However, it is possible to override its default using the &lt;a href="https://doc.rust-lang.org/reference/attributes/codegen.html#the-inline-attribute"&gt;&lt;code&gt;#[inline(always)]&lt;/code&gt; directive&lt;/a&gt;. 
In performance-critical code, inlined code allows us to organize large functions into smaller ones without paying the runtime cost of function invocation.&lt;/p&gt;
&lt;p&gt;However, function inlining is &lt;strong&gt;&lt;em&gt;not&lt;/em&gt;&lt;/strong&gt; always better, as it leads to larger function bodies that are harder for LLVM to optimize (for example, suboptimal &lt;a href="https://en.wikipedia.org/wiki/Register_allocation"&gt;register spilling&lt;/a&gt;) and risk overflowing the CPU’s instruction cache. We observed several performance regressions where function inlining caused &lt;em&gt;slower&lt;/em&gt; performance when implementing the StringViewArray comparison kernels. Careful inspection and tuning of the code was required to aid the compiler in generating efficient code. More details can be found in this PR: &lt;a href="https://github.com/apache/arrow-rs/pull/5900"&gt;https://github.com/apache/arrow-rs/pull/5900&lt;/a&gt;.&lt;/p&gt;
&lt;h2 id="buffer-size-tuning"&gt;Buffer size tuning&lt;a class="headerlink" href="#buffer-size-tuning" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;StringViewArray permits multiple buffers, which enables a flexible buffer layout and potentially reduces the need to copy data. However, a large number of buffers slows down the performance of other operations. 
For example, &lt;a href="https://docs.rs/arrow/latest/arrow/array/trait.Array.html#tymethod.get_array_memory_size"&gt;&lt;code&gt;get_array_memory_size&lt;/code&gt;&lt;/a&gt; needs to sum the memory size of each buffer, which takes a long time with thousands of small buffers. 
In certain cases, we found that multiple calls to &lt;a href="https://docs.rs/arrow/latest/arrow/compute/fn.concat_batches.html"&gt;&lt;code&gt;concat_batches&lt;/code&gt;&lt;/a&gt; lead to arrays with millions of buffers, which was prohibitively expensive.&lt;/p&gt;
&lt;p&gt;For example, consider a StringViewArray with the previous default buffer size of 8 KB. With this configuration, holding 4GB of string data requires almost half a million buffers! Larger buffer sizes are needed for larger arrays, but we cannot arbitrarily increase the default buffer size, as small arrays would consume too much memory (most arrays require at least one buffer). Buffer sizing is especially problematic in query processing, as we often need to construct small batches of string arrays, and the sizes are unknown at planning time.&lt;/p&gt;
&lt;p&gt;To balance the buffer size trade-off, we again leverage the query processing (DataFusion) semantics to decide when to use larger buffers. While coalescing batches, we combine multiple small string arrays and set a smaller buffer size to keep the total memory consumption low. In string aggregation, we aggregate over an entire Datafusion partition, which can generate a large number of strings, so we set a larger buffer size (2MB).&lt;/p&gt;
&lt;p&gt;To assist situations where the semantics are unknown, we also &lt;a href="https://github.com/apache/arrow-rs/pull/6136"&gt;implemented&lt;/a&gt; a classic dynamic exponential buffer size growth strategy, which starts with a small buffer size (8KB) and doubles the size of each new buffer up to 2MB. We implemented this strategy in arrow-rs and enabled it by default so that other users of StringViewArray can also benefit from this optimization. See this issue for more details: &lt;a href="https://github.com/apache/arrow-rs/issues/6094"&gt;https://github.com/apache/arrow-rs/issues/6094&lt;/a&gt;.&lt;/p&gt;
&lt;h2 id="end-to-end-query-performance"&gt;End-to-end query performance&lt;a class="headerlink" href="#end-to-end-query-performance" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;We have made significant progress in optimizing StringViewArray filtering operations. Now, let’s test it in the real world to see how it works!&lt;/p&gt;
&lt;p&gt;Let’s consider ClickBench query 22, which selects multiple string fields (&lt;code&gt;URL&lt;/code&gt;, &lt;code&gt;Title&lt;/code&gt;, and &lt;code&gt;SearchPhase&lt;/code&gt;) and applies several filters.&lt;/p&gt;
&lt;p&gt;We ran the benchmark using the following command in the DataFusion repo. Again, the &lt;code&gt;--string-view&lt;/code&gt; option means we use StringViewArray instead of StringArray.&lt;/p&gt;
&lt;p&gt;To eliminate the impact of the faster Parquet reading using StringViewArray (see the first part of this blog), Figure 2 plots only the time spent in &lt;code&gt;FilterExec&lt;/code&gt;. Without StringViewArray, the filter takes 7.17s; with StringViewArray, the filter only takes 4.86s, a 32% reduction in time. Moreover, we see a 17% improvement in end-to-end query performance.&lt;/p&gt;
&lt;p&gt;&lt;img alt="Figure showing StringViewArray reduces the filter time by 32% on ClickBench query 22." class="img-fluid" src="/blog/images/string-view-2/figure2-filter-time.png" width="100%"/&gt;&lt;/p&gt;
&lt;p&gt;Figure 2: StringViewArray reduces the filter time by 32% on ClickBench query 22.&lt;/p&gt;
&lt;h1 id="faster-string-aggregation"&gt;Faster String Aggregation&lt;a class="headerlink" href="#faster-string-aggregation" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h1&gt;
&lt;p&gt;So far, we have discussed how to exploit two StringViewArray features: reduced copy and faster filtering. This section focuses on reusing string bytes to repeat string values.&lt;/p&gt;
&lt;p&gt;As described in part one of this blog, if two strings have identical values, StringViewArray can use two different &lt;code&gt;view&lt;/code&gt;s pointing at the same buffer range, thus avoiding repeating the string bytes in the buffer. This makes StringViewArray similar to an Arrow &lt;a href="https://docs.rs/arrow/latest/arrow/array/struct.DictionaryArray.html"&gt;DictionaryArray&lt;/a&gt; that stores Strings—both array types work well for strings with only a few distinct values.&lt;/p&gt;
&lt;p&gt;Deduplicating string values can significantly reduce memory consumption in StringViewArray. However, this process is expensive and involves hashing every string and maintaining a hash table, and so it cannot be done by default when creating a StringViewArray. We introduced an&lt;a href="https://docs.rs/arrow/latest/arrow/array/builder/struct.GenericByteViewBuilder.html#method.with_deduplicate_strings"&gt; opt-in string deduplication mode&lt;/a&gt; in arrow-rs for advanced users who know their data has a small number of distinct values, and where the benefits of reduced memory consumption outweigh the additional overhead of array construction.&lt;/p&gt;
&lt;p&gt;Once again, we leverage DataFusion query semantics to identify StringViewArray with duplicate values, such as aggregation queries with multiple group keys. For example, some &lt;a href="https://github.com/apache/datafusion/blob/main/benchmarks/queries/clickbench/queries.sql"&gt;ClickBench queries&lt;/a&gt; group by two columns:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;code&gt;UserID&lt;/code&gt; (an integer with close to 1 M distinct values)&lt;/li&gt;
&lt;li&gt;&lt;code&gt;MobilePhoneModel&lt;/code&gt; (a string with less than a hundred distinct values)&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;In this case, the output row count is&lt;code&gt;count(distinct UserID) * count(distinct MobilePhoneModel)&lt;/code&gt;,  which is 100M. Each string value of  &lt;code&gt;MobilePhoneModel&lt;/code&gt; is repeated 1M times. With StringViewArray, we can save space by pointing the repeating values to the same underlying buffer.&lt;/p&gt;
&lt;p&gt;Faster string aggregation with StringView is part of a larger project to &lt;a href="https://github.com/apache/datafusion/issues/7000"&gt;improve DataFusion aggregation performance&lt;/a&gt;. We have a &lt;a href="https://github.com/apache/datafusion/pull/11794"&gt;proof of concept implementation&lt;/a&gt; with StringView that can improve the multi-column string aggregation by 20%. We would love your help to get it production ready!&lt;/p&gt;
&lt;h1 id="stringview-pitfalls"&gt;StringView Pitfalls&lt;a class="headerlink" href="#stringview-pitfalls" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h1&gt;
&lt;p&gt;Most existing blog posts (including this one) focus on the benefits of using StringViewArray over other string representations such as StringArray. As we have discussed, even though it requires a significant engineering investment to realize, StringViewArray is a major improvement over StringArray in many cases.&lt;/p&gt;
&lt;p&gt;However, there are several cases where StringViewArray is slower than StringArray. For completeness, we have listed those instances here:&lt;/p&gt;
&lt;ol&gt;
&lt;li&gt;&lt;strong&gt;Tiny strings (when strings are shorter than 8 bytes)&lt;/strong&gt;: every element of the StringViewArray consumes at least 16 bytes of memory—the size of the &lt;code&gt;view&lt;/code&gt; struct. For an array of tiny strings, StringViewArray consumes more memory than StringArray and thus can cause slower performance due to additional memory pressure on the CPU cache.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Many repeated short strings&lt;/strong&gt;: Similar to the first point, StringViewArray can be slower and require more memory than a DictionaryArray because 1) it can only reuse the bytes in the buffer when the strings are longer than 12 bytes and 2) 32-bit offsets are always used, even when a smaller size (8 bit or 16 bit) could represent all the distinct values.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Filtering:&lt;/strong&gt; As we mentioned above, StringViewArrays often consume more memory than the corresponding StringArray, and memory bloat quickly dominates the performance without GC. However, invoking GC also reduces the benefits of less copying so must be carefully tuned.&lt;/li&gt;
&lt;/ol&gt;
&lt;h1 id="conclusion-and-takeaways"&gt;Conclusion and Takeaways&lt;a class="headerlink" href="#conclusion-and-takeaways" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h1&gt;
&lt;p&gt;In these two blog posts, we discussed what it takes to implement StringViewArray in arrow-rs and then integrate it into DataFusion. Our evaluations on ClickBench queries show that StringView can improve the performance of string-intensive workloads by up to 2x.&lt;/p&gt;
&lt;p&gt;Given that DataFusion already &lt;a href="https://benchmark.clickhouse.com/#eyJzeXN0ZW0iOnsiQWxsb3lEQiI6ZmFsc2UsIkF0aGVuYSAocGFydGl0aW9uZWQpIjpmYWxzZSwiQXRoZW5hIChzaW5nbGUpIjpmYWxzZSwiQXVyb3JhIGZvciBNeVNRTCI6ZmFsc2UsIkF1cm9yYSBmb3IgUG9zdGdyZVNRTCI6ZmFsc2UsIkJ5Q29uaXR5IjpmYWxzZSwiQnl0ZUhvdXNlIjpmYWxzZSwiY2hEQiAoUGFycXVldCwgcGFydGl0aW9uZWQpIjpmYWxzZSwiY2hEQiI6ZmFsc2UsIkNpdHVzIjpmYWxzZSwiQ2xpY2tIb3VzZSBDbG91ZCAoYXdzKSI6ZmFsc2UsIkNsaWNrSG91c2UgQ2xvdWQgKGF3cykgUGFyYWxsZWwgUmVwbGljYXMgT04iOmZhbHNlLCJDbGlja0hvdXNlIENsb3VkIChBenVyZSkiOmZhbHNlLCJDbGlja0hvdXNlIENsb3VkIChBenVyZSkgUGFyYWxsZWwgUmVwbGljYSBPTiI6ZmFsc2UsIkNsaWNrSG91c2UgQ2xvdWQgKEF6dXJlKSBQYXJhbGxlbCBSZXBsaWNhcyBPTiI6ZmFsc2UsIkNsaWNrSG91c2UgQ2xvdWQgKGdjcCkiOmZhbHNlLCJDbGlja0hvdXNlIENsb3VkIChnY3ApIFBhcmFsbGVsIFJlcGxpY2FzIE9OIjpmYWxzZSwiQ2xpY2tIb3VzZSAoZGF0YSBsYWtlLCBwYXJ0aXRpb25lZCkiOmZhbHNlLCJDbGlja0hvdXNlIChkYXRhIGxha2UsIHNpbmdsZSkiOmZhbHNlLCJDbGlja0hvdXNlIChQYXJxdWV0LCBwYXJ0aXRpb25lZCkiOmZhbHNlLCJDbGlja0hvdXNlIChQYXJxdWV0LCBzaW5nbGUpIjpmYWxzZSwiQ2xpY2tIb3VzZSAod2ViKSI6ZmFsc2UsIkNsaWNrSG91c2UiOmZhbHNlLCJDbGlja0hvdXNlICh0dW5lZCkiOmZhbHNlLCJDbGlja0hvdXNlICh0dW5lZCwgbWVtb3J5KSI6ZmFsc2UsIkNsb3VkYmVycnkiOmZhbHNlLCJDcmF0ZURCIjpmYWxzZSwiQ3J1bmNoeSBCcmlkZ2UgZm9yIEFuYWx5dGljcyAoUGFycXVldCkiOmZhbHNlLCJEYXRhYmVuZCI6ZmFsc2UsIkRhdGFGdXNpb24gKFBhcnF1ZXQsIHBhcnRpdGlvbmVkKSI6dHJ1ZSwiRGF0YUZ1c2lvbiAoUGFycXVldCwgc2luZ2xlKSI6ZmFsc2UsIkFwYWNoZSBEb3JpcyI6ZmFsc2UsIkRydWlkIjpmYWxzZSwiRHVja0RCIChQYXJxdWV0LCBwYXJ0aXRpb25lZCkiOnRydWUsIkR1Y2tEQiI6ZmFsc2UsIkVsYXN0aWNzZWFyY2giOmZhbHNlLCJFbGFzdGljc2VhcmNoICh0dW5lZCkiOmZhbHNlLCJHbGFyZURCIjpmYWxzZSwiR3JlZW5wbHVtIjpmYWxzZSwiSGVhdnlBSSI6ZmFsc2UsIkh5ZHJhIjpmYWxzZSwiSW5mb2JyaWdodCI6ZmFsc2UsIktpbmV0aWNhIjpmYWxzZSwiTWFyaWFEQiBDb2x1bW5TdG9yZSI6ZmFsc2UsIk1hcmlhREIiOmZhbHNlLCJNb25ldERCIjpmYWxzZSwiTW9uZ29EQiI6ZmFsc2UsIk1vdGhlcmR1Y2siOmZhbHNlLCJNeVNRTCAoTXlJU0FNKSI6ZmFsc2UsIk15U1FMIjpmYWxzZSwiT3hsYSI6ZmFsc2UsIlBhcmFkZURCIChQYXJxdWV0LCBwYXJ0aXRpb25lZCkiOmZhbHNlLCJQYXJhZGVEQiAoUGFycXVldCwgc2luZ2xlKSI6ZmFsc2UsIlBpbm90IjpmYWxzZSwiUG9zdGdyZVNRTCAodHVuZWQpIjpmYWxzZSwiUG9zdGdyZVNRTCI6ZmFsc2UsIlF1ZXN0REIgKHBhcnRpdGlvbmVkKSI6ZmFsc2UsIlF1ZXN0REIiOmZhbHNlLCJSZWRzaGlmdCI6ZmFsc2UsIlNlbGVjdERCIjpmYWxzZSwiU2luZ2xlU3RvcmUiOmZhbHNlLCJTbm93Zmxha2UiOmZhbHNlLCJTUUxpdGUiOmZhbHNlLCJTdGFyUm9ja3MiOmZhbHNlLCJUYWJsZXNwYWNlIjpmYWxzZSwiVGVtYm8gT0xBUCAoY29sdW1uYXIpIjpmYWxzZSwiVGltZXNjYWxlREIgKGNvbXByZXNzaW9uKSI6ZmFsc2UsIlRpbWVzY2FsZURCIjpmYWxzZSwiVW1icmEiOmZhbHNlfSwidHlwZSI6eyJDIjp0cnVlLCJjb2x1bW4tb3JpZW50ZWQiOnRydWUsIlBvc3RncmVTUUwgY29tcGF0aWJsZSI6dHJ1ZSwibWFuYWdlZCI6dHJ1ZSwiZ2NwIjp0cnVlLCJzdGF0ZWxlc3MiOnRydWUsIkphdmEiOnRydWUsIkMrKyI6dHJ1ZSwiTXlTUUwgY29tcGF0aWJsZSI6dHJ1ZSwicm93LW9yaWVudGVkIjp0cnVlLCJDbGlja0hvdXNlIGRlcml2YXRpdmUiOnRydWUsImVtYmVkZGVkIjp0cnVlLCJzZXJ2ZXJsZXNzIjp0cnVlLCJhd3MiOnRydWUsInBhcmFsbGVsIHJlcGxpY2FzIjp0cnVlLCJBenVyZSI6dHJ1ZSwiYW5hbHl0aWNhbCI6dHJ1ZSwiUnVzdCI6dHJ1ZSwic2VhcmNoIjp0cnVlLCJkb2N1bWVudCI6dHJ1ZSwic29tZXdoYXQgUG9zdGdyZVNRTCBjb21wYXRpYmxlIjp0cnVlLCJ0aW1lLXNlcmllcyI6dHJ1ZX0sIm1hY2hpbmUiOnsiMTYgdkNQVSAxMjhHQiI6dHJ1ZSwiOCB2Q1BVIDY0R0IiOnRydWUsInNlcnZlcmxlc3MiOnRydWUsIjE2YWN1Ijp0cnVlLCJjNmEuNHhsYXJnZSwgNTAwZ2IgZ3AyIjp0cnVlLCJMIjp0cnVlLCJNIjp0cnVlLCJTIjp0cnVlLCJYUyI6dHJ1ZSwiYzZhLm1ldGFsLCA1MDBnYiBncDIiOnRydWUsIjE5MkdCIjp0cnVlLCIyNEdCIjp0cnVlLCIzNjBHQiI6dHJ1ZSwiNDhHQiI6dHJ1ZSwiNzIwR0IiOnRydWUsIjk2R0IiOnRydWUsIjE0MzBHQiI6dHJ1ZSwiZGV2Ijp0cnVlLCI3MDhHQiI6dHJ1ZSwiYzVuLjR4bGFyZ2UsIDUwMGdiIGdwMiI6dHJ1ZSwiQW5hbHl0aWNzLTI1NkdCICg2NCB2Q29yZXMsIDI1NiBHQikiOnRydWUsImM1LjR4bGFyZ2UsIDUwMGdiIGdwMiI6dHJ1ZSwiYzZhLjR4bGFyZ2UsIDE1MDBnYiBncDIiOnRydWUsImNsb3VkIjp0cnVlLCJkYzIuOHhsYXJnZSI6dHJ1ZSwicmEzLjE2eGxhcmdlIjp0cnVlLCJyYTMuNHhsYXJnZSI6dHJ1ZSwicmEzLnhscGx1cyI6dHJ1ZSwiUzIiOnRydWUsIlMyNCI6dHJ1ZSwiMlhMIjp0cnVlLCIzWEwiOnRydWUsIjRYTCI6dHJ1ZSwiWEwiOnRydWUsIkwxIC0gMTZDUFUgMzJHQiI6dHJ1ZSwiYzZhLjR4bGFyZ2UsIDUwMGdiIGdwMyI6dHJ1ZX0sImNsdXN0ZXJfc2l6ZSI6eyIxIjp0cnVlLCIyIjp0cnVlLCI0Ijp0cnVlLCI4Ijp0cnVlLCIxNiI6dHJ1ZSwiMzIiOnRydWUsIjY0Ijp0cnVlLCIxMjgiOnRydWUsInNlcnZlcmxlc3MiOnRydWUsImRlZGljYXRlZCI6dHJ1ZX0sIm1ldHJpYyI6ImhvdCIsInF1ZXJpZXMiOlt0cnVlLHRydWUsdHJ1ZSx0cnVlLHRydWUsdHJ1ZSx0cnVlLHRydWUsdHJ1ZSx0cnVlLHRydWUsdHJ1ZSx0cnVlLHRydWUsdHJ1ZSx0cnVlLH"&gt;performs very well on ClickBench&lt;/a&gt;, the level of end-to-end performance improvement using StringViewArray shows the power of this technique and, of course, is a win for DataFusion and the systems that build upon it.&lt;/p&gt;
&lt;p&gt;StringView is a big project that has received tremendous community support. Specifically, we would like to thank &lt;a href="https://github.com/tustvold"&gt;@tustvold&lt;/a&gt;, &lt;a href="https://github.com/ariesdevil"&gt;@ariesdevil&lt;/a&gt;, &lt;a href="https://github.com/RinChanNOWWW"&gt;@RinChanNOWWW&lt;/a&gt;, &lt;a href="https://github.com/ClSlaid"&gt;@ClSlaid&lt;/a&gt;, &lt;a href="https://github.com/2010YOUY01"&gt;@2010YOUY01&lt;/a&gt;, &lt;a href="https://github.com/chloro-pn"&gt;@chloro-pn&lt;/a&gt;, &lt;a href="https://github.com/a10y"&gt;@a10y&lt;/a&gt;, &lt;a href="https://github.com/Kev1n8"&gt;@Kev1n8&lt;/a&gt;, &lt;a href="https://github.com/Weijun-H"&gt;@Weijun-H&lt;/a&gt;, &lt;a href="https://github.com/PsiACE"&gt;@PsiACE&lt;/a&gt;, &lt;a href="https://github.com/tshauck"&gt;@tshauck&lt;/a&gt;, and &lt;a href="https://github.com/xinlifoobar"&gt;@xinlifoobar&lt;/a&gt; for their valuable contributions!&lt;/p&gt;
&lt;p&gt;As the introduction states, “German Style Strings” is a relatively straightforward research idea that avoid some string copies and accelerates comparisons. However, applying this (great) idea in practice requires a significant investment in careful software engineering. Again, we encourage the research community to continue to help apply research ideas to industrial systems, such as DataFusion, as doing so provides valuable perspectives when evaluating future research questions for the greatest potential impact.&lt;/p&gt;
&lt;h3 id="footnotes"&gt;Footnotes&lt;a class="headerlink" href="#footnotes" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;[^5]: There are additional optimizations possible in this operation that the community is working on, such as  &lt;a href="https://github.com/apache/datafusion/issues/7957"&gt;https://github.com/apache/datafusion/issues/7957&lt;/a&gt;.&lt;/p&gt;</content><category term="blog"/></entry><entry><title>Apache DataFusion Comet 0.2.0 Release</title><link href="https://datafusion.apache.org/blog/2024/08/28/datafusion-comet-0.2.0" rel="alternate"/><published>2024-08-28T00:00:00+00:00</published><updated>2024-08-28T00:00:00+00:00</updated><author><name>pmc</name></author><id>tag:datafusion.apache.org,2024-08-28:/blog/2024/08/28/datafusion-comet-0.2.0</id><summary type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;p&gt;The Apache DataFusion PMC is pleased to announce version 0.2.0 of the &lt;a href="https://datafusion.apache.org/comet/"&gt;Comet&lt;/a&gt; subproject.&lt;/p&gt;
&lt;p&gt;Comet is an accelerator for Apache Spark that translates Spark physical plans to DataFusion physical plans for
improved performance and efficiency without requiring any code changes.&lt;/p&gt;
&lt;p&gt;Comet runs on commodity hardware and aims to …&lt;/p&gt;</summary><content type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;p&gt;The Apache DataFusion PMC is pleased to announce version 0.2.0 of the &lt;a href="https://datafusion.apache.org/comet/"&gt;Comet&lt;/a&gt; subproject.&lt;/p&gt;
&lt;p&gt;Comet is an accelerator for Apache Spark that translates Spark physical plans to DataFusion physical plans for
improved performance and efficiency without requiring any code changes.&lt;/p&gt;
&lt;p&gt;Comet runs on commodity hardware and aims to provide 100% compatibility with Apache Spark. Any operators or
expressions that are not fully compatible will fall back to Spark unless explicitly enabled by the user. Refer
to the &lt;a href="https://datafusion.apache.org/comet/user-guide/compatibility.html"&gt;compatibility guide&lt;/a&gt; for more information.&lt;/p&gt;
&lt;p&gt;This release covers approximately four weeks of development work and is the result of merging 87 PRs from 14 
contributors. See the &lt;a href="https://github.com/apache/datafusion-comet/blob/main/dev/changelog/0.2.0.md"&gt;change log&lt;/a&gt; for more information.&lt;/p&gt;
&lt;h2 id="release-highlights"&gt;Release Highlights&lt;a class="headerlink" href="#release-highlights" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;h3 id="docker-images"&gt;Docker Images&lt;a class="headerlink" href="#docker-images" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;Docker images are now available from the &lt;a href="https://github.com/apache/datafusion-comet/pkgs/container/datafusion-comet/265110454?tag=spark-3.4-scala-2.12-0.2.0"&gt;GitHub Container Registry&lt;/a&gt;.&lt;/p&gt;
&lt;h3 id="performance-improvements"&gt;Performance improvements&lt;a class="headerlink" href="#performance-improvements" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;ul&gt;
&lt;li&gt;Native shuffle is now enabled by default&lt;/li&gt;
&lt;li&gt;Improved handling of decimal types&lt;/li&gt;
&lt;li&gt;Reduced some redundant copying of batches in Filter/Scan operations&lt;/li&gt;
&lt;li&gt;Optimized performance of count aggregates&lt;/li&gt;
&lt;li&gt;Optimized performance of  CASE expressions for specific uses:&lt;/li&gt;
&lt;li&gt;CASE WHEN expr THEN column ELSE null END&lt;/li&gt;
&lt;li&gt;CASE WHEN expr THEN literal ELSE literal END&lt;/li&gt;
&lt;li&gt;Optimized performance of IS NOT NULL&lt;/li&gt;
&lt;/ul&gt;
&lt;h3 id="new-features"&gt;New Features&lt;a class="headerlink" href="#new-features" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;ul&gt;
&lt;li&gt;Window operations now support count and sum aggregates&lt;/li&gt;
&lt;li&gt;CreateArray&lt;/li&gt;
&lt;li&gt;GetStructField&lt;/li&gt;
&lt;li&gt;Support nested types in hash join&lt;/li&gt;
&lt;li&gt;Basic implementation of RLIKE expression&lt;/li&gt;
&lt;/ul&gt;
&lt;h2 id="current-performance"&gt;Current Performance&lt;a class="headerlink" href="#current-performance" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;We use benchmarks derived from the industry standard TPC-H and TPC-DS benchmarks for tracking progress with
performance. The following charts shows the time it takes to run the queries against 100 GB of data in
Parquet format using a single executor with eight cores. See the &lt;a href="https://datafusion.apache.org/comet/contributor-guide/benchmarking.html"&gt;Comet Benchmarking Guide&lt;/a&gt;
for details of the environment used for these benchmarks.&lt;/p&gt;
&lt;h3 id="benchmark-derived-from-tpc-h"&gt;Benchmark derived from TPC-H&lt;a class="headerlink" href="#benchmark-derived-from-tpc-h" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;Comet 0.2.0 provides a 62% speedup compared to Spark. This is slightly better than the Comet 0.1.0 release.&lt;/p&gt;
&lt;p&gt;&lt;img alt="Chart showing TPC-H benchmark results for Comet 0.2.0" class="img-fluid" src="/blog/images/comet-0.2.0/tpch_allqueries.png" width="100%"/&gt;&lt;/p&gt;
&lt;h3 id="benchmark-derived-from-tpc-ds"&gt;Benchmark derived from TPC-DS&lt;a class="headerlink" href="#benchmark-derived-from-tpc-ds" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;Comet 0.2.0 provides a 21% speedup compared to Spark, which is a significant improvement compared to 
Comet 0.1.0, which did not provide any speedup for this benchmark.&lt;/p&gt;
&lt;p&gt;&lt;img alt="Chart showing TPC-DS benchmark results for Comet 0.2.0" class="img-fluid" src="/blog/images/comet-0.2.0/tpcds_allqueries.png" width="100%"/&gt;&lt;/p&gt;
&lt;h2 id="getting-involved"&gt;Getting Involved&lt;a class="headerlink" href="#getting-involved" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;The Comet project welcomes new contributors. We use the same &lt;a href="https://datafusion.apache.org/contributor-guide/communication.html#slack-and-discord"&gt;Slack and Discord&lt;/a&gt; channels as the main DataFusion
project.&lt;/p&gt;
&lt;p&gt;The easiest way to get involved is to test Comet with your current Spark jobs and file issues for any bugs or
performance regressions that you find. See the &lt;a href="https://datafusion.apache.org/comet/user-guide/installation.html"&gt;Getting Started&lt;/a&gt; guide for instructions on downloading and installing
Comet.&lt;/p&gt;
&lt;p&gt;There are also many &lt;a href="https://github.com/apache/datafusion-comet/contribute"&gt;good first issues&lt;/a&gt; waiting for contributions.&lt;/p&gt;</content><category term="blog"/></entry><entry><title>Apache DataFusion Python 40.1.0 Released, Significant usability updates</title><link href="https://datafusion.apache.org/blog/2024/08/20/python-datafusion-40.0.0" rel="alternate"/><published>2024-08-20T00:00:00+00:00</published><updated>2024-08-20T00:00:00+00:00</updated><author><name>timsaucer</name></author><id>tag:datafusion.apache.org,2024-08-20:/blog/2024/08/20/python-datafusion-40.0.0</id><summary type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;h2 id="introduction"&gt;Introduction&lt;a class="headerlink" href="#introduction" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;We are happy to announce that &lt;a href="https://pypi.org/project/datafusion/40.1.0/"&gt;DataFusion in Python 40.1.0&lt;/a&gt; has been released. In addition to
bringing in all of the new features of the core &lt;a href="https://datafusion.apache.org/blog/2024/07/24/datafusion-40.0.0/"&gt;DataFusion 40.0.0&lt;/a&gt; package, this release
contains &lt;em&gt;significant&lt;/em&gt; updates to the user interface and documentation. We listened to the python …&lt;/p&gt;</summary><content type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;h2 id="introduction"&gt;Introduction&lt;a class="headerlink" href="#introduction" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;We are happy to announce that &lt;a href="https://pypi.org/project/datafusion/40.1.0/"&gt;DataFusion in Python 40.1.0&lt;/a&gt; has been released. In addition to
bringing in all of the new features of the core &lt;a href="https://datafusion.apache.org/blog/2024/07/24/datafusion-40.0.0/"&gt;DataFusion 40.0.0&lt;/a&gt; package, this release
contains &lt;em&gt;significant&lt;/em&gt; updates to the user interface and documentation. We listened to the python
user community to create a more &lt;em&gt;pythonic&lt;/em&gt; experience. If you have not used the python interface to
DataFusion before, this is an excellent time to give it a try!&lt;/p&gt;
&lt;h2 id="background"&gt;Background&lt;a class="headerlink" href="#background" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;Until now, the python bindings for DataFusion have primarily been a thin layer to expose the
underlying Rust functionality. This has been worked well for early adopters to use DataFusion
within their Python projects, but some users have found it difficult to work with. As compared to
other DataFrame libraries, these issues were raised:&lt;/p&gt;
&lt;ol&gt;
&lt;li&gt;Most of the functions had little or no documentation. Users often had to refer to the Rust
documentation or code to learn how to use DataFusion. This alienated some python users.&lt;/li&gt;
&lt;li&gt;Users could not take advantage of modern IDE features such as type hinting. These are valuable
tools for rapid testing and development.&lt;/li&gt;
&lt;li&gt;Some of the interfaces felt “clunky” to users since some Python concepts do not always map well
to their Rust counterparts.&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;This release aims to bring a better user experience to the DataFusion Python community.&lt;/p&gt;
&lt;h2 id="whats-changed"&gt;What's Changed&lt;a class="headerlink" href="#whats-changed" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;The most significant difference is that we have added wrapper functions and classes for most of the
user facing interface. These wrappers, written in Python, contain both documentation and type
annotations.&lt;/p&gt;
&lt;p&gt;This documentation is now available on the &lt;a href="https://datafusion.apache.org/python/autoapi/datafusion/index.html"&gt;DataFusion in Python API&lt;/a&gt; website. There you can browse
the available functions and classes to see the breadth of available functionality.&lt;/p&gt;
&lt;p&gt;Modern IDEs use language servers such as
&lt;a href="https://marketplace.visualstudio.com/items?itemName=ms-python.vscode-pylance"&gt;Pylance&lt;/a&gt; or
&lt;a href="https://jedi.readthedocs.io/en/latest/"&gt;Jedi&lt;/a&gt; to perform analysis of python code, provide useful
hints, and identify usage errors. These are major tools in the python user community. With this
release, users can fully use these tools in their workflow.&lt;/p&gt;
&lt;figure class="text-center"&gt;
&lt;img alt="Fig 1: Enhanced tooltips in an IDE." class="img-fluid" src="/blog/images/python-datafusion-40.0.0/vscode_hover_tooltip.png"/&gt;
&lt;figcaption&gt;
&lt;b&gt;Figure 1&lt;/b&gt;: With the enhanced python wrappers, users can see helpful tool tips with
   type annotations directly in modern IDEs.
&lt;/figcaption&gt;
&lt;/figure&gt;
&lt;p&gt;By having the type annotations, these IDEs can also identify quickly when a user has incorrectly
used a function's arguments as shown in Figure 2.&lt;/p&gt;
&lt;figure class="text-center"&gt;
&lt;img alt="Fig 2: Error checking in static analysis" class="img-fluid" src="/blog/images/python-datafusion-40.0.0/pylance_error_checking.png"/&gt;
&lt;figcaption&gt;
&lt;b&gt;Figure 2&lt;/b&gt;: Modern Python language servers can perform static analysis and quickly find
   errors in the arguments to functions.
&lt;/figcaption&gt;
&lt;/figure&gt;
&lt;p&gt;In addition to these wrapper libraries, we have enhancements to some of the functions to feel more
easy to use.&lt;/p&gt;
&lt;h3 id="improved-dataframe-filter-arguments"&gt;Improved DataFrame filter arguments&lt;a class="headerlink" href="#improved-dataframe-filter-arguments" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;You can now apply multiple &lt;code&gt;filter&lt;/code&gt; statements in a single step. When using &lt;code&gt;DataFrame.filter&lt;/code&gt; you
can pass in multiple arguments, separated by a comma. These will act as a logical &lt;code&gt;AND&lt;/code&gt; of all of
the filter arguments. The following two statements are equivalent:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-python"&gt;df.filter(col("size") &amp;lt; col("max_size")).filter(col("color") == lit("green"))
df.filter(col("size") &amp;lt; col("max_size"), col("color") == lit("green"))
&lt;/code&gt;&lt;/pre&gt;
&lt;h3 id="comparison-against-literal-values"&gt;Comparison against literal values&lt;a class="headerlink" href="#comparison-against-literal-values" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;It is very common to write DataFrame operations that compare an expression to some fixed value.
For example, filtering a DataFrame might have an operation such as &lt;code&gt;df.filter(col("size") &amp;lt; lit(16))&lt;/code&gt;.
To make these common operations more ergonomic, you can now simply use &lt;code&gt;df.filter(col("size") &amp;lt; 16)&lt;/code&gt;.&lt;/p&gt;
&lt;p&gt;For the right hand side of the comparison operator, you can now use any Python value that can be
coerced into a &lt;code&gt;Literal&lt;/code&gt;. This gives an easy to ready expression. For example, consider these few
lines from one of the
&lt;a href="https://github.com/apache/datafusion-python/tree/main/examples/tpch"&gt;TPC-H examples&lt;/a&gt; provided in
the DataFusion Python repository.&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-python"&gt;df = (
    df_lineitem.filter(col("l_shipdate") &amp;gt;= lit(date))
    .filter(col("l_discount") &amp;gt;= lit(DISCOUNT) - lit(DELTA))
    .filter(col("l_discount") &amp;lt;= lit(DISCOUNT) + lit(DELTA))
    .filter(col("l_quantity") &amp;lt; lit(QUANTITY))
)
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;The above code mirrors closely how these filters would need to be applied in rust. With this new
release, the user can simplify these lines. Also shown in the example below is that &lt;code&gt;filter()&lt;/code&gt;
now accepts a variable number of arguments and filters on all such arguments (boolean AND).&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-python"&gt;df = df_lineitem.filter(
    col("l_shipdate") &amp;gt;= date,
    col("l_discount") &amp;gt;= DISCOUNT - DELTA,
    col("l_discount") &amp;lt;= DISCOUNT + DELTA,
    col("l_quantity") &amp;lt; QUANTITY,
)
&lt;/code&gt;&lt;/pre&gt;
&lt;h3 id="select-columns-by-name"&gt;Select columns by name&lt;a class="headerlink" href="#select-columns-by-name" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;It is very common for users to perform &lt;code&gt;DataFrame&lt;/code&gt; selection where they simply want a column. For
this we have had the function &lt;code&gt;select_columns("a", "b")&lt;/code&gt; or the user could perform
&lt;code&gt;select(col("a"), col("b"))&lt;/code&gt;. In the new release, we accept either full expressions in &lt;code&gt;select()&lt;/code&gt;
or strings of the column names. You can mix these as well.&lt;/p&gt;
&lt;p&gt;Where before you may have to do an operation like&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-python"&gt;df_subset = df.select(col("a"), col("b"), f.abs(col("c")))
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;You can now simplify this to&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-python"&gt;df_subset = df.select("a", "b", f.abs(col("c")))
&lt;/code&gt;&lt;/pre&gt;
&lt;h3 id="creating-named-structs"&gt;Creating named structs&lt;a class="headerlink" href="#creating-named-structs" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;Creating a &lt;code&gt;struct&lt;/code&gt; with named fields was previously difficult to use and allowed for potential
user errors when specifying the name of each field. Now we have a cleaner interface where the
user passes a list of tuples containing the name of the field and the expression to create.&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-python"&gt;df.select(f.named_struct([
  ("a", col("a")),
  ("b", col("b"))
]))
&lt;/code&gt;&lt;/pre&gt;
&lt;h2 id="next-steps"&gt;Next Steps&lt;a class="headerlink" href="#next-steps" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;While most of the user facing classes and functions have been exposed, there are a few that require
exposure. Namely the classes in &lt;code&gt;datafusion.object_store&lt;/code&gt; and the logical plans used by
&lt;code&gt;datafusion.substrait&lt;/code&gt;. The team is working on
&lt;a href="https://github.com/apache/datafusion-python/issues/767"&gt;these issues&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;Additionally, in the next release of DataFusion there have been improvements made to the user-defined
aggregate and window functions to make them easier to use. We plan on
&lt;a href="https://github.com/apache/datafusion-python/issues/780"&gt;bringing these enhancements&lt;/a&gt; to this project.&lt;/p&gt;
&lt;h2 id="thank-you"&gt;Thank You&lt;a class="headerlink" href="#thank-you" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;We would like to thank the following members for their very helpful discussions regarding these
updates: &lt;a href="https://github.com/andygrove"&gt;@andygrove&lt;/a&gt;, &lt;a href="https://github.com/max-muoto"&gt;@max-muoto&lt;/a&gt;, &lt;a href="https://github.com/slyons"&gt;@slyons&lt;/a&gt;, &lt;a href="https://github.com/Throne3d"&gt;@Throne3d&lt;/a&gt;, &lt;a href="https://github.com/Michael-J-Ward"&gt;@Michael-J-Ward&lt;/a&gt;, &lt;a href="https://github.com/datapythonista"&gt;@datapythonista&lt;/a&gt;,
&lt;a href="https://github.com/austin362667"&gt;@austin362667&lt;/a&gt;, &lt;a href="https://github.com/kylebarron"&gt;@kylebarron&lt;/a&gt;, &lt;a href="https://github.com/simicd"&gt;@simicd&lt;/a&gt;. The &lt;a href="https://github.com/apache/datafusion-python/pull/750"&gt;primary PR (#750)&lt;/a&gt; that includes these updates
had an extensive conversation, leading to a significantly improved end product. Again, thank you
to all who provided input!&lt;/p&gt;
&lt;p&gt;We would like to give an special thank you to &lt;a href="https://github.com/3ok"&gt;@3ok&lt;/a&gt; who created the initial version of the wrapper
definitions. The work they did was time consuming and required exceptional attention to detail. It
provided enormous value to starting this project. Thank you!&lt;/p&gt;
&lt;h2 id="get-involved"&gt;Get Involved&lt;a class="headerlink" href="#get-involved" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;The DataFusion Python team is an active and engaging community and we would love
to have you join us and help the project.&lt;/p&gt;
&lt;p&gt;Here are some ways to get involved:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;
&lt;p&gt;Learn more by visiting the &lt;a href="https://datafusion.apache.org/python/index.html"&gt;DataFusion Python project&lt;/a&gt;
page.&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;
&lt;p&gt;Try out the project and provide feedback, file issues, and contribute code.&lt;/p&gt;
&lt;/li&gt;
&lt;/ul&gt;</content><category term="blog"/></entry><entry><title>Apache DataFusion 40.0.0 Released</title><link href="https://datafusion.apache.org/blog/2024/07/24/datafusion-40.0.0" rel="alternate"/><published>2024-07-24T00:00:00+00:00</published><updated>2024-07-24T00:00:00+00:00</updated><author><name>pmc</name></author><id>tag:datafusion.apache.org,2024-07-24:/blog/2024/07/24/datafusion-40.0.0</id><summary type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;!-- see https://github.com/apache/datafusion/issues/9602 for details --&gt;
&lt;h2 id="introduction"&gt;Introduction&lt;a class="headerlink" href="#introduction" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;We are proud to announce &lt;a href="https://crates.io/crates/datafusion/40.0.0"&gt;DataFusion 40.0.0&lt;/a&gt;. This blog highlights some of the
many major improvements since we released &lt;a href="https://datafusion.apache.org/blog/2024/01/19/datafusion-34.0.0/"&gt;DataFusion 34.0.0&lt;/a&gt; and a preview of
what the community is thinking about in the next 6 months. We are hoping to make
more regular blog posts …&lt;/p&gt;</summary><content type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;!-- see https://github.com/apache/datafusion/issues/9602 for details --&gt;
&lt;h2 id="introduction"&gt;Introduction&lt;a class="headerlink" href="#introduction" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;We are proud to announce &lt;a href="https://crates.io/crates/datafusion/40.0.0"&gt;DataFusion 40.0.0&lt;/a&gt;. This blog highlights some of the
many major improvements since we released &lt;a href="https://datafusion.apache.org/blog/2024/01/19/datafusion-34.0.0/"&gt;DataFusion 34.0.0&lt;/a&gt; and a preview of
what the community is thinking about in the next 6 months. We are hoping to make
more regular blog posts -- if you are interested in helping write them, please
reach out!&lt;/p&gt;
&lt;p&gt;&lt;a href="https://datafusion.apache.org/"&gt;Apache DataFusion&lt;/a&gt; is an extensible query engine, written in &lt;a href="https://www.rust-lang.org/"&gt;Rust&lt;/a&gt;, that
uses &lt;a href="https://arrow.apache.org"&gt;Apache Arrow&lt;/a&gt; as its in-memory format. DataFusion is used by developers to
create new, fast data centric systems such as databases, dataframe libraries,
machine learning and streaming applications. While &lt;a href="https://datafusion.apache.org/user-guide/introduction.html#project-goals"&gt;DataFusion’s primary design
goal&lt;/a&gt; is to accelerate the creation of other data centric systems, it has a
reasonable experience directly out of the box as a &lt;a href="https://datafusion.apache.org/python/"&gt;dataframe library&lt;/a&gt; and
&lt;a href="https://datafusion.apache.org/user-guide/cli/"&gt;command line SQL tool&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;DataFusion's core thesis is that as a community, together we can build much more
advanced technology than any of us as individuals or companies could do alone. 
Without DataFusion, highly performant vectorized query engines would remain
the domain of a few large companies and world-class research institutions. 
With DataFusion, we can all build on top of a shared foundation, and focus on
what makes our projects unique.&lt;/p&gt;
&lt;h2 id="community-growth"&gt;Community Growth  📈&lt;a class="headerlink" href="#community-growth" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;In the last 6 months, between &lt;code&gt;34.0.0&lt;/code&gt; and &lt;code&gt;40.0.0&lt;/code&gt;, our community continues to
grow in new and exciting ways.&lt;/p&gt;
&lt;ol&gt;
&lt;li&gt;DataFusion became a top level Apache Software Foundation project (read the
   &lt;a href="https://news.apache.org/foundation/entry/apache-software-foundation-announces-new-top-level-project-apache-datafusion"&gt;press release&lt;/a&gt; and &lt;a href="https://datafusion.apache.org/blog/2024/05/07/datafusion-tlp/"&gt;blog post&lt;/a&gt;).&lt;/li&gt;
&lt;li&gt;We added several PMC members and new
   committers: &lt;a href="https://github.com/comphead"&gt;@comphead&lt;/a&gt;, &lt;a href="https://github.com/mustafasrepo"&gt;@mustafasrepo&lt;/a&gt;, &lt;a href="https://github.com/ozankabak"&gt;@ozankabak&lt;/a&gt;, and &lt;a href="https://github.com/waynexia"&gt;@waynexia&lt;/a&gt; joined the PMC,
   &lt;a href="https://github.com/jonahgao"&gt;@jonahgao&lt;/a&gt; and &lt;a href="https://github.com/lewiszlw"&gt;@lewiszlw&lt;/a&gt; joined as committers. See the &lt;a href="https://lists.apache.org/list.html?dev@datafusion.apache.org"&gt;mailing list&lt;/a&gt; for
   more details.&lt;/li&gt;
&lt;li&gt;&lt;a href="https://datafusion.apache.org/comet/"&gt;DataFusion Comet&lt;/a&gt; was &lt;a href="https://arrow.apache.org/blog/2024/03/06/comet-donation/"&gt;donated&lt;/a&gt; and is nearing its first release.&lt;/li&gt;
&lt;li&gt;In the &lt;a href="https://github.com/apache/arrow-datafusion"&gt;core DataFusion repo&lt;/a&gt; alone we reviewed and accepted almost 1500 PRs from 182 different
   committers, created over 1000 issues and closed 781 of them 🚀. This is up
   almost 50% from our last post (1000 PRs from 124 committers with 650 issues
   created in our last post) 🤯. All changes are listed in the detailed
   &lt;a href="https://github.com/apache/datafusion/blob/main/datafusion/CHANGELOG.md"&gt;CHANGELOG&lt;/a&gt;.&lt;/li&gt;
&lt;li&gt;DataFusion focused meetups happened or are happening in multiple cities 
   around the world: &lt;a href="https://github.com/apache/datafusion/discussions/8522"&gt;Austin&lt;/a&gt;, &lt;a href="https://github.com/apache/datafusion/discussions/10800"&gt;San Francisco&lt;/a&gt;, &lt;a href="https://www.huodongxing.com/event/5761971909400?td=1965290734055"&gt;Hangzhou&lt;/a&gt;, &lt;a href="https://github.com/apache/datafusion/discussions/11213"&gt;New York&lt;/a&gt;, and
   &lt;a href="https://github.com/apache/datafusion/discussions/11431"&gt;Belgrade&lt;/a&gt;.&lt;/li&gt;
&lt;li&gt;Many new projects started in the &lt;a href="https://github.com/datafusion-contrib"&gt;datafusion-contrib&lt;/a&gt; organization, including
   &lt;a href="https://github.com/datafusion-contrib/datafusion-table-providers"&gt;Table Providers&lt;/a&gt;, &lt;a href="https://github.com/datafusion-contrib/datafusion-sqlancer"&gt;SQLancer&lt;/a&gt;, &lt;a href="https://github.com/datafusion-contrib/datafusion-functions-variant"&gt;Open Variant&lt;/a&gt;, &lt;a href="https://github.com/datafusion-contrib/datafusion-functions-json"&gt;JSON&lt;/a&gt;, and &lt;a href="https://github.com/datafusion-contrib/datafusion-orc"&gt;ORC&lt;/a&gt;.  &lt;/li&gt;
&lt;/ol&gt;
&lt;!--
$ git log --pretty=oneline 34.0.0..40.0.0 . | wc -l
     1453 (up from 1009)

$ git shortlog -sn 34.0.0..40.0.0 . | wc -l
      182 (up from 124)

https://crates.io/crates/datafusion/34.0.0
DataFusion 34 released Dec 17, 2023

https://crates.io/crates/datafusion/40.0.0
DataFusion 34 released July 12, 2024

Issues created in this time: 321 open, 781 closed (up from 214 open, 437 closed)
https://github.com/apache/arrow-datafusion/issues?q=is%3Aissue+created%3A2023-12-17..2024-07-12

Issues closed: 911 (up from 517)
https://github.com/apache/arrow-datafusion/issues?q=is%3Aissue+closed%3A2023-12-17..2024-07-12

PRs merged in this time 1490 (up from 908)
https://github.com/apache/arrow-datafusion/pulls?q=is%3Apr+merged%3A2023-12-17..2024-07-12

--&gt;
&lt;p&gt;In addition, DataFusion has been appearing publicly more and more, both online and offline. Here are some highlights:&lt;/p&gt;
&lt;ol&gt;
&lt;li&gt;&lt;a href="https://dl.acm.org/doi/10.1145/3626246.3653368"&gt;Apache Arrow DataFusion: A Fast, Embeddable, Modular Analytic Query Engine&lt;/a&gt;, was presented in &lt;a href="https://2024.sigmod.org/"&gt;SIGMOD '24&lt;/a&gt;, one of the major database conferences&lt;/li&gt;
&lt;li&gt;As part of the trend to define "the POSIX of databases" in &lt;a href="https://db.cs.cmu.edu/papers/2024/whatgoesaround-sigmodrec2024.pdf"&gt;"What Goes Around Comes Around... And Around..."&lt;/a&gt; from Andy Pavlo and Mike Stonebraker&lt;/li&gt;
&lt;li&gt;&lt;a href="https://www.cpard.xyz/posts/datafusion/"&gt;"Why you should keep an eye on Apache DataFusion and its community"&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;&lt;a href="https://www.tisonkun.org/2024/07/15/datafusion-meetup-san-francisco/"&gt;Apache DataFusion offline meetup in the Bay Area&lt;/a&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;h2 id="improved-performance"&gt;Improved Performance 🚀&lt;a class="headerlink" href="#improved-performance" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;Performance is a key feature of DataFusion, and the community continues to work
to keep DataFusion state of the art in this area. One major area DataFusion
improved is the time it takes to convert a SQL query into a plan that can be
executed. Planning is now almost 2x faster for TPC-DS and TPC-H queries, and
over 10x faster for some queries with many columns.&lt;/p&gt;
&lt;p&gt;Here is a chart showing the improvement due to the concerted effort of many
contributors including &lt;a href="https://github.com/jackwener"&gt;@jackwener&lt;/a&gt;, &lt;a href="https://github.com/alamb"&gt;@alamb&lt;/a&gt;, &lt;a href="https://github.com/Lordworms"&gt;@Lordworms&lt;/a&gt;, &lt;a href="https://github.com/dmitrybugakov"&gt;@dmitrybugakov&lt;/a&gt;,
&lt;a href="https://github.com/appletreeisyellow"&gt;@appletreeisyellow&lt;/a&gt;, &lt;a href="https://github.com/ClSlaid"&gt;@ClSlaid&lt;/a&gt;, &lt;a href="https://github.com/rohitrastogi"&gt;@rohitrastogi&lt;/a&gt;, &lt;a href="https://github.com/emgeee"&gt;@emgeee&lt;/a&gt;, &lt;a href="https://github.com/kevinmingtarja"&gt;@kevinmingtarja&lt;/a&gt;,
and &lt;a href="https://github.com/peter-toth"&gt;@peter-toth&lt;/a&gt; over several months (see &lt;a href="https://github.com/apache/arrow-datafusion/issues/8045"&gt;ticket&lt;/a&gt; for more details)&lt;/p&gt;
&lt;p&gt;&lt;img src="/blog/images/datafusion-40.0.0/improved-planning-time.png" width="700"/&gt;&lt;/p&gt;
&lt;p&gt;DataFusion is now up to 40% faster for queries that &lt;code&gt;GROUP BY&lt;/code&gt; a single string
or binary column due to a &lt;a href="https://github.com/apache/datafusion/pull/8827"&gt;specialization for single
Uft8/LargeUtf8/Binary/LargeBinary&lt;/a&gt;. We are working on improving performance when
there are [multiple variable length columns in the &lt;code&gt;GROUP BY&lt;/code&gt; clause].&lt;/p&gt;
&lt;p&gt;We are also in the final phases of &lt;a href="https://github.com/apache/datafusion/issues/10918"&gt;integrating&lt;/a&gt; the new &lt;a href="https://docs.rs/arrow/latest/arrow/array/struct.GenericByteViewArray.html"&gt;Arrow StringView&lt;/a&gt;
which significantly improves performance for workloads that scan, filter and
group by variable length string and binary data. We expect the improvement to be
especially pronounced for Parquet files due to &lt;a href="https://github.com/apache/arrow-rs/issues/5530"&gt;upstream work in the parquet
reader&lt;/a&gt;. Kudos to &lt;a href="https://github.com/XiangpengHong"&gt;@XiangpengHong&lt;/a&gt;, &lt;a href="https://github.com/AriesDevil"&gt;@AriesDevil&lt;/a&gt;, &lt;a href="https://github.com/PsiACE"&gt;@PsiACE&lt;/a&gt;, &lt;a href="https://github.com/Weijun-H"&gt;@Weijun-H&lt;/a&gt;,
&lt;a href="https://github.com/a10y"&gt;@a10y&lt;/a&gt;, and &lt;a href="https://github.com/RinChanNOWWW"&gt;@RinChanNOWWW&lt;/a&gt; for driving this project.&lt;/p&gt;
&lt;h2 id="improved-quality"&gt;Improved Quality 📋&lt;a class="headerlink" href="#improved-quality" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;DataFusion continues to improve overall in quality. In addition to ongoing bug
fixes, one of the most exciting improvements is the addition of a new &lt;a href="https://github.com/datafusion-contrib/datafusion-sqlancer"&gt;SQLancer&lt;/a&gt;
based &lt;a href="https://github.com/apache/datafusion/issues/11030"&gt;DataFusion Fuzzing&lt;/a&gt; suite thanks to &lt;a href="https://github.com/2010YOUY01"&gt;@2010YOUY01&lt;/a&gt; that has already found
several bugs and thanks to &lt;a href="https://github.com/jonahgao"&gt;@jonahgao&lt;/a&gt;, &lt;a href="https://github.com/tshauck"&gt;@tshauck&lt;/a&gt;, &lt;a href="https://github.com/xinlifoobar"&gt;@xinlifoobar&lt;/a&gt;,
&lt;a href="https://github.com/LorrensP-2158466"&gt;@LorrensP-2158466&lt;/a&gt; for fixing them so fast.&lt;/p&gt;
&lt;h2 id="improved-documentation"&gt;Improved Documentation 📚&lt;a class="headerlink" href="#improved-documentation" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;We continue to improve the documentation to make it easier to get started using DataFusion with
the &lt;a href="https://datafusion.apache.org/library-user-guide/index.html"&gt;Library Users Guide&lt;/a&gt;, &lt;a href="https://docs.rs/datafusion/latest/datafusion/index.html"&gt;API documentation&lt;/a&gt;, and &lt;a href="https://github.com/apache/datafusion/tree/main/datafusion-examples"&gt;Examples&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;Some notable new examples include:
* &lt;a href="https://github.com/apache/datafusion/blob/main/datafusion-examples/examples/sql_analysis.rs"&gt;sql_analysis.rs&lt;/a&gt; to analyse SQL queries with DataFusion structures (thanks &lt;a href="https://github.com/LorrensP-2158466"&gt;@LorrensP-2158466&lt;/a&gt;)
* &lt;a href="https://github.com/apache/datafusion/blob/main/datafusion-examples/examples/function_factory.rs"&gt;function_factory.rs&lt;/a&gt; to create custom functions via SQL (thanks &lt;a href="https://github.com/milenkovicm"&gt;@milenkovicm&lt;/a&gt;)
* &lt;a href="https://github.com/apache/datafusion/blob/main/datafusion-examples/examples/plan_to_sql.rs"&gt;plan_to_sql.rs&lt;/a&gt; to generate SQL from DataFusion Expr and LogicalPlan (thanks &lt;a href="https://github.com/edmondop"&gt;@edmondop&lt;/a&gt;)
* &lt;a href="https://github.com/apache/datafusion/blob/main/datafusion-examples/examples/parquet_index.rs"&gt;parquet_index.rs&lt;/a&gt; and &lt;a href="https://github.com/apache/datafusion/blob/main/datafusion-examples/examples/advanced_parquet_index.rs"&gt;advanced_parquet_index.rs&lt;/a&gt; for parquet indexing, described more below (thanks &lt;a href="https://github.com/alamb"&gt;@alamb&lt;/a&gt;)&lt;/p&gt;
&lt;h2 id="new-features"&gt;New Features ✨&lt;a class="headerlink" href="#new-features" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;There are too many new features in the last 6 months to list them all, but here
are some highlights:&lt;/p&gt;
&lt;h1 id="sql"&gt;SQL&lt;a class="headerlink" href="#sql" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h1&gt;
&lt;ul&gt;
&lt;li&gt;Support for &lt;code&gt;UNNEST&lt;/code&gt; (thanks &lt;a href="https://github.com/duongcongtoai"&gt;@duongcongtoai&lt;/a&gt;, &lt;a href="https://github.com/JasonLi-cn"&gt;@JasonLi-cn&lt;/a&gt; and &lt;a href="https://github.com/jayzhan211"&gt;@jayzhan211&lt;/a&gt;) &lt;/li&gt;
&lt;li&gt;Support for &lt;a href="https://github.com/apache/datafusion/issues/462"&gt;Recursive CTEs&lt;/a&gt; (thanks &lt;a href="https://github.com/jonahgao"&gt;@jonahgao&lt;/a&gt; and &lt;a href="https://github.com/matthewgapp"&gt;@matthewgapp&lt;/a&gt;) &lt;/li&gt;
&lt;li&gt;Support for &lt;code&gt;CREATE FUNCTION&lt;/code&gt; (see below) &lt;/li&gt;
&lt;li&gt;Many new SQL functions&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;DataFusion now has much improved support for structured types such &lt;code&gt;STRUCT&lt;/code&gt;,
&lt;code&gt;LIST&lt;/code&gt;/&lt;code&gt;ARRAY&lt;/code&gt; and &lt;code&gt;MAP&lt;/code&gt;. For example, you can now create &lt;code&gt;STRUCT&lt;/code&gt; literals 
in SQL like this:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-rust"&gt;&amp;gt; select {'foo': {'bar': 2}};
+--------------------------------------------------------------+
| named_struct(Utf8("foo"),named_struct(Utf8("bar"),Int64(2))) |
+--------------------------------------------------------------+
| {foo: {bar: 2}}                                              |
+--------------------------------------------------------------+
1 row(s) fetched.
Elapsed 0.002 seconds.
&lt;/code&gt;&lt;/pre&gt;
&lt;h1 id="sql-unparser-sql-formatter"&gt;SQL Unparser (SQL Formatter)&lt;a class="headerlink" href="#sql-unparser-sql-formatter" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h1&gt;
&lt;p&gt;DataFusion now supports converting &lt;code&gt;Expr&lt;/code&gt;s and &lt;code&gt;LogicalPlan&lt;/code&gt;s BACK to SQL text.
This can be useful in query federation to push predicates down into other
systems that only accept SQL, and for building systems that generate SQL.&lt;/p&gt;
&lt;p&gt;For example, you can now convert a logical expression back to SQL text:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-rust"&gt;// Form a logical expression that represents the SQL "a &amp;lt; 5 OR a = 8"
let expr = col("a").lt(lit(5)).or(col("a").eq(lit(8)));
// convert the expression back to SQL text
let sql = expr_to_sql(&amp;amp;expr)?.to_string();
assert_eq!(sql, "a &amp;lt; 5 OR a = 8");
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;You can also do complex things like parsing SQL, modifying the plan, and convert
it back to SQL:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-rust"&gt;let df = ctx
  // Use SQL to read some data from the parquet file
  .sql("SELECT int_col, double_col, CAST(date_string_col as VARCHAR) FROM alltypes_plain")
  .await?;
// Programmatically add new filters `id &amp;gt; 1 and tinyint_col &amp;lt; double_col`
let df = df.filter(col("id").gt(lit(1)).and(col("tinyint_col").lt(col("double_col"))))?
// Convert the new logical plan back to SQL
let sql = plan_to_sql(df.logical_plan())?.to_string();
assert_eq!(sql, 
           "SELECT alltypes_plain.int_col, alltypes_plain.double_col, CAST(alltypes_plain.date_string_col AS VARCHAR) \
           FROM alltypes_plain WHERE ((alltypes_plain.id &amp;gt; 1) AND (alltypes_plain.tinyint_col &amp;lt; alltypes_plain.double_col))")
);
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;See the &lt;a href="https://github.com/apache/datafusion/blob/main/datafusion-examples/examples/plan_to_sql.rs"&gt;Plan to SQL example&lt;/a&gt; or the APIs &lt;a href="https://docs.rs/datafusion/latest/datafusion/sql/unparser/fn.expr_to_sql.html"&gt;expr_to_sql&lt;/a&gt; and &lt;a href="https://docs.rs/datafusion/latest/datafusion/sql/unparser/fn.plan_to_sql.html"&gt;plan_to_sql&lt;/a&gt; for more details.&lt;/p&gt;
&lt;h1 id="low-level-apis-for-fast-parquet-access-indexing"&gt;Low Level APIs for Fast Parquet Access (indexing)&lt;a class="headerlink" href="#low-level-apis-for-fast-parquet-access-indexing" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h1&gt;
&lt;p&gt;With their rising prevalence, supporting efficient access to Parquet files
stored remotely on object storage is important. Part of doing this efficiently
is minimizing the number of object store requests made by caching metadata and
skipping over parts of the file that are not needed (e.g. via an index).&lt;/p&gt;
&lt;p&gt;DataFusion's Parquet reader has long internally supported &lt;a href="https://arrow.apache.org/blog/2022/12/26/querying-parquet-with-millisecond-latency/"&gt;advanced predicate
pushdown&lt;/a&gt; by reading the parquet metadata from the file footer and pruning based
on row group and data page statistics. DataFusion now also supports users
supplying their own low level pruning information via the [&lt;code&gt;ParquetAccessPlan&lt;/code&gt;]
API.&lt;/p&gt;
&lt;p&gt;This API can be used along with index information to selectively skip decoding
parts of the file. For example, Spice AI used this feature to add &lt;a href="https://github.com/spiceai/spiceai/pull/1891"&gt;efficient
support&lt;/a&gt; for reading from DeltaLake tables and handling &lt;a href="https://docs.delta.io/latest/delta-deletion-vectors.html"&gt;deletion vectors&lt;/a&gt;.&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-text"&gt;        ┌───────────────────────┐   If the RowSelection does not include any
        │          ...          │   rows from a particular Data Page, that
        │                       │   Data Page is not fetched or decoded.
        │ ┌───────────────────┐ │   Note this requires a PageIndex
        │ │     ┌──────────┐  │ │
Row     │ │     │DataPage 0│  │ │                 ┌────────────────────┐
Groups  │ │     └──────────┘  │ │                 │                    │
        │ │     ┌──────────┐  │ │                 │    ParquetExec     │
        │ │ ... │DataPage 1│ ◀┼ ┼ ─ ─ ─           │  (Parquet Reader)  │
        │ │     └──────────┘  │ │      └ ─ ─ ─ ─ ─│                    │
        │ │     ┌──────────┐  │ │                 │ ╔═══════════════╗  │
        │ │     │DataPage 2│  │ │ If only rows    │ ║ParquetMetadata║  │
        │ │     └──────────┘  │ │ from DataPage 1 │ ╚═══════════════╝  │
        │ └───────────────────┘ │ are selected,   └────────────────────┘
        │                       │ only DataPage 1
        │          ...          │ is fetched and
        │                       │ decoded
        │ ╔═══════════════════╗ │
        │ ║  Thrift metadata  ║ │
        │ ╚═══════════════════╝ │
        └───────────────────────┘
         Parquet File
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;See the &lt;a href="https://github.com/apache/datafusion/blob/main/datafusion-examples/examples/parquet_index.rs"&gt;parquet_index.rs&lt;/a&gt; and &lt;a href="https://github.com/apache/datafusion/blob/main/datafusion-examples/examples/advanced_parquet_index.rs"&gt;advanced_parquet_index.rs&lt;/a&gt; examples for more details. &lt;/p&gt;
&lt;p&gt;Thanks to &lt;a href="https://github.com/alamb"&gt;@alamb&lt;/a&gt; and &lt;a href="https://github.com/Ted-Jiang"&gt;@Ted-Jiang&lt;/a&gt; for this feature.  &lt;/p&gt;
&lt;h2 id="building-systems-is-easier-with-datafusion"&gt;Building Systems is Easier with DataFusion 🛠️&lt;a class="headerlink" href="#building-systems-is-easier-with-datafusion" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;In addition to many incremental API improvements, there are several new APIs that make
it easier to build systems on top of DataFusion:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;Faster and easier to use &lt;a href="https://docs.rs/datafusion/latest/datafusion/common/tree_node/trait.TreeNode.html#overview"&gt;TreeNode API&lt;/a&gt; for traversing and manipulating plans and expressions.&lt;/li&gt;
&lt;li&gt;All functions now use the same &lt;a href="https://docs.rs/datafusion/latest/datafusion/logical_expr/trait.ScalarUDFImpl.html"&gt;Scalar User Defined Function API&lt;/a&gt;, making it easier to customize
  DataFusion's behavior without sacrificing performance. See &lt;a href="https://github.com/apache/arrow-datafusion/issues/8045"&gt;ticket&lt;/a&gt; for more details.&lt;/li&gt;
&lt;li&gt;DataFusion can now be compiled to &lt;a href="https://github.com/apache/datafusion/discussions/9834"&gt;WASM&lt;/a&gt;. &lt;/li&gt;
&lt;/ul&gt;
&lt;h1 id="user-defined-sql-parsing-extensions"&gt;User Defined SQL Parsing Extensions&lt;a class="headerlink" href="#user-defined-sql-parsing-extensions" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h1&gt;
&lt;p&gt;As of DataFusion 40.0.0, you can use the [&lt;code&gt;ExprPlanner&lt;/code&gt;] trait to extend
DataFusion's SQL planner to support custom operators or syntax.&lt;/p&gt;
&lt;p&gt;For example the &lt;a href="https://github.com/datafusion-contrib/datafusion-functions-json"&gt;datafusion-functions-json&lt;/a&gt; project uses this API to support
JSON operators in SQL queries. It provides a custom implementation for
planning JSON operators such as &lt;code&gt;-&amp;gt;&lt;/code&gt; and &lt;code&gt;-&amp;gt;&amp;gt;&lt;/code&gt; with code like:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-rust"&gt;struct MyCustomPlanner;

impl ExprPlanner for MyCustomPlanner {
    // Provide custom implementation for planning a binary operators
    // such as `-&amp;gt;` and `-&amp;gt;&amp;gt;`
    fn plan_binary_op(
        &amp;amp;self,
        expr: RawBinaryExpr,
        _schema: &amp;amp;DFSchema,
    ) -&amp;gt; Result&amp;lt;PlannerResult&amp;lt;RawBinaryExpr&amp;gt;&amp;gt; {
        match &amp;amp;expr.op {
           BinaryOperator::Arrow =&amp;gt; { /* plan -&amp;gt; operator */ }
           BinaryOperator::LongArrow =&amp;gt; { /* plan -&amp;gt;&amp;gt; operator */ }
           ...
        }
    }
}
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;Thanks to &lt;a href="https://github.com/samuelcolvin"&gt;@samuelcolvin&lt;/a&gt;, &lt;a href="https://github.com/jayzhan211"&gt;@jayzhan211&lt;/a&gt; and &lt;a href="https://github.com/dharanad"&gt;@dharanad&lt;/a&gt; for helping make this
feature happen.&lt;/p&gt;
&lt;h1 id="pluggable-support-for-create-function"&gt;Pluggable Support for &lt;code&gt;CREATE FUNCTION&lt;/code&gt;&lt;a class="headerlink" href="#pluggable-support-for-create-function" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h1&gt;
&lt;p&gt;DataFusion's new [&lt;code&gt;FunctionFactory&lt;/code&gt;] API let's users provide a handler for
&lt;code&gt;CREATE FUNCTION&lt;/code&gt; SQL statements. This feature lets you build systems that
support defining functions in SQL such as&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-sql"&gt;-- SQL based functions
CREATE FUNCTION my_func(DOUBLE, DOUBLE) RETURNS DOUBLE
    RETURN $1 + $3
;

-- ML Models
CREATE FUNCTION iris(FLOAT[]) RETURNS FLOAT[] 
LANGUAGE TORCH AS 'models:/iris@champion';

-- WebAssembly
CREATE FUNCTION func(FLOAT[]) RETURNS FLOAT[] 
LANGUAGE WASM AS 'func.wasm'
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;Huge thanks to &lt;a href="https://github.com/milenkovicm"&gt;@milenkovicm&lt;/a&gt; for this feature. There is an example of how to
make macro like functions in &lt;a href="https://github.com/apache/datafusion/blob/main/datafusion-examples/examples/function_factory.rs"&gt;function_factory.rs&lt;/a&gt;. It would be
great if &lt;a href="https://github.com/apache/datafusion/issues/9326"&gt;someone made a demo&lt;/a&gt; showing how to create WASMs 🎣.&lt;/p&gt;
&lt;h2 id="looking-ahead-the-next-six-months"&gt;Looking Ahead: The Next Six Months 🔭&lt;a class="headerlink" href="#looking-ahead-the-next-six-months" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;The community has been &lt;a href="https://github.com/apache/datafusion/issues/11442"&gt;discussing what we will work on in the next six months&lt;/a&gt;.
Some major initiatives from that discussion are:&lt;/p&gt;
&lt;ol&gt;
&lt;li&gt;
&lt;p&gt;&lt;em&gt;Performance&lt;/em&gt;: Improve the speed of &lt;a href="https://github.com/apache/arrow-datafusion/issues/7000"&gt;aggregating "high cardinality"&lt;/a&gt;
  data when there are many (e.g. millions) of distinct groups as well as additional
  ideas to improve parquet performance. &lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;
&lt;p&gt;&lt;em&gt;Modularity&lt;/em&gt;: Make DataFusion even more modular, by completely unifying
   built in and user &lt;a href="https://github.com/apache/datafusion/issues/8708"&gt;aggregate functions&lt;/a&gt; and &lt;a href="https://github.com/apache/datafusion/issues/8709"&gt;window functions&lt;/a&gt;.&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;
&lt;p&gt;&lt;em&gt;LogicalTypes&lt;/em&gt;: &lt;a href="https://github.com/apache/datafusion/issues/11513"&gt;Introduce Logical Types&lt;/a&gt; to make it easier to use
   different encodings like &lt;code&gt;StringView&lt;/code&gt;, &lt;code&gt;RunEnd&lt;/code&gt; and &lt;code&gt;Dictionary&lt;/code&gt; arrays as well
   as user defined types. Thanks &lt;a href="https://github.com/notfilippo"&gt;@notfilippo&lt;/a&gt; for driving this. &lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;
&lt;p&gt;&lt;em&gt;Improved Documentation&lt;/em&gt;: Write blog posts and videos explaining
   how to use DataFusion for real-world use cases.&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;
&lt;p&gt;&lt;em&gt;Testing&lt;/em&gt;: Improve CI infrastructure and test coverage, more fuzz
   testing, and better functional and performance regression testing.&lt;/p&gt;
&lt;/li&gt;
&lt;/ol&gt;
&lt;h1 id="how-to-get-involved"&gt;How to Get Involved&lt;a class="headerlink" href="#how-to-get-involved" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h1&gt;
&lt;p&gt;DataFusion is not a project built or driven by a single person, company, or
foundation. Rather, our community of users and contributors work together to
build a shared technology that none of us could have built alone.&lt;/p&gt;
&lt;p&gt;If you are interested in joining us we would love to have you. You can try out
DataFusion on some of your own data and projects and let us know how it goes,
contribute suggestions, documentation, bug reports, or a PR with documentation,
tests or code. A list of open issues suitable for beginners is &lt;a href="https://github.com/apache/arrow-datafusion/issues?q=is%3Aissue+is%3Aopen+label%3A%22good+first+issue%22"&gt;here&lt;/a&gt; and you
can find how to reach us on the &lt;a href="https://datafusion.apache.org/contributor-guide/communication.html"&gt;communication doc&lt;/a&gt;.&lt;/p&gt;</content><category term="blog"/></entry><entry><title>Apache DataFusion Comet 0.1.0 Release</title><link href="https://datafusion.apache.org/blog/2024/07/20/datafusion-comet-0.1.0" rel="alternate"/><published>2024-07-20T00:00:00+00:00</published><updated>2024-07-20T00:00:00+00:00</updated><author><name>pmc</name></author><id>tag:datafusion.apache.org,2024-07-20:/blog/2024/07/20/datafusion-comet-0.1.0</id><summary type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;p&gt;The Apache DataFusion PMC is pleased to announce the first official source release of the &lt;a href="https://datafusion.apache.org/comet/"&gt;Comet&lt;/a&gt; subproject.&lt;/p&gt;
&lt;p&gt;Comet is an accelerator for Apache Spark that translates Spark physical plans to DataFusion physical plans for
improved performance and efficiency without requiring any code changes.&lt;/p&gt;
&lt;p&gt;Comet runs on commodity hardware and aims …&lt;/p&gt;</summary><content type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;p&gt;The Apache DataFusion PMC is pleased to announce the first official source release of the &lt;a href="https://datafusion.apache.org/comet/"&gt;Comet&lt;/a&gt; subproject.&lt;/p&gt;
&lt;p&gt;Comet is an accelerator for Apache Spark that translates Spark physical plans to DataFusion physical plans for
improved performance and efficiency without requiring any code changes.&lt;/p&gt;
&lt;p&gt;Comet runs on commodity hardware and aims to provide 100% compatibility with Apache Spark. Any operators or
expressions that are not fully compatible will fall back to Spark unless explicitly enabled by the user. Refer
to the &lt;a href="https://datafusion.apache.org/comet/user-guide/compatibility.html"&gt;compatibility guide&lt;/a&gt; for more information.&lt;/p&gt;
&lt;p&gt;This release covers five months of development work since the project was &lt;a href="https://datafusion.apache.org/blog/2024/03/06/comet-donation/"&gt;donated&lt;/a&gt; to the Apache DataFusion
project and is the result of merging 343 PRs from 41 contributors. See the &lt;a href="https://github.com/apache/datafusion-comet/blob/main/dev/changelog/0.1.0.md"&gt;change log&lt;/a&gt; for more information.&lt;/p&gt;
&lt;p&gt;This first release supports 15 &lt;a href="https://datafusion.apache.org/comet/user-guide/datatypes.html#"&gt;data types&lt;/a&gt;, 13 &lt;a href="https://datafusion.apache.org/comet/user-guide/operators.html#"&gt;operators&lt;/a&gt;, and 106 &lt;a href="https://datafusion.apache.org/comet/user-guide/expressions.html#"&gt;expressions&lt;/a&gt;. Comet is compatible with Apache
Spark versions 3.3, 3.4, and 3.5. There is also experimental support for preview versions of Spark 4.0.&lt;/p&gt;
&lt;h2 id="project-status"&gt;Project Status&lt;a class="headerlink" href="#project-status" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;The project's recent focus has been on fixing correctness and stability issues and implementing additional
native operators and expressions so that a broader range of queries can be executed natively.&lt;/p&gt;
&lt;p&gt;Here are some of the highlights since the project was donated:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;Implemented native support for:&lt;/li&gt;
&lt;li&gt;SortMergeJoin&lt;/li&gt;
&lt;li&gt;HashJoin&lt;/li&gt;
&lt;li&gt;BroadcastHashJoin&lt;/li&gt;
&lt;li&gt;Columnar Shuffle&lt;/li&gt;
&lt;li&gt;More aggregate expressions&lt;/li&gt;
&lt;li&gt;Window aggregates&lt;/li&gt;
&lt;li&gt;Many Spark-compatible CAST expressions&lt;/li&gt;
&lt;li&gt;Implemented a simple Spark Fuzz Testing utility to find correctness issues&lt;/li&gt;
&lt;li&gt;Published a &lt;a href="https://datafusion.apache.org/comet/user-guide/overview.html"&gt;User Guide&lt;/a&gt; and &lt;a href="https://datafusion.apache.org/comet/contributor-guide/contributing.html"&gt;Contributors Guide&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;Created a &lt;a href="https://github.com/apache/datafusion-benchmarks"&gt;DataFusion Benchmarks&lt;/a&gt; repository with scripts and documentation for running benchmarks derived&lt;br/&gt;
  from TPC-H and TPC-DS with DataFusion and Comet&lt;/li&gt;
&lt;/ul&gt;
&lt;h2 id="current-performance"&gt;Current Performance&lt;a class="headerlink" href="#current-performance" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;Comet already delivers a modest performance speedup for many queries, enabling faster data processing and
shorter time-to-insights.&lt;/p&gt;
&lt;p&gt;We use benchmarks derived from the industry standard TPC-H and TPC-DS benchmarks for tracking progress with
performance. The following chart shows the time it takes to run the 22 TPC-H queries against 100 GB of data in
Parquet format using a single executor with eight cores. See the &lt;a href="https://datafusion.apache.org/comet/contributor-guide/benchmarking.html"&gt;Comet Benchmarking Guide&lt;/a&gt;
for details of the environment used for these benchmarks.&lt;/p&gt;
&lt;p&gt;&lt;img alt="Chart showing TPC-H benchmark results for Comet 0.1.0" class="img-fluid" src="/blog/images/comet-0.1.0/tpch_allqueries.png" width="100%"/&gt;&lt;/p&gt;
&lt;p&gt;Comet reduces the overall execution time from 626 seconds to 407 seconds, a 54% speedup (1.54x faster).&lt;/p&gt;
&lt;p&gt;Running the same queries with DataFusion standalone using the same number of cores results in a 3.9x speedup
compared to Spark. Although this isn’t a fair comparison (DataFusion does not have shuffle or match Spark
semantics in some cases, for example), it does give some idea about the potential future performance of
Comet. Comet aims to provide a 2x-4x speedup for a wide range of queries once more operators and expressions
can run natively.&lt;/p&gt;
&lt;p&gt;The following chart shows how much Comet currently accelerates each query from the benchmark.&lt;/p&gt;
&lt;p&gt;&lt;img alt="Chart showing TPC-H benchmark results for Comet 0.1.0" class="img-fluid" src="/blog/images/comet-0.1.0/tpch_queries_speedup.png" width="100%"/&gt;&lt;/p&gt;
&lt;p&gt;These benchmarks can be reproduced in any environment using the documentation in the &lt;a href="https://datafusion.apache.org/comet/contributor-guide/benchmarking.html"&gt;Comet Benchmarking Guide&lt;/a&gt;. We
encourage you to run these benchmarks in your environment or, even better, try Comet out with your existing Spark jobs.&lt;/p&gt;
&lt;h2 id="roadmap"&gt;Roadmap&lt;a class="headerlink" href="#roadmap" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;Comet is an open-source project, and contributors are welcome to work on any features they are interested in, but
here are some current focus areas.&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;Improve Performance &amp;amp; Reliability:&lt;/li&gt;
&lt;li&gt;Implement the remaining features needed so that all TPC-H queries can run entirely natively&lt;/li&gt;
&lt;li&gt;Implement spill support in SortMergeJoin&lt;/li&gt;
&lt;li&gt;Enable columnar shuffle by default&lt;/li&gt;
&lt;li&gt;Fully support Spark version 4.0.0&lt;/li&gt;
&lt;li&gt;Support more Spark operators and expressions&lt;/li&gt;
&lt;li&gt;We would like to support many more expressions natively in Comet, and this is a great place to start
    contributing. The contributors' guide has a section covering &lt;a href="https://datafusion.apache.org/comet/contributor-guide/adding_a_new_expression.html"&gt;adding support for new expressions&lt;/a&gt;.&lt;/li&gt;
&lt;li&gt;Move more Spark expressions into the &lt;a href="https://crates.io/crates/datafusion-comet-spark-expr"&gt;datafusion-comet-spark-expr&lt;/a&gt; crate. Although the main focus of the Comet
  project is to provide an accelerator for Apache Spark, we also publish a standalone crate containing
  Spark-compatible expressions that can be used by any project using DataFusion, without adding any dependencies
  on JVM or Apache Spark.&lt;/li&gt;
&lt;li&gt;Release Process &amp;amp; Documentation&lt;/li&gt;
&lt;li&gt;Implement a binary release process so that we can publish JAR files to Maven for all supported platforms&lt;/li&gt;
&lt;li&gt;Add documentation for running Spark and Comet in Kubernetes, and add example Dockerfiles.&lt;/li&gt;
&lt;/ul&gt;
&lt;h2 id="getting-involved"&gt;Getting Involved&lt;a class="headerlink" href="#getting-involved" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;The Comet project welcomes new contributors. We use the same &lt;a href="https://datafusion.apache.org/contributor-guide/communication.html#slack-and-discord"&gt;Slack and Discord&lt;/a&gt; channels as the main DataFusion
project, and there is a Comet community video call held every four weeks on Wednesdays at 11:30 a.m. Eastern Time,
which is 16:30 UTC during Eastern Standard Time and 15:30 UTC during Eastern Daylight Time. See the
&lt;a href="https://docs.google.com/document/d/1NBpkIAuU7O9h8Br5CbFksDhX-L9TyO9wmGLPMe0Plc8/edit?usp=sharing"&gt;Comet Community Meeting&lt;/a&gt; Google Document for the next scheduled meeting date, the video call link, and
recordings of previous calls.&lt;/p&gt;
&lt;p&gt;The easiest way to get involved is to test Comet with your current Spark jobs and file issues for any bugs or
performance regressions that you find. See the &lt;a href="https://datafusion.apache.org/comet/user-guide/installation.html"&gt;Getting Started&lt;/a&gt; guide for instructions on downloading and installing
Comet.&lt;/p&gt;
&lt;p&gt;There are also many &lt;a href="https://github.com/apache/datafusion-comet/contribute"&gt;good first issues&lt;/a&gt; waiting for contributions.&lt;/p&gt;</content><category term="blog"/></entry><entry><title>Announcing Apache Arrow DataFusion is now Apache DataFusion</title><link href="https://datafusion.apache.org/blog/2024/05/07/datafusion-tlp" rel="alternate"/><published>2024-05-07T00:00:00+00:00</published><updated>2024-05-07T00:00:00+00:00</updated><author><name>pmc</name></author><id>tag:datafusion.apache.org,2024-05-07:/blog/2024/05/07/datafusion-tlp</id><summary type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;h2 id="introduction"&gt;Introduction&lt;a class="headerlink" href="#introduction" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;TLDR; &lt;a href="https://arrow.apache.org/"&gt;Apache Arrow&lt;/a&gt; DataFusion --&amp;gt; &lt;a href="https://datafusion.apache.org/"&gt;Apache DataFusion&lt;/a&gt;&lt;/p&gt;
&lt;p&gt;The Arrow PMC and newly created DataFusion PMC are happy to announce that as of
April 16, 2024 the Apache Arrow DataFusion subproject is now a top level
&lt;a href="https://www.apache.org/"&gt;Apache Software Foundation&lt;/a&gt; project.&lt;/p&gt;
&lt;h2 id="background"&gt;Background&lt;a class="headerlink" href="#background" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;Apache DataFusion is a fast, extensible query engine for building …&lt;/p&gt;</summary><content type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;h2 id="introduction"&gt;Introduction&lt;a class="headerlink" href="#introduction" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;TLDR; &lt;a href="https://arrow.apache.org/"&gt;Apache Arrow&lt;/a&gt; DataFusion --&amp;gt; &lt;a href="https://datafusion.apache.org/"&gt;Apache DataFusion&lt;/a&gt;&lt;/p&gt;
&lt;p&gt;The Arrow PMC and newly created DataFusion PMC are happy to announce that as of
April 16, 2024 the Apache Arrow DataFusion subproject is now a top level
&lt;a href="https://www.apache.org/"&gt;Apache Software Foundation&lt;/a&gt; project.&lt;/p&gt;
&lt;h2 id="background"&gt;Background&lt;a class="headerlink" href="#background" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;Apache DataFusion is a fast, extensible query engine for building high-quality
data-centric systems in Rust, using the Apache Arrow in-memory format.&lt;/p&gt;
&lt;p&gt;When DataFusion was &lt;a href="https://arrow.apache.org/blog/2019/02/04/datafusion-donation/"&gt;donated to the Apache Software Foundation&lt;/a&gt; in 2019, the
DataFusion community was not large enough to stand on its own and the Arrow
project agreed to help support it. The community has grown significantly since
2019, benefiting immensely from being part of Arrow and following &lt;a href="https://www.apache.org/theapacheway/"&gt;The Apache
Way&lt;/a&gt;.&lt;/p&gt;
&lt;h2 id="why-now"&gt;Why now?&lt;a class="headerlink" href="#why-now" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;The community &lt;a href="https://github.com/apache/datafusion/discussions/6475"&gt;discussed graduating to a top level project publicly&lt;/a&gt; for almost
a year, as the project seemed ready to stand on its own and would benefit from
more focused governance. For example, earlier in DataFusion's life many
contributed to both &lt;a href="https://github.com/apache/arrow-rs"&gt;arrow-rs&lt;/a&gt; and DataFusion, but as DataFusion has matured many
contributors, committers and PMC members focused more and more exclusively on
DataFusion.&lt;/p&gt;
&lt;h2 id="looking-forward"&gt;Looking forward&lt;a class="headerlink" href="#looking-forward" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;The future looks bright. There are now &lt;a href="https://datafusion.apache.org/user-guide/introduction.html#known-users"&gt;10s of known projects built with
DataFusion&lt;/a&gt;, and that number continues to grow. We recently held our &lt;a href="https://github.com/apache/datafusion/discussions/8522"&gt;first in
person meetup&lt;/a&gt; passed &lt;a href="https://github.com/apache/datafusion/stargazers"&gt;5000 stars&lt;/a&gt; on GitHub, &lt;a href="https://github.com/apache/datafusion/issues/8373#issuecomment-2025133714"&gt;wrote a paper that was accepted
at SIGMOD 2024&lt;/a&gt;, and began work on &lt;a href="https://github.com/apache/datafusion-comet"&gt;Comet&lt;/a&gt;, an &lt;a href="https://spark.apache.org/"&gt;Apache Spark&lt;/a&gt; accelerator
&lt;a href="https://arrow.apache.org/blog/2024/03/06/comet-donation/"&gt;initially donated by Apple&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;Thank you to everyone in the Arrow community who helped DataFusion grow and
mature over the years, and we look forward to continuing our collaboration as
projects. All future blogs and announcements will be posted on the &lt;a href="https://datafusion.apache.org/"&gt;Apache
DataFusion&lt;/a&gt; website.&lt;/p&gt;
&lt;h2 id="get-involved"&gt;Get Involved&lt;a class="headerlink" href="#get-involved" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;If you are interested in joining the community, we would love to have you join
us. Get in touch using &lt;a href="https://datafusion.apache.org/contributor-guide/communication.html"&gt;Communication Doc&lt;/a&gt; and learn how to get involved in the
&lt;a href="https://datafusion.apache.org/contributor-guide/index.html"&gt;Contributor Guide&lt;/a&gt;. We welcome everyone to try DataFusion on their
own data and projects and let us know how it goes, contribute suggestions,
documentation, bug reports, or a PR with documentation, tests or code.&lt;/p&gt;</content><category term="blog"/></entry><entry><title>Announcing Apache Arrow DataFusion Comet</title><link href="https://datafusion.apache.org/blog/2024/03/06/comet-donation" rel="alternate"/><published>2024-03-06T00:00:00+00:00</published><updated>2024-03-06T00:00:00+00:00</updated><author><name>pmc</name></author><id>tag:datafusion.apache.org,2024-03-06:/blog/2024/03/06/comet-donation</id><summary type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;h1 id="introduction"&gt;Introduction&lt;a class="headerlink" href="#introduction" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h1&gt;
&lt;p&gt;The Apache Arrow PMC is pleased to announce the donation of the &lt;a href="https://github.com/apache/arrow-datafusion-comet"&gt;Comet project&lt;/a&gt;,
a native Spark SQL Accelerator built on &lt;a href="https://arrow.apache.org/datafusion"&gt;Apache Arrow DataFusion&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;Comet is an Apache Spark plugin that uses Apache Arrow DataFusion to
accelerate Spark workloads. It is designed as a drop-in
replacement for Spark's JVM …&lt;/p&gt;</summary><content type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;h1 id="introduction"&gt;Introduction&lt;a class="headerlink" href="#introduction" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h1&gt;
&lt;p&gt;The Apache Arrow PMC is pleased to announce the donation of the &lt;a href="https://github.com/apache/arrow-datafusion-comet"&gt;Comet project&lt;/a&gt;,
a native Spark SQL Accelerator built on &lt;a href="https://arrow.apache.org/datafusion"&gt;Apache Arrow DataFusion&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;Comet is an Apache Spark plugin that uses Apache Arrow DataFusion to
accelerate Spark workloads. It is designed as a drop-in
replacement for Spark's JVM based SQL execution engine and offers significant
performance improvements for some workloads as shown below.&lt;/p&gt;
&lt;figure class="text-center"&gt;
&lt;img alt="Fig 1: Adaptive Arrow schema architecture overview." class="img-fluid" src="/blog/images/datafusion-comet/comet-architecture.png"/&gt;
&lt;figcaption&gt;
&lt;b&gt;Figure 1&lt;/b&gt;: With Comet, users interact with the same Spark ecosystem, tools
    and APIs such as Spark SQL. Queries still run through Spark's query optimizer and planner. 
    However, the execution is delegated to Comet,
    which is significantly faster and more resource efficient than a JVM based
    implementation.
&lt;/figcaption&gt;
&lt;/figure&gt;
&lt;p&gt;Comet is one of a growing class of projects that aim to accelerate Spark using
native columnar engines such as the proprietary &lt;a href="https://www.databricks.com/product/photon"&gt;Databricks Photon Engine&lt;/a&gt; and
open source projects &lt;a href="https://incubator.apache.org/projects/gluten.html"&gt;Gluten&lt;/a&gt;, &lt;a href="https://github.com/NVIDIA/spark-rapids"&gt;Spark RAPIDS&lt;/a&gt;, and &lt;a href="https://github.com/kwai/blaze"&gt;Blaze&lt;/a&gt; (also built using
DataFusion).&lt;/p&gt;
&lt;p&gt;Comet was originally implemented at Apple and the engineers who worked on the
project are also significant contributors to Arrow and DataFusion. Bringing 
Comet into the Apache Software Foundation will accelerate its development and 
grow its community of contributors and users.&lt;/p&gt;
&lt;h1 id="get-involved"&gt;Get Involved&lt;a class="headerlink" href="#get-involved" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h1&gt;
&lt;p&gt;Comet is still in the early stages of development and we would love to have you
join us and help shape the project. We are working on an initial release, and 
expect to post another update with more details at that time.&lt;/p&gt;
&lt;p&gt;Before then, here are some ways to get involved:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;
&lt;p&gt;Learn more by visiting the &lt;a href="https://github.com/apache/arrow-datafusion-comet"&gt;Comet project&lt;/a&gt; page and reading the &lt;a href="https://lists.apache.org/thread/0q1rb11jtpopc7vt1ffdzro0omblsh0s"&gt;mailing list
  discussion&lt;/a&gt; about the initial donation.&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;
&lt;p&gt;Help us plan out the &lt;a href="https://github.com/apache/arrow-datafusion-comet/issues/19"&gt;roadmap&lt;/a&gt;&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;
&lt;p&gt;Try out the project and provide feedback, file issues, and contribute code.&lt;/p&gt;
&lt;/li&gt;
&lt;/ul&gt;</content><category term="blog"/></entry><entry><title>Apache Arrow DataFusion 34.0.0 Released, Looking Forward to 2024</title><link href="https://datafusion.apache.org/blog/2024/01/19/datafusion-34.0.0" rel="alternate"/><published>2024-01-19T00:00:00+00:00</published><updated>2024-01-19T00:00:00+00:00</updated><author><name>pmc</name></author><id>tag:datafusion.apache.org,2024-01-19:/blog/2024/01/19/datafusion-34.0.0</id><summary type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;h2 id="introduction"&gt;Introduction&lt;a class="headerlink" href="#introduction" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;We recently &lt;a href="https://crates.io/crates/datafusion/34.0.0"&gt;released DataFusion 34.0.0&lt;/a&gt;. This blog highlights some of the major
improvements since we &lt;a href="https://arrow.apache.org/blog/2023/06/24/datafusion-25.0.0/."&gt;released DataFusion 26.0.0&lt;/a&gt; (spoiler alert there are many)
and a preview of where the community plans to focus in the next 6 months.&lt;/p&gt;
&lt;p&gt;&lt;a href="https://arrow.apache.org/datafusion/"&gt;Apache Arrow DataFusion&lt;/a&gt; is an extensible query …&lt;/p&gt;</summary><content type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;h2 id="introduction"&gt;Introduction&lt;a class="headerlink" href="#introduction" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;We recently &lt;a href="https://crates.io/crates/datafusion/34.0.0"&gt;released DataFusion 34.0.0&lt;/a&gt;. This blog highlights some of the major
improvements since we &lt;a href="https://arrow.apache.org/blog/2023/06/24/datafusion-25.0.0/."&gt;released DataFusion 26.0.0&lt;/a&gt; (spoiler alert there are many)
and a preview of where the community plans to focus in the next 6 months.&lt;/p&gt;
&lt;p&gt;&lt;a href="https://arrow.apache.org/datafusion/"&gt;Apache Arrow DataFusion&lt;/a&gt; is an extensible query engine, written in &lt;a href="https://www.rust-lang.org/"&gt;Rust&lt;/a&gt;, that
uses &lt;a href="https://arrow.apache.org"&gt;Apache Arrow&lt;/a&gt; as its in-memory format. DataFusion is used by developers to
create new, fast data centric systems such as databases, dataframe libraries,
machine learning and streaming applications. While &lt;a href="https://arrow.apache.org/datafusion/user-guide/introduction.html#project-goals"&gt;DataFusion’s primary design
goal&lt;/a&gt; is to accelerate creating other data centric systems, it has a
reasonable experience directly out of the box as a &lt;a href="https://arrow.apache.org/datafusion-python/"&gt;dataframe library&lt;/a&gt; and
&lt;a href="https://arrow.apache.org/datafusion/user-guide/cli.html"&gt;command line SQL tool&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;This may also be our last update on the Apache Arrow Site. Future
updates will likely be on the DataFusion website as we are working to &lt;a href="https://github.com/apache/arrow-datafusion/discussions/6475"&gt;graduate
to a top level project&lt;/a&gt; (Apache Arrow DataFusion → Apache DataFusion!) which
will help focus governance and project growth. Also exciting, our &lt;a href="https://github.com/apache/arrow-datafusion/discussions/8522"&gt;first
DataFusion in person meetup&lt;/a&gt; is planned for March 2024.&lt;/p&gt;
&lt;p&gt;DataFusion is very much a community endeavor. Our core thesis is that as a
community we can build much more advanced technology than any of us as
individuals or companies could alone. In the last 6 months between &lt;code&gt;26.0.0&lt;/code&gt; and
&lt;code&gt;34.0.0&lt;/code&gt;, community growth has been strong. We accepted and reviewed over a
thousand PRs from 124 different committers, created over 650 issues and closed 517
of them.
You can find a list of all changes in the detailed &lt;a href="https://github.com/apache/arrow-datafusion/blob/main/datafusion/CHANGELOG.md"&gt;CHANGELOG&lt;/a&gt;.&lt;/p&gt;
&lt;!--
$ git log --pretty=oneline 26.0.0..34.0.0 . | wc -l
     1009

$ git shortlog -sn 26.0.0..34.0.0 . | wc -l
      124

https://crates.io/crates/datafusion/26.0.0
DataFusion 26 released June 7, 2023

https://crates.io/crates/datafusion/34.0.0
DataFusion 34 released Dec 17, 2023

Issues created in this time: 214 open, 437 closed
https://github.com/apache/arrow-datafusion/issues?q=is%3Aissue+created%3A2023-06-23..2023-12-17

Issues closes: 517
https://github.com/apache/arrow-datafusion/issues?q=is%3Aissue+closed%3A2023-06-23..2023-12-17+

PRs merged in this time 908
https://github.com/apache/arrow-datafusion/pulls?q=is%3Apr+merged%3A2023-06-23..2023-12-17
--&gt;
&lt;h1 id="improved-performance"&gt;Improved Performance 🚀&lt;a class="headerlink" href="#improved-performance" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h1&gt;
&lt;p&gt;Performance is a key feature of DataFusion, DataFusion is 
more than 2x faster on &lt;a href="https://benchmark.clickhouse.com/"&gt;ClickBench&lt;/a&gt; compared to version &lt;code&gt;25.0.0&lt;/code&gt;, as shown below:&lt;/p&gt;
&lt;!--
  Scripts: https://github.com/alamb/datafusion-duckdb-benchmark/tree/datafusion-25-34
  Spreadsheet: https://docs.google.com/spreadsheets/d/1FtI3652WIJMC5LmJbLfT3G06w0JQIxEPG4yfMafexh8/edit#gid=1879366976
  Average runtime on 25.0.0: 7.2s (for the queries that actually ran)
  Average runtime on 34.0.0: 3.6s (for the same queries that ran in 25.0.0)
--&gt;
&lt;figure class="text-center"&gt;
&lt;img alt="Fig 1: Adaptive Arrow schema architecture overview." class="img-fluid" src="/blog/images/datafusion-34.0.0/compare-new.png"/&gt;
&lt;figcaption&gt;
&lt;b&gt;Figure 1&lt;/b&gt;: Performance improvement between &lt;code&gt;25.0.0&lt;/code&gt; and &lt;code&gt;34.0.0&lt;/code&gt; on ClickBench. 
    Note that DataFusion &lt;code&gt;25.0.0&lt;/code&gt;, could not run several queries due to 
    unsupported SQL (Q9, Q11, Q12, Q14) or memory requirements (Q33).
  &lt;/figcaption&gt;
&lt;/figure&gt;
&lt;figure class="text-center"&gt;
&lt;img alt="Fig 1: Adaptive Arrow schema architecture overview." class="img-fluid" src="/blog/images/datafusion-34.0.0/compare.png"/&gt;
&lt;figcaption&gt;
&lt;b&gt;Figure 2&lt;/b&gt;: Total query runtime for DataFusion &lt;code&gt;34.0.0&lt;/code&gt; and DataFusion &lt;code&gt;25.0.0&lt;/code&gt;.
  &lt;/figcaption&gt;
&lt;/figure&gt;
&lt;p&gt;Here are some specific enhancements we have made to improve performance:
* &lt;a href="https://arrow.apache.org/blog/2023/08/05/datafusion_fast_grouping/"&gt;2-3x better aggregation performance with many distinct groups&lt;/a&gt;
* Partially ordered grouping / streaming grouping
* [Specialized operator for "TopK" &lt;code&gt;ORDER BY LIMIT XXX&lt;/code&gt;] 
* [Specialized operator for &lt;code&gt;min(col) GROUP BY .. ORDER by min(col) LIMIT XXX&lt;/code&gt;]
* &lt;a href="https://github.com/apache/arrow-datafusion/pull/8126"&gt;Improved join performance&lt;/a&gt;
* Eliminate redundant sorting with sort order aware optimizers&lt;/p&gt;
&lt;h1 id="new-features"&gt;New Features ✨&lt;a class="headerlink" href="#new-features" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h1&gt;
&lt;h2 id="dml-insert-creating-files"&gt;DML / Insert / Creating Files&lt;a class="headerlink" href="#dml-insert-creating-files" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;DataFusion now supports writing data in parallel, to individual or multiple
files, using &lt;code&gt;Parquet&lt;/code&gt;, &lt;code&gt;CSV&lt;/code&gt;, &lt;code&gt;JSON&lt;/code&gt;, &lt;code&gt;ARROW&lt;/code&gt; and user defined formats.
&lt;a href="https://github.com/apache/arrow-datafusion/pull/7655"&gt;Benchmark results&lt;/a&gt; show improvements up to 5x in some cases.&lt;/p&gt;
&lt;p&gt;Similarly to reading, data can now be written to any [&lt;code&gt;ObjectStore&lt;/code&gt;]
implementation, including AWS S3, Azure Blob Storage, GCP Cloud Storage, local
files, and user defined implementations. While reading from &lt;a href="https://docs.rs/datafusion/latest/datafusion/datasource/listing/struct.ListingTable.html#features"&gt;hive style
partitioned tables&lt;/a&gt; has long been supported, it is now possible to write to such
tables as well.&lt;/p&gt;
&lt;p&gt;For example, to write to a local file:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-sql"&gt;❯ CREATE EXTERNAL TABLE awesome_table(x INT) STORED AS PARQUET LOCATION '/tmp/my_awesome_table';
0 rows in set. Query took 0.003 seconds.

❯ INSERT INTO awesome_table SELECT x * 10 FROM my_source_table;
+-------+
| count |
+-------+
| 3     |
+-------+
1 row in set. Query took 0.024 seconds.
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;You can also write to files with the [&lt;code&gt;COPY&lt;/code&gt;], similarly to [DuckDB’s &lt;code&gt;COPY&lt;/code&gt;]:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-sql"&gt;❯ COPY (SELECT x + 1 FROM my_source_table) TO '/tmp/output.json';
+-------+
| count |
+-------+
| 3     |
+-------+
1 row in set. Query took 0.014 seconds.
&lt;/code&gt;&lt;/pre&gt;
&lt;pre&gt;&lt;code class="language-shell"&gt;$ cat /tmp/output.json
{"x":1}
{"x":2}
{"x":3}
&lt;/code&gt;&lt;/pre&gt;
&lt;h2 id="improved-struct-and-array-support"&gt;Improved &lt;code&gt;STRUCT&lt;/code&gt; and &lt;code&gt;ARRAY&lt;/code&gt; support&lt;a class="headerlink" href="#improved-struct-and-array-support" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;DataFusion &lt;code&gt;34.0.0&lt;/code&gt; has much improved &lt;code&gt;STRUCT&lt;/code&gt; and &lt;code&gt;ARRAY&lt;/code&gt;
support, including a full range of &lt;a href="https://arrow.apache.org/datafusion/user-guide/sql/scalar_functions.html#struct-functions"&gt;struct functions&lt;/a&gt; and &lt;a href="https://arrow.apache.org/datafusion/user-guide/sql/scalar_functions.html#array-functions"&gt;array functions&lt;/a&gt;.&lt;/p&gt;
&lt;!--
❯ create table my_table as values ([1,2,3]), ([2]), ([4,5]);
--&gt;
&lt;p&gt;For example, you can now use &lt;code&gt;[]&lt;/code&gt; syntax and &lt;code&gt;array_length&lt;/code&gt; to access and inspect arrays:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-sql"&gt;❯ SELECT column1, 
         column1[1] AS first_element, 
         array_length(column1) AS len 
  FROM my_table;
+-----------+---------------+-----+
| column1   | first_element | len |
+-----------+---------------+-----+
| [1, 2, 3] | 1             | 3   |
| [2]       | 2             | 1   |
| [4, 5]    | 4             | 2   |
+-----------+---------------+-----+
&lt;/code&gt;&lt;/pre&gt;
&lt;pre&gt;&lt;code class="language-sql"&gt;❯ SELECT column1, column1['c0'] FROM  my_table;
+------------------+----------------------+
| column1          | my_table.column1[c0] |
+------------------+----------------------+
| {c0: foo, c1: 1} | foo                  |
| {c0: bar, c1: 2} | bar                  |
+------------------+----------------------+
2 rows in set. Query took 0.002 seconds.
&lt;/code&gt;&lt;/pre&gt;
&lt;h2 id="other-features"&gt;Other Features&lt;a class="headerlink" href="#other-features" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;Other notable features include:
* Support aggregating datasets that exceed memory size, with &lt;a href="https://github.com/apache/arrow-datafusion/pull/7400"&gt;group by spill to disk&lt;/a&gt;
* All operators now track and limit their memory consumption, including Joins&lt;/p&gt;
&lt;h1 id="building-systems-is-easier-with-datafusion"&gt;Building Systems is Easier with DataFusion 🛠️&lt;a class="headerlink" href="#building-systems-is-easier-with-datafusion" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h1&gt;
&lt;h2 id="documentation"&gt;Documentation&lt;a class="headerlink" href="#documentation" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;It is easier than ever to get started using DataFusion with the
new &lt;a href="https://arrow.apache.org/datafusion/library-user-guide/index.html"&gt;Library Users Guide&lt;/a&gt; as well as significantly improved the &lt;a href="https://docs.rs/datafusion/latest/datafusion/index.html"&gt;API documentation&lt;/a&gt;. &lt;/p&gt;
&lt;h2 id="user-defined-window-and-table-functions"&gt;User Defined Window and Table Functions&lt;a class="headerlink" href="#user-defined-window-and-table-functions" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;In addition to DataFusion's &lt;a href="https://arrow.apache.org/datafusion/library-user-guide/adding-udfs.html#adding-a-scalar-udf"&gt;User Defined Scalar Functions&lt;/a&gt;, and &lt;a href="https://arrow.apache.org/datafusion/library-user-guide/adding-udfs.html#adding-an-aggregate-udf"&gt;User Defined Aggregate Functions&lt;/a&gt;, DataFusion now supports &lt;a href="https://arrow.apache.org/datafusion/library-user-guide/adding-udfs.html#adding-a-window-udf"&gt;User Defined Window Functions&lt;/a&gt; 
 and &lt;a href="https://arrow.apache.org/datafusion/library-user-guide/adding-udfs.html#adding-a-user-defined-table-function"&gt;User Defined Table Functions&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;For example, [the &lt;code&gt;datafusion-cli&lt;/code&gt;] implements a DuckDB style [&lt;code&gt;parquet_metadata&lt;/code&gt;]
function as a user defined table function (&lt;a href="https://github.com/apache/arrow-datafusion/blob/3f219bc929cfd418b0e3d3501f8eba1d5a2c87ae/datafusion-cli/src/functions.rs#L222-L248"&gt;source code here&lt;/a&gt;): &lt;/p&gt;
&lt;pre&gt;&lt;code class="language-sql"&gt;❯ SELECT 
      path_in_schema, row_group_id, row_group_num_rows, stats_min, stats_max, total_compressed_size 
FROM 
      parquet_metadata('hits.parquet')
WHERE path_in_schema = '"WatchID"' 
LIMIT 3;

+----------------+--------------+--------------------+---------------------+---------------------+-----------------------+
| path_in_schema | row_group_id | row_group_num_rows | stats_min           | stats_max           | total_compressed_size |
+----------------+--------------+--------------------+---------------------+---------------------+-----------------------+
| "WatchID"      | 0            | 450560             | 4611687214012840539 | 9223369186199968220 | 3883759               |
| "WatchID"      | 1            | 612174             | 4611689135232456464 | 9223371478009085789 | 5176803               |
| "WatchID"      | 2            | 344064             | 4611692774829951781 | 9223363791697310021 | 3031680               |
+----------------+--------------+--------------------+---------------------+---------------------+-----------------------+
3 rows in set. Query took 0.053 seconds.
&lt;/code&gt;&lt;/pre&gt;
&lt;h3 id="growth-of-datafusion"&gt;Growth of DataFusion 📈&lt;a class="headerlink" href="#growth-of-datafusion" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;DataFusion has been appearing more publicly in the wild. For example
* New projects built using DataFusion such as &lt;a href="https://lancedb.com/"&gt;lancedb&lt;/a&gt;, &lt;a href="https://glaredb.com/"&gt;GlareDB&lt;/a&gt;, &lt;a href="https://www.arroyo.dev/"&gt;Arroyo&lt;/a&gt;, and &lt;a href="https://github.com/cmu-db/optd"&gt;optd&lt;/a&gt;.
* Public talks such as &lt;a href="https://www.youtube.com/watch?v=AJU9rdRNk9I"&gt;Apache Arrow Datafusion: Vectorized
  Execution Framework For Maximum Performance&lt;/a&gt; in &lt;a href="https://www.bagevent.com/event/8432178"&gt;CommunityOverCode Asia 2023&lt;/a&gt; 
* Blogs posts such as &lt;a href="https://www.synnada.ai/blog/apache-arrow-arrow-datafusion-ai-native-data-infra-an-interview-with-our-ceo-ozan"&gt;Apache Arrow, Arrow/DataFusion, AI-native Data Infra&lt;/a&gt;,
  &lt;a href="https://www.influxdata.com/blog/flight-datafusion-arrow-parquet-fdap-architecture-influxdb/"&gt;Flight, DataFusion, Arrow, and Parquet: Using the FDAP Architecture to build InfluxDB 3.0&lt;/a&gt;, and 
  &lt;a href="https://www.linkedin.com/pulse/guide-user-defined-functions-apache-arrow-datafusion-dade-aderemi/"&gt;A Guide to User-Defined Functions in Apache Arrow DataFusion&lt;/a&gt;&lt;/p&gt;
&lt;p&gt;We have also &lt;a href="https://github.com/apache/arrow-datafusion/issues/6782"&gt;submitted a paper&lt;/a&gt; to &lt;a href="https://2024.sigmod.org/"&gt;SIGMOD 2024&lt;/a&gt;, one of the
premiere database conferences, describing DataFusion in a technically formal
style and making the case that it is possible to create a modular and extensive query engine 
without sacrificing performance. We hope this paper helps people 
evaluating DataFusion for their needs understand it better.&lt;/p&gt;
&lt;h1 id="datafusion-in-2024"&gt;DataFusion in 2024 🥳&lt;a class="headerlink" href="#datafusion-in-2024" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h1&gt;
&lt;p&gt;Some major initiatives from contributors we know of this year are:&lt;/p&gt;
&lt;ol&gt;
&lt;li&gt;
&lt;p&gt;&lt;em&gt;Modularity&lt;/em&gt;: Make DataFusion even more modular, such as &lt;a href="https://github.com/apache/arrow-datafusion/issues/8045"&gt;unifying
   built in and user functions&lt;/a&gt;, making it easier to customize 
   DataFusion's behavior.&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;
&lt;p&gt;&lt;em&gt;Community Growth&lt;/em&gt;: Graduate to our own top level Apache project, and
   subsequently add more committers and PMC members to keep pace with project
   growth.&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;
&lt;p&gt;&lt;em&gt;Use case white papers&lt;/em&gt;: Write blog posts and videos explaining
   how to use DataFusion for real-world use cases.&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;
&lt;p&gt;&lt;em&gt;Testing&lt;/em&gt;: Improve CI infrastructure and test coverage, more fuzz
   testing, and better functional and performance regression testing.&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;
&lt;p&gt;&lt;em&gt;Planning Time&lt;/em&gt;: Reduce the time taken to plan queries, both &lt;a href="https://github.com/apache/arrow-datafusion/issues/7698"&gt;wide
   tables of 1000s of columns&lt;/a&gt;, and in &lt;a href="https://github.com/apache/arrow-datafusion/issues/5637"&gt;general&lt;/a&gt;.&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;
&lt;p&gt;&lt;em&gt;Aggregate Performance&lt;/em&gt;: Improve the speed of &lt;a href="https://github.com/apache/arrow-datafusion/issues/7000"&gt;aggregating "high cardinality"&lt;/a&gt; data
   when there are many (e.g. millions) of distinct groups.&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;
&lt;p&gt;&lt;em&gt;Statistics&lt;/em&gt;: &lt;a href="https://github.com/apache/arrow-datafusion/issues/8227"&gt;Improved statistics handling&lt;/a&gt; with an eye towards more
   sophisticated expression analysis and cost models.&lt;/p&gt;
&lt;/li&gt;
&lt;/ol&gt;
&lt;h1 id="how-to-get-involved"&gt;How to Get Involved&lt;a class="headerlink" href="#how-to-get-involved" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h1&gt;
&lt;p&gt;If you are interested in contributing to DataFusion we would love to have you
join us. You can try out DataFusion on some of your own data and projects and
let us know how it goes, contribute suggestions, documentation, bug reports, or
a PR with documentation, tests or code. A list of open issues
suitable for beginners is &lt;a href="https://github.com/apache/arrow-datafusion/issues?q=is%3Aissue+is%3Aopen+label%3A%22good+first+issue%22"&gt;here&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;As the community grows, we are also looking to restart biweekly calls /
meetings. Timezones are always a challenge for such meetings, but we hope to
have two calls that can work for most attendees. If you are interested
in helping, or just want to say hi, please drop us a note via one of 
the methods listed in our &lt;a href="https://arrow.apache.org/datafusion/contributor-guide/communication.html"&gt;Communication Doc&lt;/a&gt;.&lt;/p&gt;</content><category term="blog"/></entry><entry><title>Aggregating Millions of Groups Fast in Apache Arrow DataFusion 28.0.0</title><link href="https://datafusion.apache.org/blog/2023/08/05/datafusion_fast_grouping" rel="alternate"/><published>2023-08-05T00:00:00+00:00</published><updated>2023-08-05T00:00:00+00:00</updated><author><name>alamb, Dandandan, tustvold</name></author><id>tag:datafusion.apache.org,2023-08-05:/blog/2023/08/05/datafusion_fast_grouping</id><summary type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;!-- Converted from Google Docs using https://www.buymeacoffee.com/docstomarkdown --&gt;
&lt;h2 id="aggregating-millions-of-groups-fast-in-apache-arrow-datafusion"&gt;Aggregating Millions of Groups Fast in Apache Arrow DataFusion&lt;a class="headerlink" href="#aggregating-millions-of-groups-fast-in-apache-arrow-datafusion" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;Andrew Lamb, Daniël Heres, Raphael Taylor-Davies,&lt;/p&gt;
&lt;p&gt;&lt;em&gt;Note: this article was originally published on the &lt;a href="https://www.influxdata.com/blog/aggregating-millions-groups-fast-apache-arrow-datafusion"&gt;InfluxData Blog&lt;/a&gt;&lt;/em&gt;&lt;/p&gt;
&lt;h2 id="tldr"&gt;TLDR&lt;a class="headerlink" href="#tldr" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;Grouped aggregations are a core part of any analytic tool, creating understandable summaries of huge data volumes. &lt;a href="https://arrow.apache.org/datafusion/"&gt;Apache Arrow DataFusion&lt;/a&gt;’s parallel aggregation capability …&lt;/p&gt;</summary><content type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;!-- Converted from Google Docs using https://www.buymeacoffee.com/docstomarkdown --&gt;
&lt;h2 id="aggregating-millions-of-groups-fast-in-apache-arrow-datafusion"&gt;Aggregating Millions of Groups Fast in Apache Arrow DataFusion&lt;a class="headerlink" href="#aggregating-millions-of-groups-fast-in-apache-arrow-datafusion" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;Andrew Lamb, Daniël Heres, Raphael Taylor-Davies,&lt;/p&gt;
&lt;p&gt;&lt;em&gt;Note: this article was originally published on the &lt;a href="https://www.influxdata.com/blog/aggregating-millions-groups-fast-apache-arrow-datafusion"&gt;InfluxData Blog&lt;/a&gt;&lt;/em&gt;&lt;/p&gt;
&lt;h2 id="tldr"&gt;TLDR&lt;a class="headerlink" href="#tldr" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;Grouped aggregations are a core part of any analytic tool, creating understandable summaries of huge data volumes. &lt;a href="https://arrow.apache.org/datafusion/"&gt;Apache Arrow DataFusion&lt;/a&gt;’s parallel aggregation capability is 2-3x faster in the &lt;a href="https://crates.io/crates/datafusion/28.0.0"&gt;newly released version &lt;code&gt;28.0.0&lt;/code&gt;&lt;/a&gt; for queries with a large number (10,000 or more) of groups.&lt;/p&gt;
&lt;p&gt;Improving aggregation performance matters to all users of DataFusion. For example, both InfluxDB, a &lt;a href="https://github.com/influxdata/influxdb"&gt;time series data platform&lt;/a&gt; and Coralogix, a &lt;a href="https://coralogix.com/?utm_source=InfluxDB&amp;amp;utm_medium=Blog&amp;amp;utm_campaign=organic"&gt;full-stack observability&lt;/a&gt; platform, aggregate vast amounts of raw data to monitor and create insights for our customers. Improving DataFusion’s performance lets us provide better user experiences by generating insights faster with fewer resources. Because DataFusion is open source and released under the permissive &lt;a href="https://github.com/apache/arrow-datafusion/blob/main/LICENSE.txt"&gt;Apache 2.0&lt;/a&gt; license, the whole DataFusion community benefits as well.&lt;/p&gt;
&lt;p&gt;With the new optimizations, DataFusion’s grouping speed is now close to DuckDB, a system that regularly reports &lt;a href="https://duckdblabs.github.io/db-benchmark/"&gt;great&lt;/a&gt; &lt;a href="https://duckdb.org/2022/03/07/aggregate-hashtable.html#experiments"&gt;grouping&lt;/a&gt; benchmark performance numbers. Figure 1 contains a representative sample of &lt;a href="https://github.com/ClickHouse/ClickBench/tree/main"&gt;ClickBench&lt;/a&gt; on a single Parquet file, and the full results are at the end of this article.&lt;/p&gt;
&lt;p&gt;&lt;img src="/blog/images/datafusion_fast_grouping/summary.png" width="700"/&gt;&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Figure 1&lt;/strong&gt;: Query performance for ClickBench queries on queries 16, 17, 18 and 19 on a single Parquet file for DataFusion &lt;code&gt;27.0.0&lt;/code&gt;, DataFusion &lt;code&gt;28.0.0&lt;/code&gt; and DuckDB &lt;code&gt;0.8.1&lt;/code&gt;.&lt;/p&gt;
&lt;h2 id="introduction-to-high-cardinality-grouping"&gt;Introduction to high cardinality grouping&lt;a class="headerlink" href="#introduction-to-high-cardinality-grouping" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;Aggregation is a fancy word for computing summary statistics across many rows that have the same value in one or more columns. We call the rows with the same values &lt;em&gt;groups&lt;/em&gt; and “high cardinality” means there are a large number of distinct groups in the dataset. At the time of writing, a “large” number of groups in analytic engines is around 10,000.&lt;/p&gt;
&lt;p&gt;For example the &lt;a href="https://github.com/ClickHouse/ClickBench"&gt;ClickBench&lt;/a&gt; &lt;em&gt;hits&lt;/em&gt; dataset contains 100 million anonymized user clicks across a set of websites. ClickBench Query 17 is:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-sql"&gt;SELECT "UserID", "SearchPhrase", COUNT(*)
FROM hits
GROUP BY "UserID", "SearchPhrase"
ORDER BY COUNT(*)
DESC LIMIT 10;
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;In English, this query finds “the top ten (user, search phrase) combinations, across all clicks” and produces the following results (there are no search phrases for the top ten users):&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-text"&gt;+---------------------+--------------+-----------------+
| UserID              | SearchPhrase | COUNT(UInt8(1)) |
+---------------------+--------------+-----------------+
| 1313338681122956954 |              | 29097           |
| 1907779576417363396 |              | 25333           |
| 2305303682471783379 |              | 10597           |
| 7982623143712728547 |              | 6669            |
| 7280399273658728997 |              | 6408            |
| 1090981537032625727 |              | 6196            |
| 5730251990344211405 |              | 6019            |
| 6018350421959114808 |              | 5990            |
| 835157184735512989  |              | 5209            |
| 770542365400669095  |              | 4906            |
+---------------------+--------------+-----------------+
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;The ClickBench dataset contains&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;99,997,497 total rows[^1]&lt;/li&gt;
&lt;li&gt;17,630,976 different users (distinct UserIDs)[^2]&lt;/li&gt;
&lt;li&gt;6,019,103 different search phrases[^3]&lt;/li&gt;
&lt;li&gt;24,070,560 distinct combinations[^4] of (UserID, SearchPhrase)
Thus, to answer the query, DataFusion must map each of the 100M different input rows into one of the &lt;strong&gt;24 million different groups&lt;/strong&gt;, and keep count of how many such rows there are in each group.&lt;/li&gt;
&lt;/ul&gt;
&lt;h2 id="the-solution"&gt;The solution&lt;a class="headerlink" href="#the-solution" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;Like most concepts in databases and other analytic systems, the basic ideas of this algorithm are straightforward and taught in introductory computer science courses. You could compute the query with a program such as this[^5]:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-python"&gt;import pandas as pd
from collections import defaultdict
from operator import itemgetter

# read file
hits = pd.read_parquet('hits.parquet', engine='pyarrow')

# build groups
counts = defaultdict(int)
for index, row in hits.iterrows():
    group = (row['UserID'], row['SearchPhrase']);
    # update the dict entry for the corresponding key
    counts[group] += 1

# Print the top 10 values
print (dict(sorted(counts.items(), key=itemgetter(1), reverse=True)[:10]))
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;This approach, while simple, is both slow and very memory inefficient. It requires over 40 seconds to compute the results for less than 1% of the dataset[^6]. Both DataFusion &lt;code&gt;28.0.0&lt;/code&gt; and DuckDB &lt;code&gt;0.8.1&lt;/code&gt; compute results in under 10 seconds for the &lt;em&gt;entire&lt;/em&gt; dataset.&lt;/p&gt;
&lt;p&gt;To answer this query quickly and efficiently, you have to write your code such that it:&lt;/p&gt;
&lt;ol&gt;
&lt;li&gt;Keeps all cores busy aggregating via parallelized computation&lt;/li&gt;
&lt;li&gt;Updates aggregate values quickly, using vectorizable loops that are easy for compilers to translate into the high performance &lt;a href="https://en.wikipedia.org/wiki/Single_instruction,_multiple_data"&gt;SIMD&lt;/a&gt; instructions available in modern CPUs.&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;The rest of this article explains how grouping works in DataFusion and the improvements we made in &lt;code&gt;28.0.0&lt;/code&gt;.&lt;/p&gt;
&lt;h3 id="two-phase-parallel-partitioned-grouping"&gt;Two phase parallel partitioned grouping&lt;a class="headerlink" href="#two-phase-parallel-partitioned-grouping" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;Both DataFusion &lt;code&gt;27.0.&lt;/code&gt; and &lt;code&gt;28.0.0&lt;/code&gt; use state-of-the-art, two phase parallel hash partitioned grouping, similar to other high-performance vectorized engines like &lt;a href="https://duckdb.org/2022/03/07/aggregate-hashtable.html"&gt;DuckDB’s Parallel Grouped Aggregates&lt;/a&gt;. In pictures this looks like:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-text"&gt;            ▲                        ▲
            │                        │
            │                        │
            │                        │
┌───────────────────────┐  ┌───────────────────┐
│        GroupBy        │  │      GroupBy      │      Step 4
│        (Final)        │  │      (Final)      │
└───────────────────────┘  └───────────────────┘
            ▲                        ▲
            │                        │
            └────────────┬───────────┘
                         │
                         │
            ┌─────────────────────────┐
            │       Repartition       │               Step 3
            │         HASH(x)         │
            └─────────────────────────┘
                         ▲
                         │
            ┌────────────┴──────────┐
            │                       │
            │                       │
 ┌────────────────────┐  ┌─────────────────────┐
 │      GroupyBy      │  │       GroupBy       │      Step 2
 │     (Partial)      │  │      (Partial)      │
 └────────────────────┘  └─────────────────────┘
            ▲                       ▲
         ┌──┘                       └─┐
         │                            │
    .─────────.                  .─────────.
 ,─'           '─.            ,─'           '─.
;      Input      :          ;      Input      :      Step 1
:    Stream 1     ;          :    Stream 2     ;
 ╲               ╱            ╲               ╱
  '─.         ,─'              '─.         ,─'
     `───────'                    `───────'
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;&lt;strong&gt;Figure 2&lt;/strong&gt;: Two phase repartitioned grouping: data flows from bottom (source) to top (results) in two phases. First (Steps 1 and 2), each core reads the data into a core-specific hash table, computing intermediate aggregates without any cross-core coordination. Then (Steps 3 and 4) DataFusion divides the data (“repartitions”) into distinct subsets by group value, and each subset is sent to a specific core which computes the final aggregate.&lt;/p&gt;
&lt;p&gt;The two phases are critical for keeping cores busy in a multi-core system. Both phases use the same hash table approach (explained in the next section), but differ in how the groups are distributed and the partial results emitted from the accumulators. The first phase aggregates data as soon as possible after it is produced. However, as shown in Figure 2, the groups can be anywhere in any input, so the same group is often found on many different cores. The second phase uses a hash function to redistribute data evenly across the cores, so each group value is processed by exactly one core which emits the final results for that group.&lt;/p&gt;
&lt;pre&gt;&lt;code&gt;    ┌─────┐    ┌─────┐
    │  1  │    │  3  │
    │  2  │    │  4  │   2. After Repartitioning: each
    └─────┘    └─────┘   group key  appears in exactly
    ┌─────┐    ┌─────┐   one partition
    │  1  │    │  3  │
    │  2  │    │  4  │
    └─────┘    └─────┘

─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─

    ┌─────┐    ┌─────┐
    │  2  │    │  2  │
    │  1  │    │  2  │
    │  3  │    │  3  │
    │  4  │    │  1  │
    └─────┘    └─────┘    1. Input Stream: groups
      ...        ...      values are spread
    ┌─────┐    ┌─────┐    arbitrarily over each input
    │  1  │    │  4  │
    │  4  │    │  3  │
    │  1  │    │  1  │
    │  4  │    │  3  │
    │  3  │    │  2  │
    │  2  │    │  2  │
    │  2  │    └─────┘
    └─────┘

    Core A      Core B

&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;&lt;strong&gt;Figure 3&lt;/strong&gt;: Group value distribution across 2 cores during aggregation phases. In the first phase, every group value &lt;code&gt;1&lt;/code&gt;, &lt;code&gt;2&lt;/code&gt;, &lt;code&gt;3&lt;/code&gt;, &lt;code&gt;4&lt;/code&gt;, is present in the input stream processed by each core. In the second phase, after repartitioning, the group values &lt;code&gt;1&lt;/code&gt; and &lt;code&gt;2&lt;/code&gt; are processed by core A, and values &lt;code&gt;3&lt;/code&gt; and &lt;code&gt;4&lt;/code&gt; are processed only by core B.&lt;/p&gt;
&lt;p&gt;There are some additional subtleties in the &lt;a href="https://github.com/apache/arrow-datafusion/blob/main/datafusion/core/src/physical_plan/aggregates/row_hash.rs"&gt;DataFusion implementation&lt;/a&gt; not mentioned above due to space constraints, such as:&lt;/p&gt;
&lt;ol&gt;
&lt;li&gt;The policy of when to emit data from the first phase’s hash table (e.g. because the data is partially sorted)&lt;/li&gt;
&lt;li&gt;Handling specific filters per aggregate (due to the &lt;code&gt;FILTER&lt;/code&gt; SQL clause)&lt;/li&gt;
&lt;li&gt;Data types of intermediate values (which may not be the same as the final output for some aggregates such as &lt;code&gt;AVG&lt;/code&gt;).&lt;/li&gt;
&lt;li&gt;Action taken when memory use exceeds its budget.&lt;/li&gt;
&lt;/ol&gt;
&lt;h3 id="hash-grouping"&gt;Hash grouping&lt;a class="headerlink" href="#hash-grouping" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;DataFusion queries can compute many different aggregate functions for each group, both &lt;a href="https://arrow.apache.org/datafusion/user-guide/sql/aggregate_functions.html"&gt;built in&lt;/a&gt; and/or user defined &lt;a href="https://docs.rs/datafusion/latest/datafusion/logical_expr/struct.AggregateUDF.html"&gt;&lt;code&gt;AggregateUDFs&lt;/code&gt;&lt;/a&gt;. The state for each aggregate function, called an &lt;em&gt;accumulator&lt;/em&gt;, is tracked with a hash table (DataFusion uses the excellent &lt;a href="https://docs.rs/hashbrown/latest/hashbrown/index.html"&gt;HashBrown&lt;/a&gt; &lt;a href="https://docs.rs/hashbrown/latest/hashbrown/raw/struct.RawTable.html"&gt;RawTable API&lt;/a&gt;), which logically stores the “index”  identifying the specific group value.&lt;/p&gt;
&lt;h3 id="hash-grouping-in-2700"&gt;Hash grouping in &lt;code&gt;27.0.0&lt;/code&gt;&lt;a class="headerlink" href="#hash-grouping-in-2700" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;As shown in Figure 3, DataFusion &lt;code&gt;27.0.0&lt;/code&gt; stores the data in a &lt;a href="https://github.com/apache/arrow-datafusion/blob/4d93b6a3802151865b68967bdc4c7d7ef425b49a/datafusion/core/src/physical_plan/aggregates/utils.rs#L38-L50"&gt;&lt;code&gt;GroupState&lt;/code&gt;&lt;/a&gt; structure which, unsurprisingly, tracks the state for each group. The state for each group consists of:&lt;/p&gt;
&lt;ol&gt;
&lt;li&gt;The actual value of the group columns, in &lt;a href="https://docs.rs/arrow-row/latest/arrow_row/index.html"&gt;Arrow Row&lt;/a&gt; format.&lt;/li&gt;
&lt;li&gt;In-progress accumulations (e.g. the running counts for the &lt;code&gt;COUNT&lt;/code&gt; aggregate) for each group, in one of two possible formats (&lt;a href="https://github.com/apache/arrow-datafusion/blob/a6dcd943051a083693c352c6b4279156548490a0/datafusion/expr/src/accumulator.rs#L24-L49"&gt;&lt;code&gt;Accumulator&lt;/code&gt;&lt;/a&gt;  or &lt;a href="https://github.com/apache/arrow-datafusion/blob/a6dcd943051a083693c352c6b4279156548490a0/datafusion/physical-expr/src/aggregate/row_accumulator.rs#L26-L46"&gt;&lt;code&gt;RowAccumulator&lt;/code&gt;&lt;/a&gt;).&lt;/li&gt;
&lt;li&gt;Scratch space for tracking which rows match each aggregate in each batch.&lt;/li&gt;
&lt;/ol&gt;
&lt;pre&gt;&lt;code&gt;                           ┌──────────────────────────────────────┐
                           │                                      │
                           │                  ...                 │
                           │ ┏━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓ │
                           │ ┃                                  ┃ │
    ┌─────────┐            │ ┃ ┌──────────────────────────────┐ ┃ │
    │         │            │ ┃ │group values: OwnedRow        │ ┃ │
    │ ┌─────┐ │            │ ┃ └──────────────────────────────┘ ┃ │
    │ │  5  │ │            │ ┃ ┌──────────────────────────────┐ ┃ │
    │ ├─────┤ │            │ ┃ │Row accumulator:              │ ┃ │
    │ │  9  │─┼────┐       │ ┃ │Vec&amp;lt;u8&amp;gt;                       │ ┃ │
    │ ├─────┤ │    │       │ ┃ └──────────────────────────────┘ ┃ │
    │ │ ... │ │    │       │ ┃ ┌──────────────────────┐         ┃ │
    │ ├─────┤ │    │       │ ┃ │┌──────────────┐      │         ┃ │
    │ │  1  │ │    │       │ ┃ ││Accumulator 1 │      │         ┃ │
    │ ├─────┤ │    │       │ ┃ │└──────────────┘      │         ┃ │
    │ │ ... │ │    │       │ ┃ │┌──────────────┐      │         ┃ │
    │ └─────┘ │    │       │ ┃ ││Accumulator 2 │      │         ┃ │
    │         │    │       │ ┃ │└──────────────┘      │         ┃ │
    └─────────┘    │       │ ┃ │ Box&amp;lt;dyn Accumulator&amp;gt; │         ┃ │
    Hash Table     │       │ ┃ └──────────────────────┘         ┃ │
                   │       │ ┃ ┌─────────────────────────┐      ┃ │
                   │       │ ┃ │scratch indices: Vec&amp;lt;u32&amp;gt;│      ┃ │
                   │       │ ┃ └─────────────────────────┘      ┃ │
                   │       │ ┃ GroupState                       ┃ │
                   └─────▶ │ ┗━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┛ │
                           │                                      │
  Hash table tracks an     │                 ...                  │
  index into group_states  │                                      │
                           └──────────────────────────────────────┘
                           group_states: Vec&amp;lt;GroupState&amp;gt;

                           There is one GroupState PER GROUP

&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;&lt;strong&gt;Figure 4&lt;/strong&gt;: Hash group operator structure in DataFusion &lt;code&gt;27.0.0&lt;/code&gt;. A hash table maps each group to a GroupState which contains all the per-group states.&lt;/p&gt;
&lt;p&gt;To compute the aggregate, DataFusion performs the following steps for each input batch:&lt;/p&gt;
&lt;ol&gt;
&lt;li&gt;Calculate hash using &lt;a href="https://github.com/apache/arrow-datafusion/blob/a6dcd943051a083693c352c6b4279156548490a0/datafusion/physical-expr/src/hash_utils.rs#L264-L307"&gt;efficient vectorized code&lt;/a&gt;, specialized for each data type.&lt;/li&gt;
&lt;li&gt;Determine group indexes for each input row using the hash table (creating new entries for newly seen groups).&lt;/li&gt;
&lt;li&gt;&lt;a href="https://github.com/apache/arrow-datafusion/blob/4ab8be57dee3bfa72dd105fbd7b8901b873a4878/datafusion/core/src/physical_plan/aggregates/row_hash.rs#L562-L602"&gt;Update Accumulators for each group that had input rows,&lt;/a&gt; assembling the rows into a contiguous range for vectorized accumulator if there are a sufficient number of them.&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;DataFusion also stores the hash values in the table to avoid potentially costly hash recomputation when resizing the hash table.&lt;/p&gt;
&lt;p&gt;This scheme works very well for a relatively small number of distinct groups: all accumulators are efficiently updated with large contiguous batches of rows.&lt;/p&gt;
&lt;p&gt;However, this scheme is not ideal for high cardinality grouping due to:&lt;/p&gt;
&lt;ol&gt;
&lt;li&gt;&lt;strong&gt;Multiple allocations per group&lt;/strong&gt; for the group value row format, as well as for the &lt;code&gt;RowAccumulator&lt;/code&gt;s and each  &lt;code&gt;Accumulator&lt;/code&gt;. The &lt;code&gt;Accumulator&lt;/code&gt; may have additional allocations within it as well.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Non-vectorized updates:&lt;/strong&gt; Accumulator updates often fall back to a slower non-vectorized form because the number of distinct groups is large (and thus number of values per group is small) in each input batch.&lt;/li&gt;
&lt;/ol&gt;
&lt;h3 id="hash-grouping-in-2800"&gt;Hash grouping in &lt;code&gt;28.0.0&lt;/code&gt;&lt;a class="headerlink" href="#hash-grouping-in-2800" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;For &lt;code&gt;28.0.0&lt;/code&gt;, we rewrote the core group by implementation following traditional system optimization principles: fewer allocations, type specialization, and aggressive vectorization.&lt;/p&gt;
&lt;p&gt;DataFusion &lt;code&gt;28.0.0&lt;/code&gt; uses the same RawTable and still stores group indexes. The major differences, as shown in Figure 4, are:&lt;/p&gt;
&lt;ol&gt;
&lt;li&gt;Group values are stored either&lt;ol&gt;
&lt;li&gt;Inline in the &lt;code&gt;RawTable&lt;/code&gt; (for single columns of primitive types), where the conversion to Row format costs more than its benefit&lt;/li&gt;
&lt;li&gt;In a separate &lt;a href="https://docs.rs/arrow-row/latest/arrow_row/struct.Row.html"&gt;Rows&lt;/a&gt; structure with a single contiguous allocation for all groups values, rather than an allocation per group. Accumulators manage the state for all the groups internally, so the code to update intermediate values is a tight type specialized loop. The new &lt;a href="https://github.com/apache/arrow-datafusion/blob/a6dcd943051a083693c352c6b4279156548490a0/datafusion/physical-expr/src/aggregate/groups_accumulator/mod.rs#L66-L75"&gt;&lt;code&gt;GroupsAccumulator&lt;/code&gt;&lt;/a&gt; interface results in highly efficient type accumulator update loops.&lt;/li&gt;
&lt;/ol&gt;
&lt;/li&gt;
&lt;/ol&gt;
&lt;pre&gt;&lt;code&gt;┌───────────────────────────────────┐     ┌───────────────────────┐
│ ┌ ─ ─ ─ ─ ─ ┐  ┌─────────────────┐│     │ ┏━━━━━━━━━━━━━━━━━━━┓ │
│                │                 ││     │ ┃  ┌──────────────┐ ┃ │
│ │           │  │ ┌ ─ ─ ┐┌─────┐  ││     │ ┃  │┌───────────┐ │ ┃ │
│                │    X   │  5  │  ││     │ ┃  ││  value1   │ │ ┃ │
│ │           │  │ ├ ─ ─ ┤├─────┤  ││     │ ┃  │└───────────┘ │ ┃ │
│                │    Q   │  9  │──┼┼──┐  │ ┃  │     ...      │ ┃ │
│ │           │  │ ├ ─ ─ ┤├─────┤  ││  └──┼─╋─▶│              │ ┃ │
│                │   ...  │ ... │  ││     │ ┃  │┌───────────┐ │ ┃ │
│ │           │  │ ├ ─ ─ ┤├─────┤  ││     │ ┃  ││  valueN   │ │ ┃ │
│                │    H   │  1  │  ││     │ ┃  │└───────────┘ │ ┃ │
│ │           │  │ ├ ─ ─ ┤├─────┤  ││     │ ┃  │values: Vec&amp;lt;T&amp;gt;│ ┃ │
│     Rows       │   ...  │ ... │  ││     │ ┃  └──────────────┘ ┃ │
│ │           │  │ └ ─ ─ ┘└─────┘  ││     │ ┃                   ┃ │
│  ─ ─ ─ ─ ─ ─   │                 ││     │ ┃ GroupsAccumulator ┃ │
│                └─────────────────┘│     │ ┗━━━━━━━━━━━━━━━━━━━┛ │
│                  Hash Table       │     │                       │
│                                   │     │          ...          │
└───────────────────────────────────┘     └───────────────────────┘
  GroupState                               Accumulators


Hash table value stores group_indexes     One  GroupsAccumulator
and group values.                         per aggregate. Each
                                          stores the state for
Group values are stored either inline     *ALL* groups, typically
in the hash table or in a single          using a native Vec&amp;lt;T&amp;gt;
allocation using the arrow Row format
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;&lt;strong&gt;Figure 5&lt;/strong&gt;: Hash group operator structure in DataFusion &lt;code&gt;28.0.0&lt;/code&gt;. Group values are stored either directly in the hash table, or in a single allocation using the arrow Row format. The hash table contains group indexes. A single &lt;code&gt;GroupsAccumulator&lt;/code&gt; stores the per-aggregate state for &lt;em&gt;all&lt;/em&gt; groups.&lt;/p&gt;
&lt;p&gt;This new structure improves performance significantly for high cardinality groups due to:&lt;/p&gt;
&lt;ol&gt;
&lt;li&gt;&lt;strong&gt;Reduced allocations&lt;/strong&gt;: There are no longer any individual allocations per group.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Contiguous native accumulator states&lt;/strong&gt;: Type-specialized accumulators store the values for all groups in a single contiguous allocation using a &lt;a href="https://doc.rust-lang.org/std/vec/struct.Vec.html"&gt;Rust Vec&amp;lt;T&amp;gt;&lt;/a&gt; of some native type.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Vectorized state update&lt;/strong&gt;: The inner aggregate update loops, which are type-specialized and in terms of native &lt;code&gt;Vec&lt;/code&gt;s, are well-vectorized by the Rust compiler (thanks &lt;a href="https://llvm.org/"&gt;LLVM&lt;/a&gt;!).&lt;/li&gt;
&lt;/ol&gt;
&lt;h3 id="notes"&gt;Notes&lt;a class="headerlink" href="#notes" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;Some vectorized grouping implementations store the accumulator state row-wise directly in the hash table, which often uses modern CPU caches efficiently. Managing accumulator state in columnar fashion may sacrifice some cache locality, however it ensures the size of the hash table remains small, even when there are large numbers of groups and aggregates, making it easier for the compiler to vectorize the accumulator update.&lt;/p&gt;
&lt;p&gt;Depending on the cost of recomputing hash values, DataFusion &lt;code&gt;28.0.0&lt;/code&gt; may or may not store the hash values in the table. This optimizes the tradeoff between the cost of computing the hash value (which is expensive for strings, for example) vs. the cost of storing it in the hash table.&lt;/p&gt;
&lt;p&gt;One subtlety that arises from pushing state updates into GroupsAccumulators is that each accumulator must handle similar variations with/without filtering and with/without nulls in the input. DataFusion &lt;code&gt;28.0.0&lt;/code&gt; uses a templated &lt;a href="https://github.com/apache/arrow-datafusion/blob/a6dcd943051a083693c352c6b4279156548490a0/datafusion/physical-expr/src/aggregate/groups_accumulator/accumulate.rs#L28-L54"&gt;&lt;code&gt;NullState&lt;/code&gt;&lt;/a&gt; which encapsulates these common patterns across accumulators.&lt;/p&gt;
&lt;p&gt;The code structure is heavily influenced by the fact DataFusion is implemented using &lt;a href="https://www.rust-lang.org/"&gt;Rust&lt;/a&gt;, a new(ish) systems programming language focused on speed and safety. Rust heavily discourages many of the traditional pointer casting “tricks” used in C/C++ hash grouping implementations. The DataFusion aggregation code is almost entirely &lt;a href="https://doc.rust-lang.org/nomicon/meet-safe-and-unsafe.html#:~:text=Safe%20Rust%20is%20the%20true,Undefined%20Behavior%20(a.k.a.%20UB)."&gt;&lt;code&gt;safe&lt;/code&gt;&lt;/a&gt;, deviating into &lt;code&gt;unsafe&lt;/code&gt; only when necessary. (Rust is a great choice because it makes DataFusion fast, easy to embed, and prevents many crashes and security issues often associated with multi-threaded C/C++ code).&lt;/p&gt;
&lt;h2 id="clickbench-results"&gt;ClickBench results&lt;a class="headerlink" href="#clickbench-results" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;The full results of running the &lt;a href="https://github.com/ClickHouse/ClickBench/tree/main"&gt;ClickBench&lt;/a&gt; queries against the single Parquet file with DataFusion &lt;code&gt;27.0.0&lt;/code&gt;, DataFusion &lt;code&gt;28.0.0&lt;/code&gt;, and DuckDB &lt;code&gt;0.8.1&lt;/code&gt; are below. These numbers were run on a GCP &lt;code&gt;e2-standard-8 machine&lt;/code&gt; with 8 cores and 32 GB of RAM, using the scripts &lt;a href="https://github.com/alamb/datafusion-duckdb-benchmark"&gt;here&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;As the industry moves towards data systems assembled from components, it is increasingly important that they exchange data using open standards such as &lt;a href="https://arrow.apache.org/"&gt;Apache Arrow&lt;/a&gt; and &lt;a href="https://parquet.apache.org/"&gt;Parquet&lt;/a&gt; rather than custom storage and in-memory formats. Thus, this benchmark uses a single input Parquet file representative of many DataFusion users and aligned with the current trend in analytics of avoiding a costly load/transformation into a custom storage format prior to query.&lt;/p&gt;
&lt;p&gt;DataFusion now reaches near-DuckDB-speeds querying Parquet data. While we don’t plan to engage in a benchmarking shootout with a team that literally wrote &lt;a href="https://dl.acm.org/doi/abs/10.1145/3209950.3209955"&gt;Fair Benchmarking Considered Difficult&lt;/a&gt;, hopefully everyone can agree that DataFusion &lt;code&gt;28.0.0&lt;/code&gt; is a significant improvement.&lt;/p&gt;
&lt;p&gt;&lt;img src="/blog/images/datafusion_fast_grouping/full.png" width="700"/&gt;&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Figure 6&lt;/strong&gt;: Performance of DataFusion &lt;code&gt;27.0.0&lt;/code&gt;, DataFusion &lt;code&gt;28.0.0&lt;/code&gt;, and DuckDB &lt;code&gt;0.8.1&lt;/code&gt; on all 43 ClickBench queries against a single &lt;code&gt;hits.parquet&lt;/code&gt; file. Lower is better.&lt;/p&gt;
&lt;h3 id="notes_1"&gt;Notes&lt;a class="headerlink" href="#notes_1" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;DataFusion &lt;code&gt;27.0.0&lt;/code&gt; was not able to run several queries due to either planner bugs (Q9, Q11, Q12, 14) or running out of memory (Q33). DataFusion &lt;code&gt;28.0.0&lt;/code&gt; solves those issues.&lt;/p&gt;
&lt;p&gt;DataFusion is faster than DuckDB for query 21 and 22, likely due to optimized implementations of string pattern matching.&lt;/p&gt;
&lt;h2 id="conclusion-performance-matters"&gt;Conclusion: performance matters&lt;a class="headerlink" href="#conclusion-performance-matters" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;Improving aggregation performance by more than a factor of two allows developers building products and projects with DataFusion to spend more time on value-added domain specific features. We believe building systems with DataFusion is much faster than trying to build something similar from scratch. DataFusion increases productivity because it eliminates the need to rebuild well-understood, but costly to implement, analytic database technology. While we’re pleased with the improvements in DataFusion &lt;code&gt;28.0.0&lt;/code&gt;, we are by no means done and are pursuing &lt;a href="https://github.com/apache/arrow-datafusion/issues/7000"&gt;(Even More) Aggregation Performance&lt;/a&gt;. The future for performance is bright.&lt;/p&gt;
&lt;h2 id="acknowledgments"&gt;Acknowledgments&lt;a class="headerlink" href="#acknowledgments" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;DataFusion is a &lt;a href="https://arrow.apache.org/datafusion/contributor-guide/communication.html"&gt;community effort&lt;/a&gt; and this work was not possible without contributions from many in the community. A special shout out to &lt;a href="https://github.com/sunchao"&gt;sunchao&lt;/a&gt;, &lt;a href="https://github.com/jyshen"&gt;yjshen&lt;/a&gt;, &lt;a href="https://github.com/yahoNanJing"&gt;yahoNanJing&lt;/a&gt;, &lt;a href="https://github.com/mingmwang"&gt;mingmwang&lt;/a&gt;, &lt;a href="https://github.com/ozankabak"&gt;ozankabak&lt;/a&gt;, &lt;a href="https://github.com/mustafasrepo"&gt;mustafasrepo&lt;/a&gt;, and everyone else who contributed ideas, reviews, and encouragement &lt;a href="https://github.com/apache/arrow-datafusion/pull/6800"&gt;during&lt;/a&gt; this &lt;a href="https://github.com/apache/arrow-datafusion/pull/6904"&gt;work&lt;/a&gt;.&lt;/p&gt;
&lt;h2 id="about-datafusion"&gt;About DataFusion&lt;a class="headerlink" href="#about-datafusion" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;&lt;a href="https://arrow.apache.org/datafusion/"&gt;Apache Arrow DataFusion&lt;/a&gt; is an extensible query engine and database toolkit, written in &lt;a href="https://www.rust-lang.org/"&gt;Rust&lt;/a&gt;, that uses &lt;a href="https://arrow.apache.org/"&gt;Apache Arrow&lt;/a&gt; as its in-memory format. DataFusion, along with &lt;a href="https://calcite.apache.org/"&gt;Apache Calcite&lt;/a&gt;, Facebook’s &lt;a href="https://github.com/facebookincubator/velox"&gt;Velox&lt;/a&gt;, and similar technology are part of the next generation “&lt;a href="https://www.usenix.org/publications/login/winter2018/khurana"&gt;Deconstructed Database&lt;/a&gt;” architectures, where new systems are built on a foundation of fast, modular components, rather than as a single tightly integrated system.&lt;/p&gt;
&lt;!-- Footnotes themselves at the bottom. --&gt;
&lt;h2 id="notes_2"&gt;Notes&lt;a class="headerlink" href="#notes_2" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;[^1]: &lt;code&gt;SELECT COUNT(*) FROM 'hits.parquet';&lt;/code&gt;&lt;/p&gt;
&lt;p&gt;[^2]: &lt;code&gt;SELECT COUNT(DISTINCT "UserID") as num_users FROM 'hits.parquet';&lt;/code&gt;&lt;/p&gt;
&lt;p&gt;[^3]: &lt;code&gt;SELECT COUNT(DISTINCT "SearchPhrase") as num_phrases FROM 'hits.parquet';&lt;/code&gt;&lt;/p&gt;
&lt;p&gt;[^4]: &lt;code&gt;SELECT COUNT(*) FROM (SELECT DISTINCT "UserID", "SearchPhrase" FROM 'hits.parquet')&lt;/code&gt;&lt;/p&gt;
&lt;p&gt;[^5]: Full script at &lt;a href="https://github.com/alamb/datafusion-duckdb-benchmark/blob/main/hash.py"&gt;hash.py&lt;/a&gt;&lt;/p&gt;
&lt;p&gt;[^6]: &lt;a href="https://datasets.clickhouse.com/hits_compatible/athena_partitioned/hits_%7B%7D.parquet"&gt;hits_0.parquet&lt;/a&gt;, one of the files from the partitioned ClickBench dataset, which has &lt;code&gt;100,000&lt;/code&gt; rows and is 117 MB in size. The entire dataset has &lt;code&gt;100,000,000&lt;/code&gt; rows in a single 14 GB Parquet file. The script did not complete on the entire dataset after 40 minutes, and used 212 GB RAM at peak.&lt;/p&gt;</content><category term="blog"/></entry><entry><title>Apache Arrow DataFusion 26.0.0</title><link href="https://datafusion.apache.org/blog/2023/06/24/datafusion-25.0.0" rel="alternate"/><published>2023-06-24T00:00:00+00:00</published><updated>2023-06-24T00:00:00+00:00</updated><author><name>pmc</name></author><id>tag:datafusion.apache.org,2023-06-24:/blog/2023/06/24/datafusion-25.0.0</id><summary type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;p&gt;It has been a whirlwind 6 months of DataFusion development since &lt;a href="https://arrow.apache.org/blog/2023/01/19/datafusion-16.0.0"&gt;our
last update&lt;/a&gt;: the community has grown, many features have been added,
performance improved and we are &lt;a href="https://github.com/apache/arrow-datafusion/discussions/6475"&gt;discussing&lt;/a&gt; branching out to our own
top level Apache Project.&lt;/p&gt;
&lt;h2 id="background"&gt;Background&lt;a class="headerlink" href="#background" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;&lt;a href="https://arrow.apache.org/datafusion/"&gt;Apache Arrow DataFusion&lt;/a&gt; is an extensible query engine and database
toolkit …&lt;/p&gt;</summary><content type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;p&gt;It has been a whirlwind 6 months of DataFusion development since &lt;a href="https://arrow.apache.org/blog/2023/01/19/datafusion-16.0.0"&gt;our
last update&lt;/a&gt;: the community has grown, many features have been added,
performance improved and we are &lt;a href="https://github.com/apache/arrow-datafusion/discussions/6475"&gt;discussing&lt;/a&gt; branching out to our own
top level Apache Project.&lt;/p&gt;
&lt;h2 id="background"&gt;Background&lt;a class="headerlink" href="#background" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;&lt;a href="https://arrow.apache.org/datafusion/"&gt;Apache Arrow DataFusion&lt;/a&gt; is an extensible query engine and database
toolkit, written in &lt;a href="https://www.rust-lang.org/"&gt;Rust&lt;/a&gt;, that uses &lt;a href="https://arrow.apache.org"&gt;Apache Arrow&lt;/a&gt; as its in-memory
format.&lt;/p&gt;
&lt;p&gt;DataFusion, along with &lt;a href="https://calcite.apache.org"&gt;Apache Calcite&lt;/a&gt;, Facebook's &lt;a href="https://github.com/facebookincubator/velox"&gt;Velox&lt;/a&gt; and
similar technology are part of the next generation "&lt;a href="https://www.usenix.org/publications/login/winter2018/khurana"&gt;Deconstructed
Database&lt;/a&gt;" architectures, where new systems are built on a foundation
of fast, modular components, rather as a single tightly integrated
system.&lt;/p&gt;
&lt;p&gt;While single tightly integrated systems such as &lt;a href="https://spark.apache.org/"&gt;Spark&lt;/a&gt;, &lt;a href="https://duckdb.org"&gt;DuckDB&lt;/a&gt; and
&lt;a href="https://www.pola.rs/"&gt;Pola.rs&lt;/a&gt; are great pieces of technology, our community believes that
anyone developing new data heavy application, such as those common in
machine learning in the next 5 years, will &lt;strong&gt;require&lt;/strong&gt; a high
performance, vectorized, query engine to remain relevant. The only
practical way to gain access to such technology without investing many
millions of dollars to build a new tightly integrated engine, is
though open source projects like DataFusion and similar enabling
technologies such as &lt;a href="https://arrow.apache.org"&gt;Apache Arrow&lt;/a&gt; and &lt;a href="https://www.rust-lang.org/"&gt;Rust&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;DataFusion is targeted primarily at developers creating other data
intensive analytics, and offers:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;High performance, native, parallel streaming execution engine&lt;/li&gt;
&lt;li&gt;Mature &lt;a href="https://arrow.apache.org/datafusion/user-guide/sql/index.html"&gt;SQL support&lt;/a&gt;, featuring  subqueries, window functions, grouping sets, and more&lt;/li&gt;
&lt;li&gt;Built in support for Parquet, Avro, CSV, JSON and Arrow formats and easy extension for others&lt;/li&gt;
&lt;li&gt;Native DataFrame API and &lt;a href="https://arrow.apache.org/datafusion-python/"&gt;python bindings&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;&lt;a href="https://docs.rs/datafusion/latest/datafusion/index.html"&gt;Well documented&lt;/a&gt; source code and architecture, designed to be customized to suit downstream project needs&lt;/li&gt;
&lt;li&gt;High quality, easy to use code &lt;a href="https://crates.io/crates/datafusion/versions"&gt;released every 2 weeks to crates.io&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;Welcoming, open community, governed by the highly regarded and well understood &lt;a href="https://www.apache.org/"&gt;Apache Software Foundation&lt;/a&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;The rest of this post highlights some of the improvements we have made
to DataFusion over the last 6 months and a preview of where we are
heading. You can see a list of all changes in the detailed
&lt;a href="https://github.com/apache/arrow-datafusion/blob/main/datafusion/CHANGELOG.md"&gt;CHANGELOG&lt;/a&gt;.&lt;/p&gt;
&lt;h2 id="even-better-performance"&gt;(Even) Better Performance&lt;a class="headerlink" href="#even-better-performance" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;&lt;a href="https://voltrondata.com/resources/speeds-and-feeds-hardware-and-software-matter"&gt;Various&lt;/a&gt; benchmarks show DataFusion to be quite close or &lt;a href="https://github.com/tustvold/access-log-bench"&gt;even
faster&lt;/a&gt; to the state of the art in analytic performance (at the moment
this seems to be DuckDB). We continually work on improving performance
(see &lt;a href="https://github.com/apache/arrow-datafusion/issues/5546"&gt;#5546&lt;/a&gt; for a list) and would love additional help in this area.&lt;/p&gt;
&lt;p&gt;DataFusion now reads single large Parquet files significantly faster by
&lt;a href="https://github.com/apache/arrow-datafusion/pull/5057"&gt;parallelizing across multiple cores&lt;/a&gt;. Native speeds for reading JSON
and CSV files are also up to 2.5x faster thanks to improvements
upstream in arrow-rs &lt;a href="https://github.com/apache/arrow-rs/pull/3479#issuecomment-1384353159"&gt;JSON reader&lt;/a&gt; and &lt;a href="https://github.com/apache/arrow-rs/pull/3365"&gt;CSV reader&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;Also, we have integrated the &lt;a href="https://arrow.apache.org/blog/2022/11/07/multi-column-sorts-in-arrow-rust-part-1/"&gt;arrow-rs Row Format&lt;/a&gt; into DataFusion resulting in up to &lt;a href="https://github.com/apache/arrow-datafusion/pull/6163"&gt;2-3x faster sorting and merging&lt;/a&gt;.&lt;/p&gt;
&lt;h2 id="improved-documentation-and-website"&gt;Improved Documentation and Website&lt;a class="headerlink" href="#improved-documentation-and-website" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;Part of growing the DataFusion community is ensuring that DataFusion's
features are understood and that it is easy to contribute and
participate. To that end the &lt;a href="https://arrow.apache.org/datafusion/"&gt;website&lt;/a&gt; has been cleaned up, &lt;a href="https://docs.rs/datafusion/latest/datafusion/index.html#architecture"&gt;the
architecture guide&lt;/a&gt; expanded, the &lt;a href="https://arrow.apache.org/datafusion/contributor-guide/roadmap.html"&gt;roadmap&lt;/a&gt; updated, and several
overview talks created:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;Apr 2023 &lt;em&gt;Query Engine&lt;/em&gt;: &lt;a href="https://youtu.be/NVKujPxwSBA"&gt;recording&lt;/a&gt; and &lt;a href="https://docs.google.com/presentation/d/1D3GDVas-8y0sA4c8EOgdCvEjVND4s2E7I6zfs67Y4j8/edit#slide=id.p"&gt;slides&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;April 2023 &lt;em&gt;Logical Plan and Expressions&lt;/em&gt;: &lt;a href="https://youtu.be/EzZTLiSJnhY"&gt;recording&lt;/a&gt; and &lt;a href="https://docs.google.com/presentation/d/1ypylM3-w60kVDW7Q6S99AHzvlBgciTdjsAfqNP85K30"&gt;slides&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;April 2023 &lt;em&gt;Physical Plan and Execution&lt;/em&gt;: &lt;a href="https://youtu.be/2jkWU3_w6z0"&gt;recording&lt;/a&gt; and &lt;a href="https://docs.google.com/presentation/d/1cA2WQJ2qg6tx6y4Wf8FH2WVSm9JQ5UgmBWATHdik0hg"&gt;slides&lt;/a&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;h2 id="new-features"&gt;New Features&lt;a class="headerlink" href="#new-features" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;h3 id="more-streaming-less-memory"&gt;More Streaming, Less Memory&lt;a class="headerlink" href="#more-streaming-less-memory" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;We have made significant progress on the &lt;a href="https://github.com/apache/arrow-datafusion/issues/4285"&gt;streaming execution roadmap&lt;/a&gt;
such as &lt;a href="https://docs.rs/datafusion/latest/datafusion/physical_plan/trait.ExecutionPlan.html#method.unbounded_output"&gt;unbounded datasources&lt;/a&gt;, &lt;a href="https://docs.rs/datafusion/latest/datafusion/physical_plan/aggregates/enum.GroupByOrderMode.html"&gt;streaming group by&lt;/a&gt;, sophisticated
&lt;a href="https://docs.rs/datafusion/latest/datafusion/physical_optimizer/global_sort_selection/index.html"&gt;sort&lt;/a&gt; and &lt;a href="https://docs.rs/datafusion/latest/datafusion/physical_optimizer/repartition/index.html"&gt;repartitioning&lt;/a&gt; improvements in the optimizer, and support
for &lt;a href="https://docs.rs/datafusion/latest/datafusion/physical_plan/joins/struct.SymmetricHashJoinExec.html"&gt;symmetric hash join&lt;/a&gt; (read more about that in the great &lt;a href="https://www.synnada.ai/blog/general-purpose-stream-joins-via-pruning-symmetric-hash-joins"&gt;Synnada
Blog Post&lt;/a&gt; on the topic). Together, these features both 1) make it
easier to build streaming systems using DataFusion that can
incrementally generate output before (or ever) seeing the end of the
input and 2) allow general queries to use less memory and generate their
results faster.&lt;/p&gt;
&lt;p&gt;We have also improved the runtime &lt;a href="https://docs.rs/datafusion/latest/datafusion/execution/memory_pool/index.html"&gt;memory management&lt;/a&gt; system so that
DataFusion now stays within its declared memory budget &lt;a href="https://github.com/apache/arrow-datafusion/issues/3941"&gt;generate
runtime errors&lt;/a&gt;.&lt;/p&gt;
&lt;h3 id="dml-support-insert-delete-update-etc"&gt;DML Support (&lt;code&gt;INSERT&lt;/code&gt;, &lt;code&gt;DELETE&lt;/code&gt;, &lt;code&gt;UPDATE&lt;/code&gt;, etc)&lt;a class="headerlink" href="#dml-support-insert-delete-update-etc" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;Part of building high performance data systems includes writing data,
and DataFusion supports several features for creating new files:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;code&gt;INSERT INTO&lt;/code&gt; and &lt;code&gt;SELECT ... INTO&lt;/code&gt; support for memory backed and CSV tables&lt;/li&gt;
&lt;li&gt;New &lt;a href="https://docs.rs/datafusion/latest/datafusion/physical_plan/insert/trait.DataSink.html"&gt;API for writing data into TableProviders&lt;/a&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;We are working on easier to use &lt;a href="https://github.com/apache/arrow-datafusion/issues/5654"&gt;COPY INTO&lt;/a&gt; syntax, better support
for writing parquet, JSON, and AVRO, and more -- see our &lt;a href="https://github.com/apache/arrow-datafusion/issues/6569"&gt;tracking epic&lt;/a&gt;
for more details.&lt;/p&gt;
&lt;h3 id="timestamp-and-intervals"&gt;Timestamp and Intervals&lt;a class="headerlink" href="#timestamp-and-intervals" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;One mark of the maturity of a SQL engine is how it handles the tricky
world of timestamp, date, times and interval arithmetic. DataFusion is
feature complete in this area and behaves as you would expect,
supporting queries such as&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-sql"&gt;SELECT now() + '1 month' FROM my_table;
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;We still have a long tail of &lt;a href="https://github.com/apache/arrow-datafusion/issues/3148"&gt;date and time improvements&lt;/a&gt;, which we are working on as well.&lt;/p&gt;
&lt;h3 id="querying-structured-types-list-and-structs"&gt;Querying Structured Types (&lt;code&gt;List&lt;/code&gt; and &lt;code&gt;Struct&lt;/code&gt;s)&lt;a class="headerlink" href="#querying-structured-types-list-and-structs" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;Arrow and Parquet &lt;a href="https://arrow.apache.org/blog/2022/10/08/arrow-parquet-encoding-part-2/"&gt;support nested data&lt;/a&gt; well and DataFusion lets you
easily query such &lt;code&gt;Struct&lt;/code&gt; and &lt;code&gt;List&lt;/code&gt;. For example, you can use
DataFusion to read and query the &lt;a href="https://data.mendeley.com/datasets/ct8f9skv97"&gt;JSON Datasets for Exploratory OLAP -
Mendeley Data&lt;/a&gt; like this:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-sql"&gt;----------
-- Explore structured data using SQL
----------
SELECT delete FROM 'twitter-sample-head-100000.parquet' WHERE delete IS NOT NULL limit 10;
+---------------------------------------------------------------------------------------------------------------------------+
| delete                                                                                                                    |
+---------------------------------------------------------------------------------------------------------------------------+
| {status: {id: {$numberLong: 135037425050320896}, id_str: 135037425050320896, user_id: 334902461, user_id_str: 334902461}} |
| {status: {id: {$numberLong: 134703982051463168}, id_str: 134703982051463168, user_id: 405383453, user_id_str: 405383453}} |
| {status: {id: {$numberLong: 134773741740765184}, id_str: 134773741740765184, user_id: 64823441, user_id_str: 64823441}}   |
| {status: {id: {$numberLong: 132543659655704576}, id_str: 132543659655704576, user_id: 45917834, user_id_str: 45917834}}   |
| {status: {id: {$numberLong: 133786431926697984}, id_str: 133786431926697984, user_id: 67229952, user_id_str: 67229952}}   |
| {status: {id: {$numberLong: 134619093570560002}, id_str: 134619093570560002, user_id: 182430773, user_id_str: 182430773}} |
| {status: {id: {$numberLong: 134019857527214080}, id_str: 134019857527214080, user_id: 257396311, user_id_str: 257396311}} |
| {status: {id: {$numberLong: 133931546469076993}, id_str: 133931546469076993, user_id: 124539548, user_id_str: 124539548}} |
| {status: {id: {$numberLong: 134397743350296576}, id_str: 134397743350296576, user_id: 139836391, user_id_str: 139836391}} |
| {status: {id: {$numberLong: 127833661767823360}, id_str: 127833661767823360, user_id: 244442687, user_id_str: 244442687}} |
+---------------------------------------------------------------------------------------------------------------------------+

----------
-- Select some deeply nested fields
----------
SELECT
  delete['status']['id']['$numberLong'] as delete_id,
  delete['status']['user_id'] as delete_user_id
FROM 'twitter-sample-head-100000.parquet' WHERE delete IS NOT NULL LIMIT 10;

+--------------------+----------------+
| delete_id          | delete_user_id |
+--------------------+----------------+
| 135037425050320896 | 334902461      |
| 134703982051463168 | 405383453      |
| 134773741740765184 | 64823441       |
| 132543659655704576 | 45917834       |
| 133786431926697984 | 67229952       |
| 134619093570560002 | 182430773      |
| 134019857527214080 | 257396311      |
| 133931546469076993 | 124539548      |
| 134397743350296576 | 139836391      |
| 127833661767823360 | 244442687      |
+--------------------+----------------+
&lt;/code&gt;&lt;/pre&gt;
&lt;h3 id="subqueries-all-the-way-down"&gt;Subqueries All the Way Down&lt;a class="headerlink" href="#subqueries-all-the-way-down" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;DataFusion can run many different subqueries by rewriting them to
joins. It has been able to run the full suite of TPC-H queries for at
least the last year, but recently we have implemented significant
improvements to this logic, sufficient to run almost all queries in
the TPC-DS benchmark as well.&lt;/p&gt;
&lt;h2 id="community-and-project-growth"&gt;Community and Project Growth&lt;a class="headerlink" href="#community-and-project-growth" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;The six months since &lt;a href="https://arrow.apache.org/blog/2023/01/19/datafusion-16.0.0"&gt;our last update&lt;/a&gt; saw significant growth in
the DataFusion community. Between versions &lt;code&gt;17.0.0&lt;/code&gt; and &lt;code&gt;26.0.0&lt;/code&gt;,
DataFusion merged 711 PRs from 107 distinct contributors, not
including all the work that goes into our core dependencies such as
&lt;a href="https://crates.io/crates/arrow"&gt;arrow&lt;/a&gt;,
&lt;a href="https://crates.io/crates/parquet"&gt;parquet&lt;/a&gt;, and
&lt;a href="https://crates.io/crates/object_store"&gt;object_store&lt;/a&gt;, that much of
the same community helps support.&lt;/p&gt;
&lt;p&gt;In addition, we have added 7 new committers and 1 new PMC member to
the Apache Arrow project, largely focused on DataFusion, and we
learned about some of the cool &lt;a href="https://arrow.apache.org/datafusion/user-guide/introduction.html#known-users"&gt;new systems&lt;/a&gt; which are using
DataFusion. Given the growth of the community and interest in the
project, we also clarified the &lt;a href="https://github.com/apache/arrow-datafusion/discussions/6441"&gt;mission statement&lt;/a&gt; and are
&lt;a href="https://github.com/apache/arrow-datafusion/discussions/6475"&gt;discussing&lt;/a&gt; "graduate"ing DataFusion to a new top level
Apache Software Foundation project.&lt;/p&gt;
&lt;!--
$ git log --pretty=oneline 17.0.0..26.0.0 . | wc -l
     711

$ git shortlog -sn 17.0.0..26.0.0 . | wc -l
      107
--&gt;
&lt;h1 id="how-to-get-involved"&gt;How to Get Involved&lt;a class="headerlink" href="#how-to-get-involved" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h1&gt;
&lt;p&gt;Kudos to everyone in the community who has contributed ideas,
discussions, bug reports, documentation and code. It is exciting to be
innovating on the next generation of database architectures together!&lt;/p&gt;
&lt;p&gt;If you are interested in contributing to DataFusion, we would love to
have you join us. You can try out DataFusion on some of your own
data and projects and let us know how it goes or contribute a PR with
documentation, tests or code. A list of open issues suitable for
beginners is &lt;a href="https://github.com/apache/arrow-datafusion/issues?q=is%3Aissue+is%3Aopen+label%3A%22good+first+issue%22"&gt;here&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;Check out our &lt;a href="https://arrow.apache.org/datafusion/contributor-guide/communication.html"&gt;Communication Doc&lt;/a&gt; for more ways to engage with the
community.&lt;/p&gt;</content><category term="blog"/></entry><entry><title>Apache Arrow DataFusion 16.0.0 Project Update</title><link href="https://datafusion.apache.org/blog/2023/01/19/datafusion-16.0.0" rel="alternate"/><published>2023-01-19T00:00:00+00:00</published><updated>2023-01-19T00:00:00+00:00</updated><author><name>pmc</name></author><id>tag:datafusion.apache.org,2023-01-19:/blog/2023/01/19/datafusion-16.0.0</id><summary type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;h1 id="introduction"&gt;Introduction&lt;a class="headerlink" href="#introduction" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h1&gt;
&lt;p&gt;&lt;a href="https://arrow.apache.org/datafusion/"&gt;DataFusion&lt;/a&gt; is an extensible
query execution framework, written in &lt;a href="https://www.rust-lang.org/"&gt;Rust&lt;/a&gt;,
that uses &lt;a href="https://arrow.apache.org"&gt;Apache Arrow&lt;/a&gt; as its
in-memory format. It is targeted primarily at developers creating data
intensive analytics, and offers mature
&lt;a href="https://arrow.apache.org/datafusion/user-guide/sql/index.html"&gt;SQL support&lt;/a&gt;,
a DataFrame API, and many extension points.&lt;/p&gt;
&lt;p&gt;Systems based on DataFusion perform very well in benchmarks …&lt;/p&gt;</summary><content type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;h1 id="introduction"&gt;Introduction&lt;a class="headerlink" href="#introduction" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h1&gt;
&lt;p&gt;&lt;a href="https://arrow.apache.org/datafusion/"&gt;DataFusion&lt;/a&gt; is an extensible
query execution framework, written in &lt;a href="https://www.rust-lang.org/"&gt;Rust&lt;/a&gt;,
that uses &lt;a href="https://arrow.apache.org"&gt;Apache Arrow&lt;/a&gt; as its
in-memory format. It is targeted primarily at developers creating data
intensive analytics, and offers mature
&lt;a href="https://arrow.apache.org/datafusion/user-guide/sql/index.html"&gt;SQL support&lt;/a&gt;,
a DataFrame API, and many extension points.&lt;/p&gt;
&lt;p&gt;Systems based on DataFusion perform very well in benchmarks,
especially considering they operate directly on parquet files rather
than first loading into a specialized format.  Some recent highlights
include &lt;a href="https://benchmark.clickhouse.com/"&gt;clickbench&lt;/a&gt; and the
&lt;a href="https://www.cloudfuse.io/dashboards/standalone-engines"&gt;Cloudfuse.io standalone query
engines&lt;/a&gt; page.&lt;/p&gt;
&lt;p&gt;DataFusion is also part of a longer term trend, articulated clearly by
&lt;a href="http://www.cs.cmu.edu/~pavlo/"&gt;Andy Pavlo&lt;/a&gt; in his &lt;a href="https://ottertune.com/blog/2022-databases-retrospective/"&gt;2022 Databases
Retrospective&lt;/a&gt;.
Database frameworks are proliferating and it is likely that all OLAP
DBMSs and other data heavy applications, such as machine learning,
will &lt;strong&gt;require&lt;/strong&gt; a vectorized, highly performant query engine in the next
5 years to remain relevant.  The only practical way to make such
technology so widely available without many millions of dollars of
investment is though open source engine such as DataFusion or
&lt;a href="https://github.com/facebookincubator/velox"&gt;Velox&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;The rest of this post describes the improvements made to DataFusion
over the last three months and some hints of where we are heading.&lt;/p&gt;
&lt;h2 id="community-growth"&gt;Community Growth&lt;a class="headerlink" href="#community-growth" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;We again saw significant growth in the DataFusion community since &lt;a href="https://arrow.apache.org/blog/2022/10/25/datafusion-13.0.0/"&gt;our last update&lt;/a&gt;. There are some interesting metrics on &lt;a href="https://ossrank.com/p/1573-apache-arrow-datafusion"&gt;OSSRank&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;The DataFusion 16.0.0 release consists of 543 PRs from 73 distinct contributors, not including all the work that goes into dependencies such as &lt;a href="https://crates.io/crates/arrow"&gt;arrow&lt;/a&gt;, &lt;a href="https://crates.io/crates/parquet"&gt;parquet&lt;/a&gt;, and &lt;a href="https://crates.io/crates/object_store"&gt;object_store&lt;/a&gt;, that much of the same community helps support. Thank you all for your help&lt;/p&gt;
&lt;!--
$ git log --pretty=oneline 13.0.0..16.0.0 . | wc -l
     543

$ git shortlog -sn 13.0.0..16.0.0 . | wc -l
      73
--&gt;
&lt;p&gt;Several &lt;a href="https://github.com/apache/arrow-datafusion#known-uses"&gt;new systems based on DataFusion&lt;/a&gt; were recently added:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;a href="https://github.com/GreptimeTeam/greptimedb"&gt;Greptime DB&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;&lt;a href="https://synnada.ai/"&gt;Synnada&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;&lt;a href="https://github.com/PRQL/prql-query"&gt;PRQL&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;&lt;a href="https://github.com/parseablehq/parseable"&gt;Parseable&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;&lt;a href="https://github.com/splitgraph/seafowl"&gt;SeaFowl&lt;/a&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;h2 id="performance"&gt;Performance 🚀&lt;a class="headerlink" href="#performance" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;Performance and efficiency are core values for
DataFusion. While there is still a gap between DataFusion and the best of
breed, tightly integrated systems such as &lt;a href="https://duckdb.org"&gt;DuckDB&lt;/a&gt;
and &lt;a href="https://www.pola.rs/"&gt;Polars&lt;/a&gt;, DataFusion is
closing the gap quickly. Performance highlights from the last three
months:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;Up to 30% Faster Sorting and Merging using the new &lt;a href="https://arrow.apache.org/blog/2022/11/07/multi-column-sorts-in-arrow-rust-part-1/"&gt;Row Format&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;&lt;a href="https://arrow.apache.org/blog/2022/12/26/querying-parquet-with-millisecond-latency/"&gt;Advanced predicate pushdown&lt;/a&gt;, directly on parquet, directly from object storage, enabling sub millisecond filtering. &lt;!-- Andrew nots: we should really get this turned on by default --&gt;&lt;/li&gt;
&lt;li&gt;&lt;code&gt;70%&lt;/code&gt; faster &lt;code&gt;IN&lt;/code&gt; expressions evaluation (&lt;a href="https://github.com/apache/arrow-datafusion/issues/4057"&gt;#4057&lt;/a&gt;)&lt;/li&gt;
&lt;li&gt;Sort and partition aware optimizations (&lt;a href="https://github.com/apache/arrow-datafusion/issues/3969"&gt;#3969&lt;/a&gt; and  &lt;a href="https://github.com/apache/arrow-datafusion/issues/4691"&gt;#4691&lt;/a&gt;)&lt;/li&gt;
&lt;li&gt;Filter selectivity analysis (&lt;a href="https://github.com/apache/arrow-datafusion/issues/3868"&gt;#3868&lt;/a&gt;)&lt;/li&gt;
&lt;/ul&gt;
&lt;h2 id="runtime-resource-limits"&gt;Runtime Resource Limits&lt;a class="headerlink" href="#runtime-resource-limits" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;Previously, DataFusion could potentially use unbounded amounts of memory for certain queries that included Sorts, Grouping or Joins.&lt;/p&gt;
&lt;p&gt;In version 16.0.0, it is possible to limit DataFusion's memory usage for Sorting and Grouping. We are looking for help adding similar limiting for Joins as well as expanding our algorithms to optionally spill to secondary storage. See &lt;a href="https://github.com/apache/arrow-datafusion/issues/3941"&gt;#3941&lt;/a&gt; for more detail.&lt;/p&gt;
&lt;h2 id="sql-window-functions"&gt;SQL Window Functions&lt;a class="headerlink" href="#sql-window-functions" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;&lt;a href="https://en.wikipedia.org/wiki/Window_function_(SQL)"&gt;SQL Window Functions&lt;/a&gt; are useful for a variety of analysis and DataFusion's implementation support expanded significantly:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;Custom window frames such as &lt;code&gt;... OVER (ORDER BY ... RANGE BETWEEN 0.2 PRECEDING AND 0.2 FOLLOWING)&lt;/code&gt;&lt;/li&gt;
&lt;li&gt;Unbounded window frames such as &lt;code&gt;... OVER (ORDER BY ... RANGE UNBOUNDED ROWS PRECEDING)&lt;/code&gt;&lt;/li&gt;
&lt;li&gt;Support for the &lt;code&gt;NTILE&lt;/code&gt; window function (&lt;a href="https://github.com/apache/arrow-datafusion/issues/4676"&gt;#4676&lt;/a&gt;)&lt;/li&gt;
&lt;li&gt;Support for &lt;code&gt;GROUPS&lt;/code&gt; mode (&lt;a href="https://github.com/apache/arrow-datafusion/issues/4155"&gt;#4155&lt;/a&gt;)&lt;/li&gt;
&lt;/ul&gt;
&lt;h1 id="improved-joins"&gt;Improved Joins&lt;a class="headerlink" href="#improved-joins" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h1&gt;
&lt;p&gt;Joins are often the most complicated operations to handle well in
analytics systems and DataFusion 16.0.0 offers significant improvements
such as&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;Cost based optimizer (CBO) automatically reorders join evaluations, selects algorithms (Merge / Hash), and pick build side based on available statistics and join type (&lt;code&gt;INNER&lt;/code&gt;, &lt;code&gt;LEFT&lt;/code&gt;, etc) (&lt;a href="https://github.com/apache/arrow-datafusion/issues/4219"&gt;#4219&lt;/a&gt;)&lt;/li&gt;
&lt;li&gt;Fast non &lt;code&gt;column=column&lt;/code&gt; equijoins such as &lt;code&gt;JOIN ON a.x + 5 = b.y&lt;/code&gt;&lt;/li&gt;
&lt;li&gt;Better performance on non-equijoins (&lt;a href="https://github.com/apache/arrow-datafusion/issues/4562"&gt;#4562&lt;/a&gt;) &lt;!-- TODO is this a good thing to mention as any time this is usd the query is going to go slow or the data size is small --&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;h1 id="streaming-execution"&gt;Streaming Execution&lt;a class="headerlink" href="#streaming-execution" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h1&gt;
&lt;p&gt;One emerging use case for Datafusion is as a foundation for
streaming-first data platforms. An important prerequisite
is support for incremental execution for queries that can be computed
incrementally.&lt;/p&gt;
&lt;p&gt;With this release, DataFusion now supports the following streaming features:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;Data ingestion from infinite files such as FIFOs (&lt;a href="https://github.com/apache/arrow-datafusion/issues/4694"&gt;#4694&lt;/a&gt;),&lt;/li&gt;
&lt;li&gt;Detection of pipeline-breaking queries in streaming use cases (&lt;a href="https://github.com/apache/arrow-datafusion/issues/4694"&gt;#4694&lt;/a&gt;),&lt;/li&gt;
&lt;li&gt;Automatic input swapping for joins so probe side is a data stream (&lt;a href="https://github.com/apache/arrow-datafusion/issues/4694"&gt;#4694&lt;/a&gt;),&lt;/li&gt;
&lt;li&gt;Intelligent elision of pipeline-breaking sort operations whenever possible (&lt;a href="https://github.com/apache/arrow-datafusion/issues/4691"&gt;#4691&lt;/a&gt;),&lt;/li&gt;
&lt;li&gt;Incremental execution for more types of queries; e.g. queries involving finite window frames (&lt;a href="https://github.com/apache/arrow-datafusion/issues/4777"&gt;#4777&lt;/a&gt;).&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;These are a major steps forward, and we plan even more improvements over the next few releases.&lt;/p&gt;
&lt;h1 id="better-support-for-distributed-catalogs"&gt;Better Support for Distributed Catalogs&lt;a class="headerlink" href="#better-support-for-distributed-catalogs" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h1&gt;
&lt;p&gt;16.0.0 has been enhanced support for asynchronous catalogs (&lt;a href="https://github.com/apache/arrow-datafusion/issues/4607"&gt;#4607&lt;/a&gt;)
to better support distributed metadata stores such as
&lt;a href="https://delta.io/"&gt;Delta.io&lt;/a&gt; and &lt;a href="https://iceberg.apache.org/"&gt;Apache
Iceberg&lt;/a&gt; which require asynchronous I/O
during planning to access remote catalogs. Previously, DataFusion
required synchronous access to all relevant catalog information.&lt;/p&gt;
&lt;h1 id="additional-sql-support"&gt;Additional SQL Support&lt;a class="headerlink" href="#additional-sql-support" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h1&gt;
&lt;p&gt;SQL support continues to improve, including some of these highlights:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;Add TPC-DS query planning regression tests &lt;a href="https://github.com/apache/arrow-datafusion/issues/4719"&gt;#4719&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;Support for &lt;code&gt;PREPARE&lt;/code&gt; statement &lt;a href="https://github.com/apache/arrow-datafusion/issues/4490"&gt;#4490&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;Automatic coercions ast between Date and Timestamp &lt;a href="https://github.com/apache/arrow-datafusion/issues/4726"&gt;#4726&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;Support type coercion for timestamp and utf8 &lt;a href="https://github.com/apache/arrow-datafusion/issues/4312"&gt;#4312&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;Full support for time32 and time64 literal values (&lt;code&gt;ScalarValue&lt;/code&gt;) &lt;a href="https://github.com/apache/arrow-datafusion/issues/4156"&gt;#4156&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;New functions, including &lt;code&gt;uuid()&lt;/code&gt; &lt;a href="https://github.com/apache/arrow-datafusion/issues/4041"&gt;#4041&lt;/a&gt;, &lt;code&gt;current_time&lt;/code&gt; &lt;a href="https://github.com/apache/arrow-datafusion/issues/4054"&gt;#4054&lt;/a&gt;, &lt;code&gt;current_date&lt;/code&gt; &lt;a href="https://github.com/apache/arrow-datafusion/issues/4022"&gt;#4022&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;Compressed CSV/JSON support &lt;a href="https://github.com/apache/arrow-datafusion/issues/3642"&gt;#3642&lt;/a&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;The community has also invested in new &lt;a href="https://github.com/apache/arrow-datafusion/blob/master/datafusion/core/tests/sqllogictests/README.md"&gt;sqllogic based&lt;/a&gt; tests to keep improving DataFusion's quality with less effort.&lt;/p&gt;
&lt;h1 id="plan-serialization-and-substrait"&gt;Plan Serialization and Substrait&lt;a class="headerlink" href="#plan-serialization-and-substrait" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h1&gt;
&lt;p&gt;DataFusion now supports serialization of physical plans, with a custom protocol buffers format. In addition, we are adding initial support for &lt;a href="https://substrait.io/"&gt;Substrait&lt;/a&gt;, a Cross-Language Serialization for Relational Algebra&lt;/p&gt;
&lt;h1 id="how-to-get-involved"&gt;How to Get Involved&lt;a class="headerlink" href="#how-to-get-involved" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h1&gt;
&lt;p&gt;Kudos to everyone in the community who contributed ideas, discussions, bug reports, documentation and code. It is exciting to be building something so cool together!&lt;/p&gt;
&lt;p&gt;If you are interested in contributing to DataFusion, we would love to
have you join us. You can try out DataFusion on some of your own
data and projects and let us know how it goes or contribute a PR with
documentation, tests or code. A list of open issues suitable for
beginners is
&lt;a href="https://github.com/apache/arrow-datafusion/issues?q=is%3Aissue+is%3Aopen+label%3A%22good+first+issue%22"&gt;here&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;Check out our &lt;a href="https://arrow.apache.org/datafusion/community/communication.html"&gt;Communication Doc&lt;/a&gt; on more
ways to engage with the community.&lt;/p&gt;
&lt;h2 id="appendix-contributor-shoutout"&gt;Appendix: Contributor Shoutout&lt;a class="headerlink" href="#appendix-contributor-shoutout" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;Here is a list of people who have contributed PRs to this project over the last three releases, derived from &lt;code&gt;git shortlog -sn 13.0.0..16.0.0 .&lt;/code&gt; Thank you all!&lt;/p&gt;
&lt;pre&gt;&lt;code&gt;   113  Andrew Lamb
    58  jakevin
    46  Raphael Taylor-Davies
    30  Andy Grove
    19  Batuhan Taskaya
    19  Remzi Yang
    17  ygf11
    16  Burak
    16  Jeffrey
    16  Marco Neumann
    14  Kun Liu
    12  Yang Jiang
    10  mingmwang
     9  Daniël Heres
     9  Mustafa akur
     9  comphead
     9  mvanschellebeeck
     9  xudong.w
     7  dependabot[bot]
     7  yahoNanJing
     6  Brent Gardner
     5  AssHero
     4  Jiayu Liu
     4  Wei-Ting Kuo
     4  askoa
     3  André Calado Coroado
     3  Jie Han
     3  Jon Mease
     3  Metehan Yıldırım
     3  Nga Tran
     3  Ruihang Xia
     3  baishen
     2  Berkay Şahin
     2  Dan Harris
     2  Dongyan Zhou
     2  Eduard Karacharov
     2  Kikkon
     2  Liang-Chi Hsieh
     2  Marko Milenković
     2  Martin Grigorov
     2  Roman Nozdrin
     2  Tim Van Wassenhove
     2  r.4ntix
     2  unconsolable
     2  unvalley
     1  Ajaya Agrawal
     1  Alexander Spies
     1  ArkashaJavelin
     1  Artjoms Iskovs
     1  BoredPerson
     1  Christian Salvati
     1  Creampanda
     1  Data Psycho
     1  Francis Du
     1  Francis Le Roy
     1  LFC
     1  Marko Grujic
     1  Matt Willian
     1  Matthijs Brobbel
     1  Max Burke
     1  Mehmet Ozan Kabak
     1  Rito Takeuchi
     1  Roman Zeyde
     1  Vrishabh
     1  Zhang Li
     1  ZuoTiJia
     1  byteink
     1  cfraz89
     1  nbr
     1  xxchan
     1  yujie.zhang
     1  zembunia
     1  哇呜哇呜呀咦耶
&lt;/code&gt;&lt;/pre&gt;</content><category term="blog"/></entry><entry><title>Apache Arrow Ballista 0.9.0 Release</title><link href="https://datafusion.apache.org/blog/2022/10/28/ballista-0.9.0" rel="alternate"/><published>2022-10-28T00:00:00+00:00</published><updated>2022-10-28T00:00:00+00:00</updated><author><name>pmc</name></author><id>tag:datafusion.apache.org,2022-10-28:/blog/2022/10/28/ballista-0.9.0</id><summary type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;h1 id="introduction"&gt;Introduction&lt;a class="headerlink" href="#introduction" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h1&gt;
&lt;p&gt;&lt;a href="https://github.com/apache/arrow-ballista"&gt;Ballista&lt;/a&gt; is an Arrow-native distributed SQL query engine implemented in Rust.&lt;/p&gt;
&lt;p&gt;Ballista 0.9.0 is now available and is the most significant release since the project was &lt;a href="http://arrow.apache.org/blog/2021/04/12/ballista-donation/"&gt;donated&lt;/a&gt; to Apache
Arrow in 2021.&lt;/p&gt;
&lt;p&gt;This release represents 4 weeks of work, with 66 commits from 14 contributors:&lt;/p&gt;
&lt;pre&gt;&lt;code&gt;    22  Andy …&lt;/code&gt;&lt;/pre&gt;</summary><content type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;h1 id="introduction"&gt;Introduction&lt;a class="headerlink" href="#introduction" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h1&gt;
&lt;p&gt;&lt;a href="https://github.com/apache/arrow-ballista"&gt;Ballista&lt;/a&gt; is an Arrow-native distributed SQL query engine implemented in Rust.&lt;/p&gt;
&lt;p&gt;Ballista 0.9.0 is now available and is the most significant release since the project was &lt;a href="http://arrow.apache.org/blog/2021/04/12/ballista-donation/"&gt;donated&lt;/a&gt; to Apache
Arrow in 2021.&lt;/p&gt;
&lt;p&gt;This release represents 4 weeks of work, with 66 commits from 14 contributors:&lt;/p&gt;
&lt;pre&gt;&lt;code&gt;    22  Andy Grove
    12  yahoNanJing
     6  Daniël Heres
     4  Brent Gardner
     4  dependabot[bot]
     4  r.4ntix
     3  Stefan Stanciulescu
     3  mingmwang
     2  Ken Suenobu
     2  Yang Jiang
     1  Metehan Yıldırım
     1  Trent Feda
     1  askoa
     1  yangzhong
&lt;/code&gt;&lt;/pre&gt;
&lt;h2 id="release-highlights"&gt;Release Highlights&lt;a class="headerlink" href="#release-highlights" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;The release notes below are not exhaustive and only expose selected highlights of the release. Many other bug fixes
and improvements have been made: we refer you to the &lt;a href="https://github.com/apache/arrow-ballista/blob/0.9.0-rc2/ballista/CHANGELOG.md"&gt;complete changelog&lt;/a&gt;.&lt;/p&gt;
&lt;h3 id="support-for-cloud-object-stores-and-distributed-file-systems"&gt;Support for Cloud Object Stores and Distributed File Systems&lt;a class="headerlink" href="#support-for-cloud-object-stores-and-distributed-file-systems" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;This is the first release of Ballista to have documented support for querying data from distributed file systems and
object stores. Currently, S3 and HDFS are supported. Support for Google Cloud Storage and Azure Blob Storage is planned
for the next release.&lt;/p&gt;
&lt;h3 id="flight-sql-jdbc-support"&gt;Flight SQL &amp;amp; JDBC support&lt;a class="headerlink" href="#flight-sql-jdbc-support" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;The Ballista scheduler now implements the &lt;a href="https://arrow.apache.org/blog/2022/02/16/introducing-arrow-flight-sql/"&gt;Flight SQL protocol&lt;/a&gt;, enabling any compliant Flight SQL client
to connect to and run queries against a Ballista cluster.&lt;/p&gt;
&lt;p&gt;The Apache Arrow Flight SQL JDBC driver can be used to connect Business Intelligence tools to a Ballista cluster.&lt;/p&gt;
&lt;h3 id="python-bindings"&gt;Python Bindings&lt;a class="headerlink" href="#python-bindings" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;It is now possible to connect to a Ballista cluster from Python and execute queries using both the DataFrame and SQL
interfaces.&lt;/p&gt;
&lt;h3 id="scheduler-web-user-interface-and-rest-api"&gt;Scheduler Web User Interface and REST API&lt;a class="headerlink" href="#scheduler-web-user-interface-and-rest-api" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;The scheduler now has a web user interface for monitoring queries. It is also possible to view graphical query plans
that show how the query was executed, along with metrics.&lt;/p&gt;
&lt;p&gt;&lt;img src="/blog/images/2022-10-28-ballista-web-ui.png" width="800"/&gt;&lt;/p&gt;
&lt;p&gt;The REST API that powers the user interface can also be accessed directly.&lt;/p&gt;
&lt;h3 id="simplified-kubernetes-deployment"&gt;Simplified Kubernetes Deployment&lt;a class="headerlink" href="#simplified-kubernetes-deployment" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;Ballista now provides a &lt;a href="https://github.com/apache/arrow-ballista/tree/master/helm"&gt;Helm chart&lt;/a&gt; for simplified Kubernetes deployment.&lt;/p&gt;
&lt;h3 id="user-guide"&gt;User Guide&lt;a class="headerlink" href="#user-guide" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;The user guide is published at &lt;a href="https://arrow.apache.org/ballista/"&gt;https://arrow.apache.org/ballista/&lt;/a&gt; and provides
deployment instructions for Docker, Docker Compose, and Kubernetes, as well as references for configuring and
tuning Ballista.&lt;/p&gt;
&lt;h2 id="roadmap"&gt;Roadmap&lt;a class="headerlink" href="#roadmap" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;The Ballista community is currently focused on the following tasks for the next release:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;Support for Azure Blob Storage and Google Cloud Storage&lt;/li&gt;
&lt;li&gt;Improve benchmark performance by implementing more query optimizations&lt;/li&gt;
&lt;li&gt;Improve scheduler web user interface&lt;/li&gt;
&lt;li&gt;Publish Docker images to GitHub Container Registry&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;The detailed list of issues planned for the 0.10.0 release can be found in the &lt;a href="https://github.com/apache/arrow-ballista/issues/361"&gt;tracking issue&lt;/a&gt;.&lt;/p&gt;
&lt;h2 id="getting-involved"&gt;Getting Involved&lt;a class="headerlink" href="#getting-involved" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;Ballista has a friendly community and we welcome contributions. A good place to start is to following the instructions
in the &lt;a href="https://arrow.apache.org/ballista/"&gt;user guide&lt;/a&gt; and try using Ballista with your own SQL queries and ETL pipelines, and file issues
for any bugs or feature suggestions.&lt;/p&gt;</content><category term="blog"/></entry><entry><title>Apache Arrow DataFusion 13.0.0 Project Update</title><link href="https://datafusion.apache.org/blog/2022/10/25/datafusion-13.0.0" rel="alternate"/><published>2022-10-25T00:00:00+00:00</published><updated>2022-10-25T00:00:00+00:00</updated><author><name>pmc</name></author><id>tag:datafusion.apache.org,2022-10-25:/blog/2022/10/25/datafusion-13.0.0</id><summary type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;h1 id="introduction"&gt;Introduction&lt;a class="headerlink" href="#introduction" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h1&gt;
&lt;p&gt;&lt;a href="https://arrow.apache.org/datafusion/"&gt;Apache Arrow DataFusion&lt;/a&gt; &lt;a href="https://crates.io/crates/datafusion"&gt;&lt;code&gt;13.0.0&lt;/code&gt;&lt;/a&gt; is released, and this blog contains an update on the project for the 5 months since our &lt;a href="https://arrow.apache.org/blog/2022/05/16/datafusion-8.0.0/"&gt;last update in May 2022&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;DataFusion is an extensible and embeddable query engine, written in Rust used to create modern, fast and efficient data pipelines, ETL …&lt;/p&gt;</summary><content type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;h1 id="introduction"&gt;Introduction&lt;a class="headerlink" href="#introduction" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h1&gt;
&lt;p&gt;&lt;a href="https://arrow.apache.org/datafusion/"&gt;Apache Arrow DataFusion&lt;/a&gt; &lt;a href="https://crates.io/crates/datafusion"&gt;&lt;code&gt;13.0.0&lt;/code&gt;&lt;/a&gt; is released, and this blog contains an update on the project for the 5 months since our &lt;a href="https://arrow.apache.org/blog/2022/05/16/datafusion-8.0.0/"&gt;last update in May 2022&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;DataFusion is an extensible and embeddable query engine, written in Rust used to create modern, fast and efficient data pipelines, ETL processes, and database systems. You may want to check out DataFusion to extend your Rust project to:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;Support &lt;a href="https://arrow.apache.org/datafusion/user-guide/sql/sql_status.html"&gt;SQL support&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;Support &lt;a href="https://docs.rs/datafusion/13.0.0/datafusion/dataframe/struct.DataFrame.html"&gt;DataFrame API&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;Support a Domain Specific Query Language&lt;/li&gt;
&lt;li&gt;Easily and quickly read and process Parquet, JSON, Avro or CSV data.&lt;/li&gt;
&lt;li&gt;Read from remote object stores such as AWS S3, Azure Blob Storage, GCP.&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;Even though DataFusion is 4 years "young," it has seen significant community growth in the last few months and the momentum continues to accelerate.&lt;/p&gt;
&lt;h1 id="background"&gt;Background&lt;a class="headerlink" href="#background" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h1&gt;
&lt;p&gt;DataFusion is used as the engine in &lt;a href="https://github.com/apache/arrow-datafusion#known-uses"&gt;many open source and commercial projects&lt;/a&gt; and was one of the early open source projects to provide this capability. 2022 has validated our belief in the need for such a &lt;a href="https://docs.google.com/presentation/d/1iNX_35sWUakee2q3zMFPyHE4IV2nC3lkCK_H6Y2qK84/edit#slide=id.p"&gt;"LLVM for database and AI systems"&lt;/a&gt;&lt;a href="https://www.slideshare.net/AndrewLamb32/20220623-apache-arrow-and-datafusion-changing-the-game-for-implementing-database-systemspdf"&gt;(alternate link)&lt;/a&gt; with announcements such as the &lt;a href="https://engineering.fb.com/2022/08/31/open-source/velox/"&gt;release of FaceBook's Velox&lt;/a&gt; engine, the major investments in &lt;a href="https://arrow.apache.org/docs/cpp/streaming_execution.html"&gt;Acero&lt;/a&gt; as well as the continued popularity of &lt;a href="https://calcite.apache.org/"&gt;Apache Calcite&lt;/a&gt; and other similar technologies.&lt;/p&gt;
&lt;p&gt;While Velox and Acero focus on execution engines, DataFusion provides the entire suite of components needed to build most analytic systems, including a SQL frontend, a dataframe API, and  extension points for just about everything. Some &lt;a href="https://github.com/apache/arrow-datafusion#known-uses"&gt;DataFusion users&lt;/a&gt; use a subset of the features such as the frontend (e.g. &lt;a href="https://dask-sql.readthedocs.io/en/latest/"&gt;dask-sql&lt;/a&gt;) or the execution engine, (e.g.  &lt;a href="https://github.com/blaze-init/blaze"&gt;Blaze&lt;/a&gt;), and some use many different components to build both SQL based and customized DSL based systems such as &lt;a href="https://github.com/influxdata/influxdb_iox/pulls"&gt;InfluxDB IOx&lt;/a&gt; and &lt;a href="https://github.com/vegafusion/vegafusion"&gt;VegaFusion&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;One of DataFusion’s advantages is its implementation in &lt;a href="https://www.rust-lang.org/"&gt;Rust&lt;/a&gt; and thus its easy integration with the broader Rust ecosystem. Rust continues to be a major source of benefit, from the &lt;a href="https://www.influxdata.com/blog/using-rustlangs-async-tokio-runtime-for-cpu-bound-tasks/"&gt;ease of parallelization with the high quality and standardized &lt;code&gt;async&lt;/code&gt; ecosystem&lt;/a&gt; , as well as its modern dependency management system and wonderful performance. &lt;!-- I wonder if we should link to clickbench?? --&gt;&lt;/p&gt;
&lt;!--While we haven’t invested in the benchmarking ratings game datafusion continues to be quite speedy (todo quantity this, with some evidence) – maybe clickbench?--&gt;
&lt;!--
Maybe we can do this un a future post
# DataFusion in Action

While DataFusion really shines as an embeddable query engine, if you want to try it out and get a feel for its power, you can use the basic[`datafusion-cli`](https://docs.rs/datafusion-cli/13.0.0/datafusion_cli/) tool to get a sense for what is possible to add in your application

(TODO example here of using datafusion-cli to query from local parquet files on disk)

TODO: also mention you can use the same thing to query data from S3
--&gt;
&lt;h1 id="summary"&gt;Summary&lt;a class="headerlink" href="#summary" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h1&gt;
&lt;p&gt;We have increased the frequency of DataFusion releases to monthly instead of quarterly. This
makes it easier for the increasing number of projects that now depend on DataFusion.&lt;/p&gt;
&lt;p&gt;We have also completed the "graduation" of &lt;a href="https://github.com/apache/arrow-ballista"&gt;Ballista to its own top-level arrow-ballista repository&lt;/a&gt;
which decouples the two projects and allows each project to move even faster.&lt;/p&gt;
&lt;p&gt;Along with numerous other bug fixes and smaller improvements, here are some of the major advances:&lt;/p&gt;
&lt;h1 id="improved-support-for-cloud-object-stores"&gt;Improved Support for Cloud Object Stores&lt;a class="headerlink" href="#improved-support-for-cloud-object-stores" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h1&gt;
&lt;p&gt;DataFusion now supports many major cloud object stores (Amazon S3, Azure Blob Storage, and Google Cloud Storage) "out of the box" via the &lt;a href="https://crates.io/crates/object_store"&gt;object_store&lt;/a&gt; crate. Using this integration, DataFusion optimizes reading parquet files by reading only the parts of the files that are needed.&lt;/p&gt;
&lt;h2 id="advanced-sql"&gt;Advanced SQL&lt;a class="headerlink" href="#advanced-sql" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;DataFusion now supports correlated subqueries, by rewriting them as joins. See the &lt;a href="https://arrow.apache.org/datafusion/user-guide/sql/subqueries.html"&gt;Subquery&lt;/a&gt; page in the User Guide for more information.&lt;/p&gt;
&lt;p&gt;In addition to numerous other small improvements, the following SQL features are now supported:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;code&gt;ROWS&lt;/code&gt;, &lt;code&gt;RANGE&lt;/code&gt;, &lt;code&gt;PRECEDING&lt;/code&gt; and &lt;code&gt;FOLLOWING&lt;/code&gt; in &lt;code&gt;OVER&lt;/code&gt; clauses &lt;a href="https://github.com/apache/arrow-datafusion/issues/3570"&gt;#3570&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;&lt;code&gt;ROLLUP&lt;/code&gt; and &lt;code&gt;CUBE&lt;/code&gt; grouping set expressions  &lt;a href="https://github.com/apache/arrow-datafusion/issues/2446"&gt;#2446&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;&lt;code&gt;SUM DISTINCT&lt;/code&gt; aggregate support  &lt;a href="https://github.com/apache/arrow-datafusion/issues/2405"&gt;#2405&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;&lt;code&gt;IN&lt;/code&gt; and &lt;code&gt;NOT IN&lt;/code&gt; Subqueries by rewriting them to &lt;code&gt;SEMI&lt;/code&gt; / &lt;code&gt;ANTI&lt;/code&gt; &lt;a href="https://github.com/apache/arrow-datafusion/issues/2885"&gt;#2421&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;Non equality predicates in  &lt;code&gt;ON&lt;/code&gt; clause of  &lt;code&gt;LEFT&lt;/code&gt;, &lt;code&gt;RIGHT,&lt;/code&gt;and &lt;code&gt;FULL&lt;/code&gt; joins &lt;a href="https://github.com/apache/arrow-datafusion/issues/2591"&gt;#2591&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;Exact &lt;code&gt;MEDIAN&lt;/code&gt; &lt;a href="https://github.com/apache/arrow-datafusion/issues/3009"&gt;#3009&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;&lt;code&gt;GROUPING SETS&lt;/code&gt;/&lt;code&gt;CUBE&lt;/code&gt;/&lt;code&gt;ROLLUP&lt;/code&gt; &lt;a href="https://github.com/apache/arrow-datafusion/issues/2716"&gt;#2716&lt;/a&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;h1 id="more-ddl-support"&gt;More DDL Support&lt;a class="headerlink" href="#more-ddl-support" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h1&gt;
&lt;p&gt;Just as it is important to query, it is also important to give users the ability to define their data sources. We have added:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;code&gt;CREATE VIEW&lt;/code&gt; &lt;a href="https://github.com/apache/arrow-datafusion/issues/2279"&gt;#2279&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;&lt;code&gt;DESCRIBE &amp;lt;table&amp;gt;&lt;/code&gt; &lt;a href="https://github.com/apache/arrow-datafusion/issues/2642"&gt;#2642&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;Custom / Dynamic table provider factories &lt;a href="https://github.com/apache/arrow-datafusion/issues/3311"&gt;#3311&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;&lt;code&gt;SHOW CREATE TABLE&lt;/code&gt; for support for views &lt;a href="https://github.com/apache/arrow-datafusion/issues/2830"&gt;#2830&lt;/a&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;h1 id="faster-execution"&gt;Faster Execution&lt;a class="headerlink" href="#faster-execution" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h1&gt;
&lt;p&gt;Performance is always an important goal for DataFusion, and there are a number of significant new optimizations such as&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;Optimizations of TopK (queries with a &lt;code&gt;LIMIT&lt;/code&gt; or &lt;code&gt;OFFSET&lt;/code&gt; clause):  &lt;a href="https://github.com/apache/arrow-datafusion/issues/3527"&gt;#3527&lt;/a&gt;, &lt;a href="https://github.com/apache/arrow-datafusion/issues/2521"&gt;#2521&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;Reduce &lt;code&gt;left&lt;/code&gt;/&lt;code&gt;right&lt;/code&gt;/&lt;code&gt;full&lt;/code&gt; joins to &lt;code&gt;inner&lt;/code&gt; join &lt;a href="https://github.com/apache/arrow-datafusion/issues/2750"&gt;#2750&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;Convert  cross joins to inner joins when possible &lt;a href="https://github.com/apache/arrow-datafusion/issues/3482"&gt;#3482&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;Sort preserving &lt;code&gt;SortMergeJoin&lt;/code&gt; &lt;a href="https://github.com/apache/arrow-datafusion/issues/2699"&gt;#2699&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;Improvements in group by and sort performance &lt;a href="https://github.com/apache/arrow-datafusion/issues/2375"&gt;#2375&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;Adaptive &lt;code&gt;regex_replace&lt;/code&gt; implementation &lt;a href="https://github.com/apache/arrow-datafusion/issues/3518"&gt;#3518&lt;/a&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;h1 id="optimizer-enhancements"&gt;Optimizer Enhancements&lt;a class="headerlink" href="#optimizer-enhancements" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h1&gt;
&lt;p&gt;Internally the optimizer has been significantly enhanced as well.&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;Casting / coercion now happens during logical planning &lt;a href="https://github.com/apache/arrow-datafusion/issues/3396"&gt;#3185&lt;/a&gt; &lt;a href="https://github.com/apache/arrow-datafusion/issues/3636"&gt;#3636&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;More sophisticated expression analysis and simplification is available&lt;/li&gt;
&lt;/ul&gt;
&lt;h1 id="parquet"&gt;Parquet&lt;a class="headerlink" href="#parquet" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h1&gt;
&lt;ul&gt;
&lt;li&gt;The parquet reader can now read directly from parquet files on remote object storage &lt;a href="https://github.com/apache/arrow-datafusion/issues/2677"&gt;#2489&lt;/a&gt; &lt;a href="https://github.com/apache/arrow-datafusion/issues/3051"&gt;#3051&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;Experimental support for “predicate pushdown” with late materialization after filtering during the scan (another blog post on this topic is coming soon).&lt;/li&gt;
&lt;li&gt;Support reading directly from AWS S3 and other object stores via &lt;code&gt;datafusion-cli&lt;/code&gt; &lt;a href="https://github.com/apache/arrow-datafusion/issues/3631"&gt;#3631&lt;/a&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;h1 id="datatype-support"&gt;DataType Support&lt;a class="headerlink" href="#datatype-support" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h1&gt;
&lt;ul&gt;
&lt;li&gt;Support for &lt;code&gt;TimestampTz&lt;/code&gt; &lt;a href="https://github.com/apache/arrow-datafusion/issues/3660"&gt;#3660&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;Expanded support for the &lt;code&gt;Decimal&lt;/code&gt; type, including  &lt;code&gt;IN&lt;/code&gt; list and better built in coercion.&lt;/li&gt;
&lt;li&gt;Expanded support for date/time manipulation such as  &lt;code&gt;date_bin&lt;/code&gt; built-in function , timestamp &lt;code&gt;+/-&lt;/code&gt; interval, &lt;code&gt;TIME&lt;/code&gt; literal values &lt;a href="https://github.com/apache/arrow-datafusion/issues/3010"&gt;#3010&lt;/a&gt;, &lt;a href="https://github.com/apache/arrow-datafusion/issues/3110"&gt;#3110&lt;/a&gt;, &lt;a href="https://github.com/apache/arrow-datafusion/issues/3034"&gt;#3034&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;Binary operations (&lt;code&gt;AND&lt;/code&gt;, &lt;code&gt;XOR&lt;/code&gt;, etc):  &lt;a href="https://github.com/apache/arrow-datafusion/issues/1619"&gt;#3037&lt;/a&gt; &lt;a href="https://github.com/apache/arrow-datafusion/issues/3430"&gt;#3420&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;&lt;code&gt;IS TRUE/FALSE&lt;/code&gt; and &lt;code&gt;IS [NOT] UNKNOWN&lt;/code&gt; &lt;a href="https://github.com/apache/arrow-datafusion/issues/3235"&gt;#3235&lt;/a&gt;, &lt;a href="https://github.com/apache/arrow-datafusion/issues/3246"&gt;#3246&lt;/a&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;h2 id="upcoming-work"&gt;Upcoming Work&lt;a class="headerlink" href="#upcoming-work" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;With the community growing and code accelerating, there is so much great stuff on the horizon. Some features we expect to land in the next few months:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;a href="https://github.com/apache/arrow-datafusion/issues/3462"&gt;Complete Parquet Pushdown&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;&lt;a href="https://github.com/apache/arrow-datafusion/issues/3148"&gt;Additional date/time support&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;Cost models, Nested Join Optimizations, analysis framework &lt;a href="https://github.com/apache/arrow-datafusion/issues/128"&gt;#128&lt;/a&gt;, &lt;a href="https://github.com/apache/arrow-datafusion/issues/3843"&gt;#3843&lt;/a&gt;, &lt;a href="https://github.com/apache/arrow-datafusion/issues/3845"&gt;#3845&lt;/a&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;h1 id="community-growth"&gt;Community Growth&lt;a class="headerlink" href="#community-growth" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h1&gt;
&lt;p&gt;The DataFusion 9.0.0 and 13.0.0 releases consists of 433 PRs from 64 distinct contributors. This does not count all the work that goes into our dependencies such as &lt;a href="https://crates.io/crates/arrow"&gt;arrow&lt;/a&gt;,  &lt;a href="https://crates.io/crates/parquet"&gt;parquet&lt;/a&gt;, and &lt;a href="https://crates.io/crates/object_store"&gt;object_store&lt;/a&gt;, that much of the same community helps nurture.&lt;/p&gt;
&lt;!--
$ git log --pretty=oneline 9.0.0..13.0.0 . | wc -l
433

$ git shortlog -sn 9.0.0..13.0.0 . | wc -l
65
--&gt;
&lt;h1 id="how-to-get-involved"&gt;How to Get Involved&lt;a class="headerlink" href="#how-to-get-involved" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h1&gt;
&lt;p&gt;Kudos to everyone in the community who contributed ideas, discussions, bug reports, documentation and code. It is exciting to be building something so cool together!&lt;/p&gt;
&lt;p&gt;If you are interested in contributing to DataFusion, we would love to
have you join us on our journey to create the most advanced open
source query engine. You can try out DataFusion on some of your own
data and projects and let us know how it goes or contribute a PR with
documentation, tests or code. A list of open issues suitable for
beginners is
&lt;a href="https://github.com/apache/arrow-datafusion/issues?q=is%3Aissue+is%3Aopen+label%3A%22good+first+issue%22"&gt;here&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;Check out our &lt;a href="https://arrow.apache.org/datafusion/community/communication.html"&gt;Communication Doc&lt;/a&gt; on more
ways to engage with the community.&lt;/p&gt;
&lt;h2 id="appendix-contributor-shoutout"&gt;Appendix: Contributor Shoutout&lt;a class="headerlink" href="#appendix-contributor-shoutout" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;To give a sense of the number of people who contribute to this project regularly, we present for your consideration the following list derived from &lt;code&gt;git shortlog -sn 9.0.0..13.0.0 .&lt;/code&gt; Thank you all again!&lt;/p&gt;
&lt;!-- Note: combined kmitchener and Kirk Mitchener --&gt;
&lt;pre&gt;&lt;code&gt;    87  Andy Grove
    71  Andrew Lamb
    29  Kun Liu
    29  Kirk Mitchener
    17  Wei-Ting Kuo
    14  Yang Jiang
    12  Raphael Taylor-Davies
    11  Batuhan Taskaya
    10  Brent Gardner
    10  Remzi Yang
    10  comphead
    10  xudong.w
     8  AssHero
     7  Ruihang Xia
     6  Dan Harris
     6  Daniël Heres
     6  Ian Alexander Joiner
     6  Mike Roberts
     6  askoa
     4  BaymaxHWY
     4  gorkem
     4  jakevin
     3  George Andronchik
     3  Sarah Yurick
     3  Stuart Carnie
     2  Dalton Modlin
     2  Dmitry Patsura
     2  JasonLi
     2  Jon Mease
     2  Marco Neumann
     2  yahoNanJing
     1  Adilet Sarsembayev
     1  Ayush Dattagupta
     1  Dezhi Wu
     1  Dhamotharan Sritharan
     1  Eduard Karacharov
     1  Francis Du
     1  Harbour Zheng
     1  Ismaël Mejía
     1  Jack Klamer
     1  Jeremy Dyer
     1  Jiayu Liu
     1  Kamil Konior
     1  Liang-Chi Hsieh
     1  Martin Grigorov
     1  Matthijs Brobbel
     1  Mehmet Ozan Kabak
     1  Metehan Yıldırım
     1  Morgan Cassels
     1  Nitish Tiwari
     1  Renjie Liu
     1  Rito Takeuchi
     1  Robert Pack
     1  Thomas Cameron
     1  Vrishabh
     1  Xin Hao
     1  Yijie Shen
     1  byteink
     1  kamille
     1  mateuszkj
     1  nvartolomei
     1  yourenawo
     1  Özgür Akkurt
&lt;/code&gt;&lt;/pre&gt;</content><category term="blog"/></entry><entry><title>Apache Arrow DataFusion 8.0.0 Release</title><link href="https://datafusion.apache.org/blog/2022/05/16/datafusion-8.0.0" rel="alternate"/><published>2022-05-16T00:00:00+00:00</published><updated>2022-05-16T00:00:00+00:00</updated><author><name>pmc</name></author><id>tag:datafusion.apache.org,2022-05-16:/blog/2022/05/16/datafusion-8.0.0</id><summary type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;h1 id="introduction"&gt;Introduction&lt;a class="headerlink" href="#introduction" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h1&gt;
&lt;p&gt;&lt;a href="https://arrow.apache.org/datafusion/"&gt;DataFusion&lt;/a&gt; is an extensible query execution framework, written in Rust, that
uses Apache Arrow as its in-memory format.&lt;/p&gt;
&lt;p&gt;When you want to extend your Rust project with &lt;a href="https://arrow.apache.org/datafusion/user-guide/sql/sql_status.html"&gt;SQL support&lt;/a&gt;,
a DataFrame API, or the ability to read and process Parquet, JSON, Avro or CSV data, DataFusion is definitely worth …&lt;/p&gt;</summary><content type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;h1 id="introduction"&gt;Introduction&lt;a class="headerlink" href="#introduction" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h1&gt;
&lt;p&gt;&lt;a href="https://arrow.apache.org/datafusion/"&gt;DataFusion&lt;/a&gt; is an extensible query execution framework, written in Rust, that
uses Apache Arrow as its in-memory format.&lt;/p&gt;
&lt;p&gt;When you want to extend your Rust project with &lt;a href="https://arrow.apache.org/datafusion/user-guide/sql/sql_status.html"&gt;SQL support&lt;/a&gt;,
a DataFrame API, or the ability to read and process Parquet, JSON, Avro or CSV data, DataFusion is definitely worth
checking out.&lt;/p&gt;
&lt;p&gt;DataFusion's SQL, &lt;code&gt;DataFrame&lt;/code&gt;, and manual &lt;code&gt;PlanBuilder&lt;/code&gt; API let users access a sophisticated query optimizer and
execution engine capable of fast, resource efficient, and parallel execution that takes optimal advantage of
today's multicore hardware. Being written in Rust means DataFusion can offer &lt;em&gt;both&lt;/em&gt; the safety of a dynamic language and
the resource efficiency of a compiled language.&lt;/p&gt;
&lt;p&gt;The Apache Arrow team is pleased to announce the DataFusion 8.0.0 release (and also the release of version 0.7.0 of
the Ballista subproject). This covers 3 months of development work and includes 279 commits from the following 49
distinct contributors.&lt;/p&gt;
&lt;!--
$ git log --pretty=oneline 7.0.0..8.0.0 datafusion datafusion-cli datafusion-examples ballista ballista-cli ballista-examples | wc -l
279

$ git shortlog -sn 7.0.0..8.0.0 datafusion datafusion-cli datafusion-examples ballista ballista-cli ballista-examples | wc -l
49

(feynman han, feynman.h, Feynman Han were assumed to be the same person)
--&gt;
&lt;pre&gt;&lt;code&gt;    39  Andy Grove
    33  Andrew Lamb
    21  DuRipeng
    20  Yijie Shen
    19  Yang Jiang
    17  Raphael Taylor-Davies
    11  Dan Harris
    11  Matthew Turner
    11  yahoNanJing
     9  dependabot[bot]
     8  jakevin
     6  Kun Liu
     5  Jiayu Liu
     4  Daniël Heres
     4  mingmwang
     4  xudong.w
     3  Carol (Nichols || Goulding)
     3  Dmitry Patsura
     3  Eduard Karacharov
     3  Jeremy Dyer
     3  Kaushik
     3  Rich
     3  comphead
     3  gaojun2048
     3  Feynman Han
     2  Jie Han
     2  Jon Mease
     2  Tim Van Wassenhove
     2  Yt
     2  Zhang Li
     2  silence-coding
     1  Alexander Spies
     1  George Andronchik
     1  Guillaume Balaine
     1  Hao Xin
     1  Jiacai Liu
     1  Jörn Horstmann
     1  Liang-Chi Hsieh
     1  Max Burke
     1  NaincyKumariKnoldus
     1  Nga Tran
     1  Patrick More
     1  Pierre Zemb
     1  Remzi Yang
     1  Sergey Melnychuk
     1  Stephen Carman
     1  doki
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;The following sections highlight some of the changes in this release. Of course, many other bug fixes and
improvements have been made and we encourage you to check out the
&lt;a href="https://github.com/apache/arrow-datafusion/blob/8.0.0/datafusion/CHANGELOG.md"&gt;changelog&lt;/a&gt; for full details.&lt;/p&gt;
&lt;h1 id="summary"&gt;Summary&lt;a class="headerlink" href="#summary" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h1&gt;
&lt;h2 id="ddl-support"&gt;DDL Support&lt;a class="headerlink" href="#ddl-support" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;DDL support has been expanded to include the following commands for creating databases, schemas, and views. This
allows DataFusion to be used more effectively from the CLI.&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;code&gt;CREATE DATABASE&lt;/code&gt;&lt;/li&gt;
&lt;li&gt;&lt;code&gt;CREATE VIEW&lt;/code&gt;&lt;/li&gt;
&lt;li&gt;&lt;code&gt;CREATE SCHEMA&lt;/code&gt;&lt;/li&gt;
&lt;li&gt;&lt;code&gt;CREATE EXTERNAL TABLE&lt;/code&gt; now supports JSON files, &lt;code&gt;IF NOT EXISTS&lt;/code&gt;, and partition columns&lt;/li&gt;
&lt;/ul&gt;
&lt;h2 id="sql-support"&gt;SQL Support&lt;a class="headerlink" href="#sql-support" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;The SQL query planner now supports a number of new SQL features, including:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;em&gt;Subqueries&lt;/em&gt;: when used via &lt;code&gt;IN&lt;/code&gt;, &lt;code&gt;EXISTS&lt;/code&gt;, and as scalars&lt;/li&gt;
&lt;li&gt;&lt;em&gt;Grouping Sets&lt;/em&gt;: &lt;code&gt;CUBE&lt;/code&gt; and &lt;code&gt;ROLLUP&lt;/code&gt; grouping sets.&lt;/li&gt;
&lt;li&gt;&lt;em&gt;Aggregate functions&lt;/em&gt;: &lt;code&gt;approx_percentile&lt;/code&gt;, &lt;code&gt;approx_percentile_cont&lt;/code&gt;, &lt;code&gt;approx_percentile_cont_with_weight&lt;/code&gt;, &lt;code&gt;approx_distinct&lt;/code&gt;, &lt;code&gt;approx_median&lt;/code&gt; and &lt;code&gt;array&lt;/code&gt;&lt;/li&gt;
&lt;li&gt;&lt;em&gt;&lt;code&gt;null&lt;/code&gt; literals&lt;/em&gt;&lt;/li&gt;
&lt;li&gt;&lt;em&gt;bitwise operations&lt;/em&gt;: for example '&lt;code&gt;|&lt;/code&gt;'&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;There are also many bug fixes and improvements around normalizing identifiers consistently.&lt;/p&gt;
&lt;p&gt;We continue our tradition of incrementally releasing support for new
features as they are developed. Thus, while the physical plan may not yet
support all new features, it gets more complete each release. These
changes also make DataFusion an increasingly compelling choice for
projects looking for a SQL parser and query planner that can produce
optimized logical plans that can be translated to
their own execution engine.&lt;/p&gt;
&lt;h2 id="query-execution-internals"&gt;Query Execution &amp;amp; Internals&lt;a class="headerlink" href="#query-execution-internals" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;There are several notable improvements and new features in the query execution engine:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;The &lt;code&gt;ExecutionContext&lt;/code&gt; has been renamed to &lt;code&gt;SessionContext&lt;/code&gt; and now supports multi-tenancy&lt;/li&gt;
&lt;li&gt;The &lt;code&gt;ExecutionPlan&lt;/code&gt; trait is no longer &lt;code&gt;async&lt;/code&gt;&lt;/li&gt;
&lt;li&gt;A new serialization API for serializing plans to bytes (based on protobuf)&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;In addition, we have added several foundational features to drive even
more advanced query processing into DataFusion, focusing on running
arbitrary queries larger than available memory, and pushing the
envelope for performance of sorting, grouping, and joining even
further:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;Morsel-Driven Scheduler based on &lt;a href="https://15721.courses.cs.cmu.edu/spring2016/papers/p743-leis.pdf"&gt;"Morsel-Driven Parallelism: A NUMA-Aware Query
  Evaluation Framework for the Many-Core Age"&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;Consolidated object store implementation and integration with parquet decoding&lt;/li&gt;
&lt;li&gt;Memory Limited Spilling sort operator&lt;/li&gt;
&lt;li&gt;Memory Limited Sort-Merge join operator&lt;/li&gt;
&lt;li&gt;High performance JIT code generation for tuple comparisons&lt;/li&gt;
&lt;li&gt;Memory efficient Row Format&lt;/li&gt;
&lt;/ul&gt;
&lt;h2 id="improved-file-support"&gt;Improved file support&lt;a class="headerlink" href="#improved-file-support" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;DataFusion now supports JSON, both for reading and writing. There are also new DataFrame methods for writing query
results to files in CSV, Parquet, and JSON format.&lt;/p&gt;
&lt;h2 id="ballista"&gt;Ballista&lt;a class="headerlink" href="#ballista" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;Ballista continues to mature and now supports a wider range of operators and expressions. There are also improvements
to the scheduler to support UDFs, and there are some robustness improvements, such as cleaning up work directories
and persisting session configs to allow schedulers to restart and continue processing in-flight jobs.&lt;/p&gt;
&lt;h2 id="upcoming-work"&gt;Upcoming Work&lt;a class="headerlink" href="#upcoming-work" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;Here are some of the initiatives that the community plans on working on prior to the next release.&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;There is a &lt;a href="https://docs.google.com/document/d/1jNRbadyStSrV5kifwn0khufAwq6OnzGczG4z8oTQJP4/edit?usp=sharing"&gt;proposal to move Ballista to its own top-level arrow-ballista repository&lt;/a&gt;
 to decouple DataFusion and Ballista releases and to allow each project to have documentation better targeted at
  its particular audience.&lt;/li&gt;
&lt;li&gt;We plan on increasing the frequency of DataFusion releases, with monthly releases now instead of quarterly. This
  is driven by requests from the increasing number of projects that now depend on DataFusion.&lt;/li&gt;
&lt;li&gt;There is ongoing work to implement new optimizer rules to rewrite queries containing subquery expressions as
  joins, to support a wider range of queries.&lt;/li&gt;
&lt;li&gt;The new scheduler based on morsel-driven execution will continue to evolve in this next release, with work to
  refine IO abstractions to improve performance and integration with the new scheduler.&lt;/li&gt;
&lt;li&gt;Improved performance for Sort, Grouping and Joins&lt;/li&gt;
&lt;/ul&gt;
&lt;h1 id="how-to-get-involved"&gt;How to Get Involved&lt;a class="headerlink" href="#how-to-get-involved" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h1&gt;
&lt;p&gt;If you are interested in contributing to DataFusion, and learning about state-of-the-art query processing, we would
love to have you join us on the journey! You can help by trying out DataFusion on some of your own data and projects
and let us know how it goes or contribute a PR with documentation, tests or code. A list of open issues suitable
for beginners is &lt;a href="https://github.com/apache/arrow-datafusion/issues?q=is%3Aissue+is%3Aopen+label%3A%22good+first+issue%22"&gt;here&lt;/a&gt;&lt;/p&gt;
&lt;p&gt;Check out our new &lt;a href="https://arrow.apache.org/datafusion/community/communication.html"&gt;Communication Doc&lt;/a&gt; on more
ways to engage with the community.&lt;/p&gt;</content><category term="blog"/></entry><entry><title>Introducing Apache Arrow DataFusion Contrib</title><link href="https://datafusion.apache.org/blog/2022/03/21/datafusion-contrib" rel="alternate"/><published>2022-03-21T00:00:00+00:00</published><updated>2022-03-21T00:00:00+00:00</updated><author><name>pmc</name></author><id>tag:datafusion.apache.org,2022-03-21:/blog/2022/03/21/datafusion-contrib</id><summary type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;
&lt;h1 id="introduction"&gt;Introduction&lt;a class="headerlink" href="#introduction" title="Permanent link"&gt;&amp;para;&lt;/a&gt;&lt;/h1&gt;
&lt;p&gt;Apache Arrow &lt;a href="https://arrow.apache.org/datafusion/"&gt;DataFusion&lt;/a&gt; is an extensible query execution framework, written in Rust, that uses &lt;a href="https://arrow.apache.org"&gt;Apache Arrow&lt;/a&gt; as its in-memory format.&lt;/p&gt;
&lt;p&gt;When you want to extend your Rust project with &lt;a href="https://arrow.apache.org/datafusion/user-guide/sql/sql_status.html"&gt;SQL support&lt;/a&gt;, a DataFrame API, or the ability to read and process Parquet, JSON, Avro or CSV data, DataFusion is …&lt;/p&gt;</summary><content type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;
&lt;h1 id="introduction"&gt;Introduction&lt;a class="headerlink" href="#introduction" title="Permanent link"&gt;&amp;para;&lt;/a&gt;&lt;/h1&gt;
&lt;p&gt;Apache Arrow &lt;a href="https://arrow.apache.org/datafusion/"&gt;DataFusion&lt;/a&gt; is an extensible query execution framework, written in Rust, that uses &lt;a href="https://arrow.apache.org"&gt;Apache Arrow&lt;/a&gt; as its in-memory format.&lt;/p&gt;
&lt;p&gt;When you want to extend your Rust project with &lt;a href="https://arrow.apache.org/datafusion/user-guide/sql/sql_status.html"&gt;SQL support&lt;/a&gt;, a DataFrame API, or the ability to read and process Parquet, JSON, Avro or CSV data, DataFusion is definitely worth checking out. DataFusion's pluggable design makes creating extensions at various points particular easy to build.&lt;/p&gt;
&lt;p&gt;DataFusion's  SQL, &lt;code&gt;DataFrame&lt;/code&gt;, and manual &lt;code&gt;PlanBuilder&lt;/code&gt; API let users access a sophisticated query optimizer and execution engine capable of fast, resource efficient, and parallel execution that takes optimal advantage of todays multicore hardware. Being written in Rust means DataFusion can offer &lt;em&gt;both&lt;/em&gt; the safety of dynamic languages as well as the resource efficiency of a compiled language.&lt;/p&gt;
&lt;p&gt;The DataFusion team is pleased to announce the creation of the &lt;a href="https://github.com/datafusion-contrib"&gt;DataFusion-Contrib&lt;/a&gt; GitHub organization to support and accelerate other projects.  While the core DataFusion library remains under Apache governance, the contrib organization provides a more flexible testing ground for new DataFusion features and a home for DataFusion extensions.  With this announcement, we are pleased to introduce the following inaugural DataFusion-Contrib repositories.&lt;/p&gt;
&lt;h2 id="datafusion-python"&gt;DataFusion-Python&lt;a class="headerlink" href="#datafusion-python" title="Permanent link"&gt;&amp;para;&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;This &lt;a href="https://github.com/datafusion-contrib/datafusion-python"&gt;project&lt;/a&gt; provides Python bindings to the core Rust implementation of DataFusion, which allows users to:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;Work with familiar SQL or DataFrame APIs to run queries in a safe, multi-threaded environment, returning results in Python&lt;/li&gt;
&lt;li&gt;Create User Defined Functions and User Defined Aggregate Functions for complex operations&lt;/li&gt;
&lt;li&gt;Pay no overhead to copy between Python and underlying Rust execution engine (by way of Apache Arrow arrays)&lt;/li&gt;
&lt;/ul&gt;
&lt;h3 id="upcoming-enhancements"&gt;Upcoming enhancements&lt;a class="headerlink" href="#upcoming-enhancements" title="Permanent link"&gt;&amp;para;&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;The team is focusing on exposing more features from the underlying Rust implementation of DataFusion and improving documentation.&lt;/p&gt;
&lt;h3 id="how-to-install"&gt;How to install&lt;a class="headerlink" href="#how-to-install" title="Permanent link"&gt;&amp;para;&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;From &lt;code&gt;pip&lt;/code&gt;&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-bash"&gt;pip install datafusion
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;Or&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-bash"&gt;python -m pip install datafusion
&lt;/code&gt;&lt;/pre&gt;
&lt;h2 id="datafusion-objectstore-s3"&gt;DataFusion-ObjectStore-S3&lt;a class="headerlink" href="#datafusion-objectstore-s3" title="Permanent link"&gt;&amp;para;&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;This &lt;a href="https://github.com/datafusion-contrib/datafusion-objectstore-s3"&gt;crate&lt;/a&gt; provides an &lt;code&gt;ObjectStore&lt;/code&gt; implementation for querying data stored in S3 or S3 compatible storage. This makes it almost as easy to query data that lives on S3 as lives in local files&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;Ability to create &lt;code&gt;S3FileSystem&lt;/code&gt; to register as part of DataFusion &lt;code&gt;ExecutionContext&lt;/code&gt;&lt;/li&gt;
&lt;li&gt;Register files or directories stored on S3 with &lt;code&gt;ctx.register_listing_table&lt;/code&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;h3 id="upcoming-enhancements_1"&gt;Upcoming enhancements&lt;a class="headerlink" href="#upcoming-enhancements_1" title="Permanent link"&gt;&amp;para;&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;The current priority is adding python bindings for &lt;code&gt;S3FileSystem&lt;/code&gt;.  After that there will be async improvements as DataFusion adopts more of that functionality and we are looking into S3 Select functionality.&lt;/p&gt;
&lt;h3 id="how-to-install_1"&gt;How to Install&lt;a class="headerlink" href="#how-to-install_1" title="Permanent link"&gt;&amp;para;&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;Add the below to your &lt;code&gt;Cargo.toml&lt;/code&gt; in your Rust Project with DataFusion.&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-toml"&gt;datafusion-objectstore-s3 = "0.1.0"
&lt;/code&gt;&lt;/pre&gt;
&lt;h2 id="datafusion-substrait"&gt;DataFusion-Substrait&lt;a class="headerlink" href="#datafusion-substrait" title="Permanent link"&gt;&amp;para;&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;&lt;a href="https://substrait.io/"&gt;Substrait&lt;/a&gt; is an emerging standard that provides a cross-language serialization format for relational algebra (e.g. expressions and query plans).&lt;/p&gt;
&lt;p&gt;This &lt;a href="https://github.com/datafusion-contrib/datafusion-substrait"&gt;crate&lt;/a&gt; provides a Substrait producer and consumer for DataFusion.  A producer converts a DataFusion logical plan into a Substrait protobuf and a consumer does the reverse.&lt;/p&gt;
&lt;p&gt;Examples of how to use this crate can be found &lt;a href="https://github.com/datafusion-contrib/datafusion-substrait/blob/main/src/lib.rs"&gt;here&lt;/a&gt;.&lt;/p&gt;
&lt;h3 id="potential-use-cases"&gt;Potential Use Cases&lt;a class="headerlink" href="#potential-use-cases" title="Permanent link"&gt;&amp;para;&lt;/a&gt;&lt;/h3&gt;
&lt;ul&gt;
&lt;li&gt;Replace custom DataFusion protobuf serialization.&lt;/li&gt;
&lt;li&gt;Make it easier to pass query plans over FFI boundaries, such as from Python to Rust&lt;/li&gt;
&lt;li&gt;Allow Apache Calcite query plans to be executed in DataFusion&lt;/li&gt;
&lt;/ul&gt;
&lt;h2 id="datafusion-bigtable"&gt;DataFusion-BigTable&lt;a class="headerlink" href="#datafusion-bigtable" title="Permanent link"&gt;&amp;para;&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;This &lt;a href="https://github.com/datafusion-contrib/datafusion-bigtable"&gt;crate&lt;/a&gt; implements &lt;a href="https://cloud.google.com/bigtable"&gt;Bigtable&lt;/a&gt; as a data source and physical executor for DataFusion queries.  It currently supports both UTF-8 string and 64-bit big-endian signed integers in Bigtable.  From a SQL perspective it supports both simple and composite row keys with &lt;code&gt;=&lt;/code&gt;, &lt;code&gt;IN&lt;/code&gt;, and &lt;code&gt;BETWEEN&lt;/code&gt; operators as well as projection pushdown.  The physical execution for queries is handled by this crate while any subsequent aggregation, group bys, or joins are handled in DataFusion.&lt;/p&gt;
&lt;h3 id="upcoming-enhancements_2"&gt;Upcoming Enhancements&lt;a class="headerlink" href="#upcoming-enhancements_2" title="Permanent link"&gt;&amp;para;&lt;/a&gt;&lt;/h3&gt;
&lt;ul&gt;
&lt;li&gt;Predicate pushdown&lt;/li&gt;
&lt;li&gt;Value range&lt;/li&gt;
&lt;li&gt;Value Regex&lt;/li&gt;
&lt;li&gt;Timestamp range&lt;/li&gt;
&lt;li&gt;Multithreaded&lt;/li&gt;
&lt;li&gt;Partition aware execution&lt;/li&gt;
&lt;li&gt;Production ready&lt;/li&gt;
&lt;/ul&gt;
&lt;h3 id="how-to-install_2"&gt;How to Install&lt;a class="headerlink" href="#how-to-install_2" title="Permanent link"&gt;&amp;para;&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;Add the below to your &lt;code&gt;Cargo.toml&lt;/code&gt; in your Rust Project with DataFusion.&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-toml"&gt;datafusion-bigtable = "0.1.0"
&lt;/code&gt;&lt;/pre&gt;
&lt;h2 id="datafusion-hdfs"&gt;DataFusion-HDFS&lt;a class="headerlink" href="#datafusion-hdfs" title="Permanent link"&gt;&amp;para;&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;This &lt;a href="https://github.com/datafusion-contrib/datafusion-objectstore-hdfs"&gt;crate&lt;/a&gt; introduces &lt;code&gt;HadoopFileSystem&lt;/code&gt; as a remote &lt;code&gt;ObjectStore&lt;/code&gt; which provides the ability to query HDFS files.  For HDFS access the &lt;a href="https://github.com/yahoNanJing/fs-hdfs"&gt;fs-hdfs&lt;/a&gt; library is used.&lt;/p&gt;
&lt;h2 id="datafusion-tokomak"&gt;DataFusion-Tokomak&lt;a class="headerlink" href="#datafusion-tokomak" title="Permanent link"&gt;&amp;para;&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;This &lt;a href="https://github.com/datafusion-contrib/datafusion-tokomak"&gt;crate&lt;/a&gt; provides an e-graph based DataFusion optimization framework based on the Rust &lt;a href="https://egraphs-good.github.io"&gt;egg&lt;/a&gt; library.  An e-graph is a data structure that powers the equality saturation optimization technique.&lt;/p&gt;
&lt;p&gt;As context, the optimizer framework within DataFusion is currently &lt;a href="https://github.com/apache/arrow-datafusion/issues/1972"&gt;under review&lt;/a&gt; with the objective of implementing a more strategic long term solution that is more efficient and simpler to develop.&lt;/p&gt;
&lt;p&gt;Some of the benefits of using &lt;code&gt;egg&lt;/code&gt; within DataFusion are:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;Implements optimized algorithms that are hard to match with manually written optimization passes&lt;/li&gt;
&lt;li&gt;Makes it easy and less verbose to add optimization rules&lt;/li&gt;
&lt;li&gt;Plugin framework to add more complex optimizations&lt;/li&gt;
&lt;li&gt;Egg does not depend on rule order and can lead to a higher level of optimization by being able to apply multiple rules at the same time until it converges&lt;/li&gt;
&lt;li&gt;Allows for cost-based optimizations&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;This is an exciting new area for DataFusion with lots of opportunity for community involvement!&lt;/p&gt;
&lt;h2 id="datafusion-tui"&gt;DataFusion-Tui&lt;a class="headerlink" href="#datafusion-tui" title="Permanent link"&gt;&amp;para;&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;&lt;a href="https://github.com/datafusion-contrib/datafusion-tui"&gt;DataFusion-tui&lt;/a&gt; aka &lt;code&gt;dft&lt;/code&gt; provides a feature rich terminal application for using DataFusion.  It has drawn inspiration and several features from &lt;code&gt;datafusion-cli&lt;/code&gt;.  In contrast to &lt;code&gt;datafusion-cli&lt;/code&gt; the objective of this tool is to provide a light SQL IDE experience for querying data with DataFusion.  This includes features such as the following which are currently implemented:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;Tab Management to provide clean and structured organization of DataFusion queries, results, &lt;code&gt;ExecutionContext&lt;/code&gt; information, and logs&lt;/li&gt;
&lt;li&gt;SQL Editor&lt;ul&gt;
&lt;li&gt;Text editor for writing SQL queries&lt;/li&gt;
&lt;/ul&gt;
&lt;/li&gt;
&lt;li&gt;Query History&lt;ul&gt;
&lt;li&gt;History of executed queries, their execution time, and the number of returned rows&lt;/li&gt;
&lt;/ul&gt;
&lt;/li&gt;
&lt;li&gt;&lt;code&gt;ExecutionContext&lt;/code&gt; information&lt;ul&gt;
&lt;li&gt;Expose information on which physical optimizers are used and which &lt;code&gt;ExecutionConfig&lt;/code&gt; settings are set&lt;/li&gt;
&lt;/ul&gt;
&lt;/li&gt;
&lt;li&gt;Logs&lt;ul&gt;
&lt;li&gt;Logs from &lt;code&gt;dft&lt;/code&gt;, DataFusion, and any dependent libraries&lt;/li&gt;
&lt;/ul&gt;
&lt;/li&gt;
&lt;li&gt;Support for custom &lt;code&gt;ObjectStore&lt;/code&gt;s&lt;/li&gt;
&lt;li&gt;S3&lt;/li&gt;
&lt;li&gt;Preload DDL from &lt;code&gt;~/.datafusionrc&lt;/code&gt; to enable having local "database" available at startup&lt;/li&gt;
&lt;/ul&gt;
&lt;h3 id="upcoming-enhancements_3"&gt;Upcoming Enhancements&lt;a class="headerlink" href="#upcoming-enhancements_3" title="Permanent link"&gt;&amp;para;&lt;/a&gt;&lt;/h3&gt;
&lt;ul&gt;
&lt;li&gt;SQL Editor&lt;/li&gt;
&lt;li&gt;Command to write query results to file&lt;/li&gt;
&lt;li&gt;Multiple SQL editor tabs&lt;/li&gt;
&lt;li&gt;Expose more information from &lt;code&gt;ExecutionContext&lt;/code&gt;&lt;/li&gt;
&lt;li&gt;A help tab that provides information on functions&lt;/li&gt;
&lt;li&gt;Query custom &lt;code&gt;TableProvider&lt;/code&gt;s such as &lt;a href="https://github.com/delta-io/delta-rs"&gt;DeltaTable&lt;/a&gt; or &lt;a href="https://github.com/datafusion-contrib/datafusion-bigtable"&gt;BigTable&lt;/a&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;h2 id="datafusion-streams"&gt;DataFusion-Streams&lt;a class="headerlink" href="#datafusion-streams" title="Permanent link"&gt;&amp;para;&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;&lt;a href="https://github.com/datafusion-contrib/datafusion-streams"&gt;DataFusion-Stream&lt;/a&gt; is a new testing ground for creating a &lt;code&gt;StreamProvider&lt;/code&gt; in DataFusion that will enable querying streaming data sources such as Apache Kafka.  The implementation for this feature is currently being designed and is under active review.  Once the design is finalized the trait and attendant data structures will be added back to the core DataFusion crate.&lt;/p&gt;
&lt;h2 id="datafusion-java"&gt;DataFusion-Java&lt;a class="headerlink" href="#datafusion-java" title="Permanent link"&gt;&amp;para;&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;This &lt;a href="https://github.com/datafusion-contrib/datafusion-java"&gt;project&lt;/a&gt; created an initial set of Java bindings to DataFusion.  The project is currently in maintenance mode and is looking for maintainers to drive future development.&lt;/p&gt;
&lt;h1 id="how-to-get-involved"&gt;How to Get Involved&lt;a class="headerlink" href="#how-to-get-involved" title="Permanent link"&gt;&amp;para;&lt;/a&gt;&lt;/h1&gt;
&lt;p&gt;If you are interested in contributing to DataFusion, and learning about state of
the art query processing, we would love to have you join us on the journey! You
can help by trying out DataFusion on some of your own data and projects and let us know how it goes or contribute a PR with documentation, tests or code. A list of open issues suitable for beginners is &lt;a href="https://github.com/apache/arrow-datafusion/issues?q=is%3Aissue+is%3Aopen+label%3A%22good+first+issue%22"&gt;here&lt;/a&gt;&lt;/p&gt;
&lt;p&gt;The best way to find out about creating new extensions within DataFusion-Contrib is reaching out on the &lt;code&gt;#arrow-rust&lt;/code&gt; channel of the Apache Software Foundation &lt;a href="https://join.slack.com/t/the-asf/shared_invite/zt-vlfbf7ch-HkbNHiU_uDlcH_RvaHv9gQ"&gt;Slack&lt;/a&gt; workspace.&lt;/p&gt;
&lt;p&gt;You can also check out our new &lt;a href="https://arrow.apache.org/datafusion/community/communication.html"&gt;Communication Doc&lt;/a&gt; on more ways to engage with the community.&lt;/p&gt;
&lt;p&gt;Links for each DataFusion-Contrib repository are provided above if you would like to contribute to those.&lt;/p&gt;</content><category term="blog"/></entry><entry><title>Apache Arrow DataFusion 7.0.0 Release</title><link href="https://datafusion.apache.org/blog/2022/02/28/datafusion-7.0.0" rel="alternate"/><published>2022-02-28T00:00:00+00:00</published><updated>2022-02-28T00:00:00+00:00</updated><author><name>pmc</name></author><id>tag:datafusion.apache.org,2022-02-28:/blog/2022/02/28/datafusion-7.0.0</id><summary type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;
&lt;h1 id="introduction"&gt;Introduction&lt;a class="headerlink" href="#introduction" title="Permanent link"&gt;&amp;para;&lt;/a&gt;&lt;/h1&gt;
&lt;p&gt;&lt;a href="https://arrow.apache.org/datafusion/"&gt;DataFusion&lt;/a&gt; is an extensible query execution framework, written in Rust, that uses Apache Arrow as its in-memory format.&lt;/p&gt;
&lt;p&gt;When you want to extend your Rust project with &lt;a href="https://arrow.apache.org/datafusion/user-guide/sql/sql_status.html"&gt;SQL support&lt;/a&gt;, a DataFrame API, or the ability to read and process Parquet, JSON, Avro or CSV data, DataFusion is definitely worth …&lt;/p&gt;</summary><content type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;
&lt;h1 id="introduction"&gt;Introduction&lt;a class="headerlink" href="#introduction" title="Permanent link"&gt;&amp;para;&lt;/a&gt;&lt;/h1&gt;
&lt;p&gt;&lt;a href="https://arrow.apache.org/datafusion/"&gt;DataFusion&lt;/a&gt; is an extensible query execution framework, written in Rust, that uses Apache Arrow as its in-memory format.&lt;/p&gt;
&lt;p&gt;When you want to extend your Rust project with &lt;a href="https://arrow.apache.org/datafusion/user-guide/sql/sql_status.html"&gt;SQL support&lt;/a&gt;, a DataFrame API, or the ability to read and process Parquet, JSON, Avro or CSV data, DataFusion is definitely worth checking out.&lt;/p&gt;
&lt;p&gt;DataFusion's  SQL, &lt;code&gt;DataFrame&lt;/code&gt;, and manual &lt;code&gt;PlanBuilder&lt;/code&gt; API let users access a sophisticated query optimizer and execution engine capable of fast, resource efficient, and parallel execution that takes optimal advantage of todays multicore hardware. Being written in Rust means DataFusion can offer &lt;em&gt;both&lt;/em&gt; the safety of dynamic languages as well as the resource efficiency of a compiled language.&lt;/p&gt;
&lt;p&gt;The Apache Arrow team is pleased to announce the DataFusion 7.0.0 release. This covers 4 months of development work
and includes 195 commits from the following 37 distinct contributors.&lt;/p&gt;
&lt;!--
git log --pretty=oneline 5.0.0..6.0.0 datafusion datafusion-cli datafusion-examples | wc -l
     134

git shortlog -sn 5.0.0..6.0.0 datafusion datafusion-cli datafusion-examples | wc -l
      29

      Carlos and xudong963 are same individual
--&gt;
&lt;pre&gt;&lt;code&gt;    44  Andrew Lamb
    24  Kun Liu
    23  Jiayu Liu
    17  xudong.w
    11  Yijie Shen
     9  Matthew Turner
     7  Liang-Chi Hsieh
     5  Lin Ma
     4  Stephen Carman
     4  James Katz
     4  Dmitry Patsura
     4  QP Hou
     3  dependabot[bot]
     3  Remzi Yang
     3  Yang
     3  ic4y
     3  Dani&amp;euml;l Heres
     2  Andy Grove
     2  Raphael Taylor-Davies
     2  Jason Tianyi Wang
     2  Dan Harris
     2  Sergey Melnychuk
     1  Nitish Tiwari
     1  Dom
     1  Eduard Karacharov
     1  Javier Goday
     1  Boaz
     1  Marko Mikulicic
     1  Max Burke
     1  Carol (Nichols || Goulding)
     1  Phillip Cloud
     1  Rich
     1  Toby Hede
     1  Will Jones
     1  r.4ntix
     1  rdettai
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;The following section highlights some of the improvements in this release. Of course, many other bug fixes and improvements have also been made and we refer you to the complete &lt;a href="https://github.com/apache/arrow-datafusion/blob/7.0.0/datafusion/CHANGELOG.md"&gt;changelog&lt;/a&gt; for the full detail.&lt;/p&gt;
&lt;h1 id="summary"&gt;Summary&lt;a class="headerlink" href="#summary" title="Permanent link"&gt;&amp;para;&lt;/a&gt;&lt;/h1&gt;
&lt;ul&gt;
&lt;li&gt;DataFusion Crate&lt;/li&gt;
&lt;li&gt;The DataFusion crate is being split into multiple crates to decrease compilation times and improve the development experience. Initially, &lt;code&gt;datafusion-common&lt;/code&gt; (the core DataFusion components) and &lt;code&gt;datafusion-expr&lt;/code&gt; (DataFusion expressions, functions, and operators) have been split out. There will be additional splits after the 7.0 release.&lt;/li&gt;
&lt;li&gt;Performance Improvements and Optimizations&lt;/li&gt;
&lt;li&gt;Arrow&amp;rsquo;s dyn scalar kernels are now used to enable efficient operations on &lt;code&gt;DictionaryArray&lt;/code&gt;s &lt;a href="https://github.com/apache/arrow-datafusion/pull/1685"&gt;#1685&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;Switch from &lt;code&gt;std::sync::Mutex&lt;/code&gt; to &lt;code&gt;parking_lot::Mutex&lt;/code&gt; &lt;a href="https://github.com/apache/arrow-datafusion/pull/1720"&gt;#1720&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;New Features&lt;/li&gt;
&lt;li&gt;Support for memory tracking and spilling to disk&lt;ul&gt;
&lt;li&gt;MemoryManager and DiskManager &lt;a href="https://github.com/apache/arrow-datafusion/pull/1526"&gt;#1526&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;Out of core sort &lt;a href="https://github.com/apache/arrow-datafusion/pull/1526"&gt;#1526&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;New metrics&lt;/li&gt;
&lt;li&gt;&lt;code&gt;Gauge&lt;/code&gt; and &lt;code&gt;CurrentMemoryUsage&lt;/code&gt; &lt;a href="https://github.com/apache/arrow-datafusion/pull/1682"&gt;#1682&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;&lt;code&gt;Spill_count&lt;/code&gt; and &lt;code&gt;spilled_bytes&lt;/code&gt; &lt;a href="https://github.com/apache/arrow-datafusion/pull/1641"&gt;#1641&lt;/a&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/li&gt;
&lt;li&gt;New math functions&lt;ul&gt;
&lt;li&gt;&lt;code&gt;Approx_quantile&lt;/code&gt; &lt;a href="https://github.com/apache/arrow-datafusion/pull/1539"&gt;#1529&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;&lt;code&gt;stddev&lt;/code&gt; and &lt;code&gt;variance&lt;/code&gt; (sample and population) &lt;a href="https://github.com/apache/arrow-datafusion/pull/1525"&gt;#1525&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;&lt;code&gt;corr&lt;/code&gt; &lt;a href="https://github.com/apache/arrow-datafusion/pull/1561"&gt;#1561&lt;/a&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/li&gt;
&lt;li&gt;Support decimal type &lt;a href="https://github.com/apache/arrow-datafusion/pull/1394"&gt;#1394&lt;/a&gt;&lt;a href="https://github.com/apache/arrow-datafusion/pull/1407"&gt;#1407&lt;/a&gt;&lt;a href="https://github.com/apache/arrow-datafusion/pull/1408"&gt;#1408&lt;/a&gt;&lt;a href="https://github.com/apache/arrow-datafusion/pull/1431"&gt;#1431&lt;/a&gt;&lt;a href="https://github.com/apache/arrow-datafusion/pull/1483"&gt;#1483&lt;/a&gt;&lt;a href="https://github.com/apache/arrow-datafusion/pull/1554"&gt;#1554&lt;/a&gt;&lt;a href="https://github.com/apache/arrow-datafusion/pull/1640"&gt;#1640&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;Support for reading Parquet files with evolved schemas &lt;a href="https://github.com/apache/arrow-datafusion/pull/1622"&gt;#1622&lt;/a&gt;&lt;a href="https://github.com/apache/arrow-datafusion/pull/1709"&gt;#1709&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;Support for registering &lt;code&gt;DataFrame&lt;/code&gt; as table &lt;a href="https://github.com/apache/arrow-datafusion/pull/1699"&gt;#1699&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;Support for the &lt;code&gt;substring&lt;/code&gt; function &lt;a href="https://github.com/apache/arrow-datafusion/pull/1621"&gt;#1621&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;Support &lt;code&gt;array_agg(distinct ...)&lt;/code&gt; &lt;a href="https://github.com/apache/arrow-datafusion/pull/1579"&gt;#1579&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;Support &lt;code&gt;sort&lt;/code&gt; on unprojected columns &lt;a href="https://github.com/apache/arrow-datafusion/pull/1415"&gt;#1415&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;Additional Integration Points&lt;/li&gt;
&lt;li&gt;A new public Expression simplification API &lt;a href="https://github.com/apache/arrow-datafusion/pull/1717"&gt;#1717&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;&lt;a href="https://github.com/datafusion-contrib"&gt;DataFusion-Contrib&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;A new GitHub organization created as a home for both &lt;code&gt;DataFusion&lt;/code&gt; extensions and as a testing ground for new features.&lt;ul&gt;
&lt;li&gt;Extensions&lt;/li&gt;
&lt;li&gt;&lt;a href="https://github.com/datafusion-contrib/datafusion-python"&gt;DataFusion-Python&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;&lt;a href="https://github.com/datafusion-contrib/datafusion-java"&gt;DataFusion-Java&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;&lt;a href="https://github.com/datafusion-contrib/datafusion-hdfs-native"&gt;DataFusion-hdsfs-native&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;&lt;a href="https://github.com/datafusion-contrib/datafusion-objectstore-s3"&gt;DataFusion-ObjectStore-s3&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;New Features&lt;/li&gt;
&lt;li&gt;&lt;a href="https://github.com/datafusion-contrib/datafusion-streams"&gt;DataFusion-Streams&lt;/a&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/li&gt;
&lt;li&gt;&lt;a href="https://github.com/jorgecarleitao/arrow2"&gt;Arrow2&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;An &lt;a href="https://github.com/apache/arrow-datafusion/tree/arrow2"&gt;Arrow2 Branch&lt;/a&gt; has been created.  There are ongoing discussions in &lt;a href="https://github.com/apache/arrow-datafusion/issues/1532"&gt;DataFusion&lt;/a&gt; and &lt;a href="https://github.com/apache/arrow-rs/issues/1176"&gt;arrow-rs&lt;/a&gt; about migrating &lt;code&gt;DataFusion&lt;/code&gt; to &lt;code&gt;Arrow2&lt;/code&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;h1 id="documentation-and-roadmap"&gt;Documentation and Roadmap&lt;a class="headerlink" href="#documentation-and-roadmap" title="Permanent link"&gt;&amp;para;&lt;/a&gt;&lt;/h1&gt;
&lt;p&gt;We are working to consolidate the documentation into the &lt;a href="https://arrow.apache.org/datafusion"&gt;official site&lt;/a&gt;.  You can find more details there on topics such as the &lt;a href="https://arrow.apache.org/datafusion/user-guide/sql/index.html"&gt;SQL status&lt;/a&gt;  and a &lt;a href="https://arrow.apache.org/datafusion/user-guide/introduction.html#introduction"&gt;user guide&lt;/a&gt;. This is also an area we would love to get help from the broader community &lt;a href="https://github.com/apache/arrow-datafusion/issues/1821"&gt;#1821&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;To provide transparency on DataFusion&amp;rsquo;s priorities to users and developers a three month roadmap will be published at the beginning of each quarter.  This can be found here &lt;a href="https://arrow.apache.org/datafusion/specification/roadmap.html"&gt;here&lt;/a&gt;.&lt;/p&gt;
&lt;h1 id="upcoming-attractions"&gt;Upcoming Attractions&lt;a class="headerlink" href="#upcoming-attractions" title="Permanent link"&gt;&amp;para;&lt;/a&gt;&lt;/h1&gt;
&lt;ul&gt;
&lt;li&gt;Ballista is gaining momentum, and several groups are now evaluating and contributing to the project.&lt;/li&gt;
&lt;li&gt;Some of the proposed improvements&lt;ul&gt;
&lt;li&gt;&lt;a href="https://github.com/apache/arrow-datafusion/issues/1701"&gt;Improvements Overview&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;&lt;a href="https://github.com/apache/arrow-datafusion/issues/1675"&gt;Extensibility&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;&lt;a href="https://github.com/apache/arrow-datafusion/issues/1702"&gt;File system access&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;&lt;a href="https://github.com/apache/arrow-datafusion/issues/1704"&gt;Cluster state&lt;/a&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/li&gt;
&lt;li&gt;Continued improvements for working with limited resources and large datasets&lt;/li&gt;
&lt;li&gt;Memory limited joins&lt;a href="https://github.com/apache/arrow-datafusion/issues/1599"&gt;#1599&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;Sort-merge join&lt;a href="https://github.com/apache/arrow-datafusion/issues/141"&gt;#141&lt;/a&gt;&lt;a href="https://github.com/apache/arrow-datafusion/pull/1776"&gt;#1776&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;Introduce row based bytes representation &lt;a href="https://github.com/apache/arrow-datafusion/pull/1708"&gt;#1708&lt;/a&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;h1 id="how-to-get-involved"&gt;How to Get Involved&lt;a class="headerlink" href="#how-to-get-involved" title="Permanent link"&gt;&amp;para;&lt;/a&gt;&lt;/h1&gt;
&lt;p&gt;If you are interested in contributing to DataFusion, and learning about state of
the art query processing, we would love to have you join us on the journey! You
can help by trying out DataFusion on some of your own data and projects and let us know how it goes or contribute a PR with documentation, tests or code. A list of open issues suitable for beginners is &lt;a href="https://github.com/apache/arrow-datafusion/issues?q=is%3Aissue+is%3Aopen+label%3A%22good+first+issue%22"&gt;here&lt;/a&gt;&lt;/p&gt;
&lt;p&gt;Check out our new &lt;a href="https://arrow.apache.org/datafusion/community/communication.html"&gt;Communication Doc&lt;/a&gt; on more
ways to engage with the community.&lt;/p&gt;</content><category term="blog"/></entry><entry><title>Apache Arrow DataFusion 6.0.0 Release</title><link href="https://datafusion.apache.org/blog/2021/11/19/2021-11-8-datafusion-6.0.0.md" rel="alternate"/><published>2021-11-19T00:00:00+00:00</published><updated>2021-11-19T00:00:00+00:00</updated><author><name>pmc</name></author><id>tag:datafusion.apache.org,2021-11-19:/blog/2021/11/19/2021-11-8-datafusion-6.0.0.md</id><summary type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;
&lt;h1 id="introduction"&gt;Introduction&lt;a class="headerlink" href="#introduction" title="Permanent link"&gt;&amp;para;&lt;/a&gt;&lt;/h1&gt;
&lt;p&gt;&lt;a href="https://arrow.apache.org/datafusion/"&gt;DataFusion&lt;/a&gt; is an embedded
query engine which leverages the unique features of
&lt;a href="https://www.rust-lang.org/"&gt;Rust&lt;/a&gt; and &lt;a href="https://arrow.apache.org/"&gt;Apache
Arrow&lt;/a&gt; to provide a system that is high
performance, easy to connect, easy to embed, and high quality.&lt;/p&gt;
&lt;p&gt;The Apache Arrow team is pleased to announce the DataFusion 6.0.0 release. This covers …&lt;/p&gt;</summary><content type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;
&lt;h1 id="introduction"&gt;Introduction&lt;a class="headerlink" href="#introduction" title="Permanent link"&gt;&amp;para;&lt;/a&gt;&lt;/h1&gt;
&lt;p&gt;&lt;a href="https://arrow.apache.org/datafusion/"&gt;DataFusion&lt;/a&gt; is an embedded
query engine which leverages the unique features of
&lt;a href="https://www.rust-lang.org/"&gt;Rust&lt;/a&gt; and &lt;a href="https://arrow.apache.org/"&gt;Apache
Arrow&lt;/a&gt; to provide a system that is high
performance, easy to connect, easy to embed, and high quality.&lt;/p&gt;
&lt;p&gt;The Apache Arrow team is pleased to announce the DataFusion 6.0.0 release. This covers 4 months of development work
and includes 134 commits from the following 28 distinct contributors.&lt;/p&gt;
&lt;!--
git log --pretty=oneline 5.0.0..6.0.0 datafusion datafusion-cli datafusion-examples | wc -l
     134

git shortlog -sn 5.0.0..6.0.0 datafusion datafusion-cli datafusion-examples | wc -l
      29

      Carlos and xudong963 are same individual
--&gt;
&lt;pre&gt;&lt;code&gt;    28  Andrew Lamb
    26  Jiayu Liu
    13  xudong963
     9  rdettai
     9  QP Hou
     6  Matthew Turner
     5  Dani&amp;euml;l Heres
     4  Guillaume Balaine
     3  Francis Du
     3  Marco Neumann
     3  Jon Mease
     3  Nga Tran
     2  Yijie Shen
     2  Ruihang Xia
     2  Liang-Chi Hsieh
     2  baishen
     2  Andy Grove
     2  Jason Tianyi Wang
     1  Nan Zhu
     1  Antoine Wendlinger
     1  Kriszti&amp;aacute;n Sz&amp;udblac;cs
     1  Mike Seddon
     1  Conner Murphy
     1  Patrick More
     1  Taehoon Moon
     1  Tiphaine Ruy
     1  adsharma
     1  lichuan6
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;The release notes below are not exhaustive and only expose selected highlights of the release. Many other bug fixes
and improvements have been made: we refer you to the complete
&lt;a href="https://github.com/apache/arrow-datafusion/blob/6.0.0/datafusion/CHANGELOG.md"&gt;changelog&lt;/a&gt;.&lt;/p&gt;
&lt;h1 id="new-website"&gt;New Website&lt;a class="headerlink" href="#new-website" title="Permanent link"&gt;&amp;para;&lt;/a&gt;&lt;/h1&gt;
&lt;p&gt;Befitting a growing project, DataFusion now has its
&lt;a href="https://arrow.apache.org/datafusion/"&gt;own website&lt;/a&gt; hosted as part of the
main &lt;a href="https://arrow.apache.org"&gt;Apache Arrow Website&lt;/a&gt;&lt;/p&gt;
&lt;h1 id="roadmap"&gt;Roadmap&lt;a class="headerlink" href="#roadmap" title="Permanent link"&gt;&amp;para;&lt;/a&gt;&lt;/h1&gt;
&lt;p&gt;The community worked to gather their thoughts about where we are
taking DataFusion into a public
&lt;a href="https://arrow.apache.org/datafusion/specification/roadmap.html"&gt;Roadmap&lt;/a&gt;
for the first time&lt;/p&gt;
&lt;h1 id="new-features"&gt;New Features&lt;a class="headerlink" href="#new-features" title="Permanent link"&gt;&amp;para;&lt;/a&gt;&lt;/h1&gt;
&lt;ul&gt;
&lt;li&gt;Runtime operator metrics collection framework&lt;/li&gt;
&lt;li&gt;Object store abstraction for unified access to local or remote storage&lt;/li&gt;
&lt;li&gt;Hive style table partitioning support, for Parquet, CSV, Avro and Json files&lt;/li&gt;
&lt;li&gt;DataFrame API support for: &lt;code&gt;except&lt;/code&gt;, &lt;code&gt;intersect&lt;/code&gt;, &lt;code&gt;show&lt;/code&gt;, &lt;code&gt;limit&lt;/code&gt; and window functions&lt;/li&gt;
&lt;li&gt;SQL&lt;/li&gt;
&lt;li&gt;&lt;code&gt;EXPLAIN ANALYZE&lt;/code&gt; with runtime metrics&lt;/li&gt;
&lt;li&gt;&lt;code&gt;trim ( [ LEADING | TRAILING | BOTH ] [ FROM ] string text [, characters text ] )&lt;/code&gt; syntax&lt;/li&gt;
&lt;li&gt;Postgres style regular expression matching operators &lt;code&gt;~&lt;/code&gt;, &lt;code&gt;~*&lt;/code&gt;, &lt;code&gt;!~&lt;/code&gt;, and &lt;code&gt;!~*&lt;/code&gt;&lt;/li&gt;
&lt;li&gt;SQL set operators &lt;code&gt;UNION&lt;/code&gt;, &lt;code&gt;INTERSECT&lt;/code&gt;, and &lt;code&gt;EXCEPT&lt;/code&gt;&lt;/li&gt;
&lt;li&gt;&lt;code&gt;cume_dist&lt;/code&gt;, &lt;code&gt;percent_rank&lt;/code&gt; window functions&lt;/li&gt;
&lt;li&gt;&lt;code&gt;digest&lt;/code&gt;, &lt;code&gt;blake2s&lt;/code&gt;, &lt;code&gt;blake2b&lt;/code&gt;, &lt;code&gt;blake3&lt;/code&gt; crypto functions&lt;/li&gt;
&lt;li&gt;HyperLogLog based &lt;code&gt;approx_distinct&lt;/code&gt;&lt;/li&gt;
&lt;li&gt;&lt;code&gt;is distinct from&lt;/code&gt; and &lt;code&gt;is not distinct from&lt;/code&gt;&lt;/li&gt;
&lt;li&gt;&lt;code&gt;CREATE TABLE AS SELECT&lt;/code&gt;&lt;/li&gt;
&lt;li&gt;Accessing elements of nested &lt;code&gt;Struct&lt;/code&gt; and &lt;code&gt;List&lt;/code&gt; columns (e.g. &lt;code&gt;SELECT struct_column['field_name'], array_column[0] FROM ...&lt;/code&gt;)&lt;/li&gt;
&lt;li&gt;Boolean expressions in &lt;code&gt;CASE&lt;/code&gt; statement&lt;/li&gt;
&lt;li&gt;&lt;code&gt;DROP TABLE&lt;/code&gt;&lt;/li&gt;
&lt;li&gt;&lt;code&gt;VALUES&lt;/code&gt; List&lt;/li&gt;
&lt;li&gt;Postgres regex match operators&lt;/li&gt;
&lt;li&gt;Support for Avro format&lt;/li&gt;
&lt;li&gt;Support for &lt;code&gt;ScalarValue::Struct&lt;/code&gt;&lt;/li&gt;
&lt;li&gt;Automatic schema inference for CSV files&lt;/li&gt;
&lt;li&gt;Better interactive editing support in &lt;code&gt;datafusion-cli&lt;/code&gt; as well as &lt;code&gt;psql&lt;/code&gt; style commands such as &lt;code&gt;\d&lt;/code&gt;, &lt;code&gt;\?&lt;/code&gt;, and &lt;code&gt;\q&lt;/code&gt;&lt;/li&gt;
&lt;li&gt;Generic constant evaluation and simplification framework&lt;/li&gt;
&lt;li&gt;Added common subexpression eliminate query plan optimization rule&lt;/li&gt;
&lt;li&gt;Python binding 0.4.0 with all Datafusion 6.0.0 features&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;With these new features, we are also now passing TPC-H queries 8, 13 and 21.&lt;/p&gt;
&lt;p&gt;For the full list of new features with their relevant PRs, see the
&lt;a href="https://github.com/apache/arrow-datafusion/blob/6.0.0/datafusion/CHANGELOG.md"&gt;enhancements section&lt;/a&gt;
in the changelog.&lt;/p&gt;
&lt;h1 id="async-planning-and-decoupling-file-format-from-table-layout"&gt;&lt;code&gt;async&lt;/code&gt; planning and decoupling file format from table layout&lt;a class="headerlink" href="#async-planning-and-decoupling-file-format-from-table-layout" title="Permanent link"&gt;&amp;para;&lt;/a&gt;&lt;/h1&gt;
&lt;p&gt;Driven by the need to support Hive style table partitioning, @rdettai
introduced the following design change to the Datafusion core.&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;The code for reading specific file formats (&lt;code&gt;Parquet&lt;/code&gt;, &lt;code&gt;Avro&lt;/code&gt;, &lt;code&gt;CSV&lt;/code&gt;, and
&lt;code&gt;JSON&lt;/code&gt;) was separated from the logic that handles grouping sets of
files into execution partitions.&lt;/li&gt;
&lt;li&gt;The query planning process was made &lt;code&gt;async&lt;/code&gt;.&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;As a result, we are able to replace the old &lt;code&gt;Parquet&lt;/code&gt;, &lt;code&gt;CSV&lt;/code&gt; and &lt;code&gt;JSON&lt;/code&gt; table
providers with a single &lt;code&gt;ListingTable&lt;/code&gt; table provider.&lt;/p&gt;
&lt;p&gt;This also sets up DataFusion and its plug-in ecosystem to
supporting a wide range of catalogs and various object store implementations.
You can read more about this change in the
&lt;a href="https://docs.google.com/document/d/1Bd4-PLLH-pHj0BquMDsJ6cVr_awnxTuvwNJuWsTHxAQ"&gt;design document&lt;/a&gt;
and on the &lt;a href="https://github.com/apache/arrow-datafusion/pull/1010"&gt;arrow-datafusion#1010 PR&lt;/a&gt;.&lt;/p&gt;
&lt;h1 id="how-to-get-involved"&gt;How to Get Involved&lt;a class="headerlink" href="#how-to-get-involved" title="Permanent link"&gt;&amp;para;&lt;/a&gt;&lt;/h1&gt;
&lt;p&gt;If you are interested in contributing to DataFusion, we would love to have you! You
can help by trying out DataFusion on some of your own data and projects and filing bug reports and helping to
improve the documentation, or contribute to the documentation, tests or code. A list of open issues suitable for
beginners is &lt;a href="https://github.com/apache/arrow-datafusion/issues?q=is%3Aissue+is%3Aopen+label%3A%22good+first+issue%22"&gt;here&lt;/a&gt;
and the full list is &lt;a href="https://github.com/apache/arrow-datafusion/issues"&gt;here&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;Check out our new &lt;a href="https://arrow.apache.org/datafusion/community/communication.html"&gt;Communication Doc&lt;/a&gt; on more
ways to engage with the community.&lt;/p&gt;</content><category term="blog"/></entry><entry><title>Apache Arrow Ballista 0.5.0 Release</title><link href="https://datafusion.apache.org/blog/2021/08/18/ballista-0.5.0" rel="alternate"/><published>2021-08-18T00:00:00+00:00</published><updated>2021-08-18T00:00:00+00:00</updated><author><name>pmc</name></author><id>tag:datafusion.apache.org,2021-08-18:/blog/2021/08/18/ballista-0.5.0</id><summary type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;p&gt;Ballista extends DataFusion to provide support for distributed queries. This is the first release of Ballista since 
the project was &lt;a href="https://arrow.apache.org/blog/2021/04/12/ballista-donation/"&gt;donated&lt;/a&gt; to the Apache Arrow project 
and includes 80 commits from 11 contributors.&lt;/p&gt;
&lt;pre&gt;&lt;code&gt;git shortlog -sn 4.0.0..5.0.0 ballista/rust/client ballista/rust/core ballista/rust …&lt;/code&gt;&lt;/pre&gt;</summary><content type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;p&gt;Ballista extends DataFusion to provide support for distributed queries. This is the first release of Ballista since 
the project was &lt;a href="https://arrow.apache.org/blog/2021/04/12/ballista-donation/"&gt;donated&lt;/a&gt; to the Apache Arrow project 
and includes 80 commits from 11 contributors.&lt;/p&gt;
&lt;pre&gt;&lt;code&gt;git shortlog -sn 4.0.0..5.0.0 ballista/rust/client ballista/rust/core ballista/rust/executor ballista/rust/scheduler
  27  Andy Grove
  15  Jiayu Liu
  12  Andrew Lamb
   8  Ximo Guanter
   6  Daniël Heres
   5  QP Hou
   2  Jorge Leitao
   1  Javier Goday
   1  K.I. (Dennis) Jung
   1  Mike Seddon
   1  sathis
&lt;/code&gt;&lt;/pre&gt;
&lt;!--
$ git log --pretty=oneline 4.0.0..5.0.0 ballista/rust/client ballista/rust/core ballista/rust/executor ballista/rust/scheduler ballista-examples/ | wc -l
80
--&gt;
&lt;p&gt;The release notes below are not exhaustive and only expose selected highlights of the release. Many other bug fixes 
and improvements have been made: we refer you to the &lt;a href="https://github.com/apache/arrow-datafusion/blob/5.0.0/ballista/CHANGELOG.md"&gt;complete changelog&lt;/a&gt;.&lt;/p&gt;
&lt;h1 id="performance-and-scalability"&gt;Performance and Scalability&lt;a class="headerlink" href="#performance-and-scalability" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h1&gt;
&lt;p&gt;Ballista is now capable of running complex SQL queries at scale and supports scalable distributed joins. We have been 
benchmarking using individual queries from the TPC-H benchmark at scale factors up to 1000 (1 TB). When running against 
CSV files, performance is generally very close to DataFusion, and significantly faster in some cases due to the fact 
that the scheduler limits the number of concurrent tasks that run at any given time. Performance against large Parquet 
datasets is currently non ideal due to some issues (&lt;a href="https://github.com/apache/arrow-datafusion/issues/867"&gt;#867&lt;/a&gt;, 
&lt;a href="https://github.com/apache/arrow-datafusion/issues/868"&gt;#868&lt;/a&gt;) that we hope to resolve for the next release. &lt;/p&gt;
&lt;h1 id="new-features"&gt;New Features&lt;a class="headerlink" href="#new-features" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h1&gt;
&lt;p&gt;The main new features in this release are:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;Ballista queries can now be executed by calling DataFrame.collect()&lt;/li&gt;
&lt;li&gt;The shuffle mechanism has been re-implemented&lt;/li&gt;
&lt;li&gt;Distributed hash-partitioned joins are now supported&lt;/li&gt;
&lt;li&gt;Keda autoscaling is supported&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;To get started with Ballista, refer to the &lt;a href="https://docs.rs/ballista/0.5.0/ballista/"&gt;crate documentation&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;Now that the basic functionality is in place, the focus for the next release will be to improve the performance and
scalability as well as improving the documentation.&lt;/p&gt;
&lt;h1 id="how-to-get-involved"&gt;How to Get Involved&lt;a class="headerlink" href="#how-to-get-involved" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h1&gt;
&lt;p&gt;If you are interested in contributing to Ballista, we would love to have you! You
can help by trying out Ballista on some of your own data and projects and filing bug reports and helping to
improve the documentation, or contribute to the documentation, tests or code. A list of open issues suitable for
beginners is &lt;a href="https://github.com/apache/arrow-datafusion/issues?q=is%3Aissue+is%3Aopen+label%3A%22good+first+issue%22"&gt;here&lt;/a&gt;
and the full list is &lt;a href="https://github.com/apache/arrow-datafusion/issues"&gt;here&lt;/a&gt;.&lt;/p&gt;</content><category term="blog"/></entry><entry><title>Apache Arrow DataFusion 5.0.0 Release</title><link href="https://datafusion.apache.org/blog/2021/08/18/datafusion-5.0.0" rel="alternate"/><published>2021-08-18T00:00:00+00:00</published><updated>2021-08-18T00:00:00+00:00</updated><author><name>pmc</name></author><id>tag:datafusion.apache.org,2021-08-18:/blog/2021/08/18/datafusion-5.0.0</id><summary type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;
&lt;p&gt;The Apache Arrow team is pleased to announce the DataFusion 5.0.0 release. This covers 4 months of development work 
and includes 211 commits from the following 31 distinct contributors.&lt;/p&gt;
&lt;pre&gt;&lt;code&gt;$ git shortlog -sn 4.0.0..5.0.0 datafusion datafusion-cli datafusion-examples
    61  Jiayu Liu
    47  Andrew Lamb
    27 …&lt;/code&gt;&lt;/pre&gt;</summary><content type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;
&lt;p&gt;The Apache Arrow team is pleased to announce the DataFusion 5.0.0 release. This covers 4 months of development work 
and includes 211 commits from the following 31 distinct contributors.&lt;/p&gt;
&lt;pre&gt;&lt;code&gt;$ git shortlog -sn 4.0.0..5.0.0 datafusion datafusion-cli datafusion-examples
    61  Jiayu Liu
    47  Andrew Lamb
    27  Dani&amp;euml;l Heres
    13  QP Hou
    13  Andy Grove
     4  Javier Goday
     4  sathis
     3  Ruan Pearce-Authers
     3  Raphael Taylor-Davies
     3  Jorge Leitao
     3  Cui Wenzheng
     3  Mike Seddon
     3  Edd Robinson
     2  思维
     2  Liang-Chi Hsieh
     2  Michael Lu
     2  Parth Sarthy
     2  Patrick More
     2  Rich
     1  Charlie Evans
     1  Gang Liao
     1  Agata Naomichi
     1  Ritchie Vink
     1  Evan Chan
     1  Ruihang Xia
     1  Todd Treece
     1  Yichen Wang
     1  baishen
     1  Nga Tran
     1  rdettai
     1  Marco Neumann
&lt;/code&gt;&lt;/pre&gt;
&lt;!--
$ git log --pretty=oneline 4.0.0..5.0.0 datafusion datafusion-cli datafusion-examples | wc -l
     211
--&gt;
&lt;p&gt;The release notes below are not exhaustive and only expose selected highlights of the release. Many other bug fixes 
and improvements have been made: we refer you to the complete 
&lt;a href="https://github.com/apache/arrow-datafusion/blob/5.0.0/datafusion/CHANGELOG.md"&gt;changelog&lt;/a&gt;.&lt;/p&gt;
&lt;h1 id="performance"&gt;Performance&lt;a class="headerlink" href="#performance" title="Permanent link"&gt;&amp;para;&lt;/a&gt;&lt;/h1&gt;
&lt;p&gt;There have been numerous performance improvements in this release. The following chart shows the relative 
performance of individual TPC-H queries compared to the previous release.&lt;/p&gt;
&lt;p&gt;&lt;i&gt;TPC-H @ scale factor 100, in parquet format. Concurrency 24.&lt;/i&gt;&lt;/p&gt;
&lt;p&gt;&lt;img src="/blog/images/2021-08-18-datafusion500perf.png"/&gt;&lt;/p&gt;
&lt;p&gt;We also extended support for more TPC-H queries: q7, q8, q9 and q13 are running successfully in DataFusion 5.0.&lt;/p&gt;
&lt;h1 id="new-features"&gt;New Features&lt;a class="headerlink" href="#new-features" title="Permanent link"&gt;&amp;para;&lt;/a&gt;&lt;/h1&gt;
&lt;ul&gt;
&lt;li&gt;Initial support for SQL-99 Analytics (WINDOW functions)&lt;/li&gt;
&lt;li&gt;Improved JOIN support: cross join, semi-join, anti join, and fixes to null handling&lt;/li&gt;
&lt;li&gt;Improved EXPLAIN support&lt;/li&gt;
&lt;li&gt;Initial implementation of metrics in the physical plan&lt;/li&gt;
&lt;li&gt;Support for SELECT DISTINCT&lt;/li&gt;
&lt;li&gt;Support for Json and NDJson formatted inputs&lt;/li&gt;
&lt;li&gt;Query column with relations&lt;/li&gt;
&lt;li&gt;Added more datetime related functions: &lt;code&gt;now&lt;/code&gt;, &lt;code&gt;date_trunc&lt;/code&gt;, &lt;code&gt;to_timestamp_millis&lt;/code&gt;, &lt;code&gt;to_timestamp_micros&lt;/code&gt;, &lt;code&gt;to_timestamp_seconds&lt;/code&gt;&lt;/li&gt;
&lt;li&gt;Streaming Dataframe.collect&lt;/li&gt;
&lt;li&gt;Support table column aliases&lt;/li&gt;
&lt;li&gt;Answer count(*), min() and max() queries using only statistics&lt;/li&gt;
&lt;li&gt;Non-equi-join filters in JOIN conditions&lt;/li&gt;
&lt;li&gt;Modulus operation&lt;/li&gt;
&lt;li&gt;Support group by column positions&lt;/li&gt;
&lt;li&gt;Added constant folding query optimizer&lt;/li&gt;
&lt;li&gt;Hash partitioned aggregation&lt;/li&gt;
&lt;li&gt;Added &lt;code&gt;random&lt;/code&gt; SQL function&lt;/li&gt;
&lt;li&gt;Implemented count distinct for floats and dictionary types&lt;/li&gt;
&lt;li&gt;Re-exported arrow and parquet crates in Datafusion&lt;/li&gt;
&lt;li&gt;General row group pruning logic that&amp;rsquo;s agnostic to storage format&lt;/li&gt;
&lt;/ul&gt;
&lt;h1 id="how-to-get-involved"&gt;How to Get Involved&lt;a class="headerlink" href="#how-to-get-involved" title="Permanent link"&gt;&amp;para;&lt;/a&gt;&lt;/h1&gt;
&lt;p&gt;If you are interested in contributing to DataFusion, we would love to have you! You 
can help by trying out DataFusion on some of your own data and projects and filing bug reports and helping to 
improve the documentation, or contribute to the documentation, tests or code. A list of open issues suitable for 
beginners is &lt;a href="https://github.com/apache/arrow-datafusion/issues?q=is%3Aissue+is%3Aopen+label%3A%22good+first+issue%22"&gt;here&lt;/a&gt; 
and the full list is &lt;a href="https://github.com/apache/arrow-datafusion/issues"&gt;here&lt;/a&gt;.&lt;/p&gt;</content><category term="blog"/></entry><entry><title>Ballista: A Distributed Scheduler for Apache Arrow</title><link href="https://datafusion.apache.org/blog/2021/04/12/ballista-donation" rel="alternate"/><published>2021-04-12T00:00:00+00:00</published><updated>2021-04-12T00:00:00+00:00</updated><author><name>agrove</name></author><id>tag:datafusion.apache.org,2021-04-12:/blog/2021/04/12/ballista-donation</id><summary type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;p&gt;We are excited to announce that &lt;a href="https://github.com/apache/arrow-datafusion/tree/master/ballista"&gt;Ballista&lt;/a&gt; has been donated 
to the Apache Arrow project. &lt;/p&gt;
&lt;p&gt;Ballista is a distributed compute platform primarily implemented in Rust, and powered by Apache Arrow. It is built
on an architecture that allows other programming languages (such as Python, C++, and Java) to be supported …&lt;/p&gt;</summary><content type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;p&gt;We are excited to announce that &lt;a href="https://github.com/apache/arrow-datafusion/tree/master/ballista"&gt;Ballista&lt;/a&gt; has been donated 
to the Apache Arrow project. &lt;/p&gt;
&lt;p&gt;Ballista is a distributed compute platform primarily implemented in Rust, and powered by Apache Arrow. It is built
on an architecture that allows other programming languages (such as Python, C++, and Java) to be supported as
first-class citizens without paying a penalty for serialization costs.&lt;/p&gt;
&lt;p&gt;The foundational technologies in Ballista are:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;a href="https://arrow.apache.org/"&gt;Apache Arrow&lt;/a&gt; memory model and compute kernels for efficient processing of data.&lt;/li&gt;
&lt;li&gt;&lt;a href="https://github.com/apache/arrow-datafusion"&gt;Apache Arrow DataFusion&lt;/a&gt; query planning and 
  execution framework, extended by Ballista to provide distributed planning and execution.&lt;/li&gt;
&lt;li&gt;&lt;a href="https://arrow.apache.org/blog/2019/10/13/introducing-arrow-flight/"&gt;Apache Arrow Flight Protocol&lt;/a&gt; for efficient
  data transfer between processes.&lt;/li&gt;
&lt;li&gt;&lt;a href="https://developers.google.com/protocol-buffers"&gt;Google Protocol Buffers&lt;/a&gt; for serializing query plans.&lt;/li&gt;
&lt;li&gt;&lt;a href="https://www.docker.com/"&gt;Docker&lt;/a&gt; for packaging up executors along with user-defined code.&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;Ballista can be deployed as a standalone cluster and also supports &lt;a href="https://kubernetes.io/"&gt;Kubernetes&lt;/a&gt;. In either
case, the scheduler can be configured to use &lt;a href="https://etcd.io/"&gt;etcd&lt;/a&gt; as a backing store to (eventually) provide
redundancy in the case of a scheduler failing.&lt;/p&gt;
&lt;h2 id="status"&gt;Status&lt;a class="headerlink" href="#status" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;The Ballista project is at an early stage of development. However, it is capable of running complex analytics queries 
in a distributed cluster with reasonable performance (comparable to more established distributed query frameworks).&lt;/p&gt;
&lt;p&gt;One of the benefits of Ballista being part of the Arrow codebase is that there is now an opportunity to push parts of 
the scheduler down to DataFusion so that is possible to seamlessly scale across cores in DataFusion, and across nodes 
in Ballista, using the same unified query scheduler.&lt;/p&gt;
&lt;h2 id="contributors-welcome"&gt;Contributors Welcome!&lt;a class="headerlink" href="#contributors-welcome" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;If you are excited about being able to use Rust for distributed compute and ETL and would like to contribute to this 
work then there are many ways to get involved. The simplest way to get started is to try out Ballista against your own 
datasets and file bug reports for any issues that you find. You could also check out the current 
&lt;a href="https://issues.apache.org/jira/issues/?jql=project%20%3D%20ARROW%20AND%20component%20%3D%20%22Rust%20-%20Ballista%22"&gt;list of issues&lt;/a&gt; and have a go at fixing one.&lt;/p&gt;
&lt;p&gt;The &lt;a href="https://github.com/apache/arrow/blob/master/rust/README.md#arrow-rust-community"&gt;Arrow Rust Community&lt;/a&gt;
section of the Rust README provides information on other ways to interact with the Ballista contributors and 
maintainers.&lt;/p&gt;</content><category term="blog"/></entry><entry><title>DataFusion: A Rust-native Query Engine for Apache Arrow</title><link href="https://datafusion.apache.org/blog/2019/02/04/datafusion-donation" rel="alternate"/><published>2019-02-04T00:00:00+00:00</published><updated>2019-02-04T00:00:00+00:00</updated><author><name>agrove</name></author><id>tag:datafusion.apache.org,2019-02-04:/blog/2019/02/04/datafusion-donation</id><summary type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;p&gt;We are excited to announce that &lt;a href="https://github.com/apache/arrow-datafusion"&gt;DataFusion&lt;/a&gt; has been donated to the Apache Arrow project. DataFusion is an in-memory query engine for the Rust implementation of Apache Arrow.&lt;/p&gt;
&lt;p&gt;Although DataFusion was started two years ago, it was recently re-implemented to be Arrow-native and currently has limited capabilities but does support …&lt;/p&gt;</summary><content type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;p&gt;We are excited to announce that &lt;a href="https://github.com/apache/arrow-datafusion"&gt;DataFusion&lt;/a&gt; has been donated to the Apache Arrow project. DataFusion is an in-memory query engine for the Rust implementation of Apache Arrow.&lt;/p&gt;
&lt;p&gt;Although DataFusion was started two years ago, it was recently re-implemented to be Arrow-native and currently has limited capabilities but does support SQL queries against iterators of RecordBatch and has support for CSV files. There are plans to &lt;a href="https://issues.apache.org/jira/browse/ARROW-4466"&gt;add support for Parquet files&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;SQL support is limited to projection (&lt;code&gt;SELECT&lt;/code&gt;), selection (&lt;code&gt;WHERE&lt;/code&gt;), and simple aggregates (&lt;code&gt;MIN&lt;/code&gt;, &lt;code&gt;MAX&lt;/code&gt;, &lt;code&gt;SUM&lt;/code&gt;) with an optional &lt;code&gt;GROUP BY&lt;/code&gt; clause.&lt;/p&gt;
&lt;p&gt;Supported expressions are identifiers, literals, simple math operations (&lt;code&gt;+&lt;/code&gt;, &lt;code&gt;-&lt;/code&gt;, &lt;code&gt;*&lt;/code&gt;, &lt;code&gt;/&lt;/code&gt;), binary expressions (&lt;code&gt;AND&lt;/code&gt;, &lt;code&gt;OR&lt;/code&gt;), equality and comparison operators (&lt;code&gt;=&lt;/code&gt;, &lt;code&gt;!=&lt;/code&gt;, &lt;code&gt;&amp;lt;&lt;/code&gt;, &lt;code&gt;&amp;lt;=&lt;/code&gt;, &lt;code&gt;&amp;gt;=&lt;/code&gt;, &lt;code&gt;&amp;gt;&lt;/code&gt;), and &lt;code&gt;CAST(expr AS type)&lt;/code&gt;.&lt;/p&gt;
&lt;h2 id="example"&gt;Example&lt;a class="headerlink" href="#example" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;The following example demonstrates running a simple aggregate SQL query against a CSV file.&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-rust"&gt;// create execution context
let mut ctx = ExecutionContext::new();

// define schema for data source (csv file)
let schema = Arc::new(Schema::new(vec![
    Field::new("c1", DataType::Utf8, false),
    Field::new("c2", DataType::UInt32, false),
    Field::new("c3", DataType::Int8, false),
    Field::new("c4", DataType::Int16, false),
    Field::new("c5", DataType::Int32, false),
    Field::new("c6", DataType::Int64, false),
    Field::new("c7", DataType::UInt8, false),
    Field::new("c8", DataType::UInt16, false),
    Field::new("c9", DataType::UInt32, false),
    Field::new("c10", DataType::UInt64, false),
    Field::new("c11", DataType::Float32, false),
    Field::new("c12", DataType::Float64, false),
    Field::new("c13", DataType::Utf8, false),
]));

// register csv file with the execution context
let csv_datasource =
    CsvDataSource::new("test/data/aggregate_test_100.csv", schema.clone(), 1024);
ctx.register_datasource("aggregate_test_100", Rc::new(RefCell::new(csv_datasource)));

let sql = "SELECT c1, MIN(c12), MAX(c12) FROM aggregate_test_100 WHERE c11 &amp;gt; 0.1 AND c11 &amp;lt; 0.9 GROUP BY c1";

// execute the query
let relation = ctx.sql(&amp;amp;sql).unwrap();
let mut results = relation.borrow_mut();

// iterate over the results
while let Some(batch) = results.next().unwrap() {
    println!(
        "RecordBatch has {} rows and {} columns",
        batch.num_rows(),
        batch.num_columns()
    );

    let c1 = batch
        .column(0)
        .as_any()
        .downcast_ref::&amp;lt;BinaryArray&amp;gt;()
        .unwrap();

    let min = batch
        .column(1)
        .as_any()
        .downcast_ref::&amp;lt;Float64Array&amp;gt;()
        .unwrap();

    let max = batch
        .column(2)
        .as_any()
        .downcast_ref::&amp;lt;Float64Array&amp;gt;()
        .unwrap();

    for i in 0..batch.num_rows() {
        let c1_value: String = String::from_utf8(c1.value(i).to_vec()).unwrap();
        println!("{}, Min: {}, Max: {}", c1_value, min.value(i), max.value(i),);
    }
}
&lt;/code&gt;&lt;/pre&gt;
&lt;h2 id="roadmap"&gt;Roadmap&lt;a class="headerlink" href="#roadmap" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;The roadmap for DataFusion will depend on interest from the Rust community, but here are some of the short term items that are planned:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;Extending test coverage of the existing functionality&lt;/li&gt;
&lt;li&gt;Adding support for Parquet data sources&lt;/li&gt;
&lt;li&gt;Implementing more SQL features such as &lt;code&gt;JOIN&lt;/code&gt;, &lt;code&gt;ORDER BY&lt;/code&gt; and &lt;code&gt;LIMIT&lt;/code&gt;&lt;/li&gt;
&lt;li&gt;Implement a DataFrame API as an alternative to SQL&lt;/li&gt;
&lt;li&gt;Adding support for partitioning and parallel query execution using Rust's async and await functionality&lt;/li&gt;
&lt;li&gt;Creating a Docker image to make it easy to use DataFusion as a standalone query tool for interactive and batch queries&lt;/li&gt;
&lt;/ul&gt;
&lt;h2 id="contributors-welcome"&gt;Contributors Welcome!&lt;a class="headerlink" href="#contributors-welcome" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;If you are excited about being able to use Rust for data science and would like to contribute to this work then there are many ways to get involved. The simplest way to get started is to try out DataFusion against your own data sources and file bug reports for any issues that you find. You could also check out the current &lt;a href="https://cwiki.apache.org/confluence/display/ARROW/Rust+JIRA+Dashboard"&gt;list of issues&lt;/a&gt; and have a go at fixing one. You can also join the &lt;a href="http://mail-archives.apache.org/mod_mbox/arrow-user/"&gt;user mailing list&lt;/a&gt; to ask questions.&lt;/p&gt;</content><category term="blog"/></entry></feed>