<?xml version="1.0" encoding="utf-8"?>
<feed xmlns="http://www.w3.org/2005/Atom"><title>Apache DataFusion Blog - pmc</title><link href="https://datafusion.apache.org/blog/" rel="alternate"/><link href="https://datafusion.apache.org/blog/feeds/pmc.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>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>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>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>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>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>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>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>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>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 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 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>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>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 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>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></feed>