datafusion/
lib.rs

1// Licensed to the Apache Software Foundation (ASF) under one
2// or more contributor license agreements.  See the NOTICE file
3// distributed with this work for additional information
4// regarding copyright ownership.  The ASF licenses this file
5// to you under the Apache License, Version 2.0 (the
6// "License"); you may not use this file except in compliance
7// with the License.  You may obtain a copy of the License at
8//
9//   http://www.apache.org/licenses/LICENSE-2.0
10//
11// Unless required by applicable law or agreed to in writing,
12// software distributed under the License is distributed on an
13// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
14// KIND, either express or implied.  See the License for the
15// specific language governing permissions and limitations
16// under the License.
17
18#![doc(
19    html_logo_url = "https://raw.githubusercontent.com/apache/datafusion/19fe44cf2f30cbdd63d4a4f52c74055163c6cc38/docs/logos/standalone_logo/logo_original.svg",
20    html_favicon_url = "https://raw.githubusercontent.com/apache/datafusion/19fe44cf2f30cbdd63d4a4f52c74055163c6cc38/docs/logos/standalone_logo/logo_original.svg"
21)]
22#![cfg_attr(docsrs, feature(doc_auto_cfg))]
23// Make sure fast / cheap clones on Arc are explicit:
24// https://github.com/apache/datafusion/issues/11143
25//
26// Eliminate unnecessary function calls(some may be not cheap) due to `xxx_or`
27// for performance. Also avoid abusing `xxx_or_else` for readability:
28// https://github.com/apache/datafusion/issues/15802
29#![cfg_attr(
30    not(test),
31    deny(
32        clippy::clone_on_ref_ptr,
33        clippy::or_fun_call,
34        clippy::unnecessary_lazy_evaluations
35    )
36)]
37#![warn(missing_docs, clippy::needless_borrow)]
38
39//! [DataFusion] is an extensible query engine written in Rust that
40//! uses [Apache Arrow] as its in-memory format. DataFusion's target users are
41//! developers building fast and feature rich database and analytic systems,
42//! customized to particular workloads. Please see the [DataFusion website] for
43//! additional documentation, [use cases] and examples.
44//!
45//! "Out of the box," DataFusion offers [SQL] and [`Dataframe`] APIs,
46//! excellent [performance], built-in support for CSV, Parquet, JSON, and Avro,
47//! extensive customization, and a great community.
48//! [Python Bindings] are also available.
49//!
50//! DataFusion features a full query planner, a columnar, streaming, multi-threaded,
51//! vectorized execution engine, and partitioned data  sources. You can
52//! customize DataFusion at almost all points including additional data sources,
53//! query languages, functions, custom operators and more.
54//! See the [Architecture] section below for more details.
55//!
56//! [DataFusion]: https://datafusion.apache.org/
57//! [DataFusion website]: https://datafusion.apache.org
58//! [Apache Arrow]: https://arrow.apache.org
59//! [use cases]: https://datafusion.apache.org/user-guide/introduction.html#use-cases
60//! [SQL]: https://datafusion.apache.org/user-guide/sql/index.html
61//! [`DataFrame`]: dataframe::DataFrame
62//! [performance]: https://benchmark.clickhouse.com/
63//! [Python Bindings]: https://github.com/apache/datafusion-python
64//! [Architecture]: #architecture
65//!
66//! # Examples
67//!
68//! The main entry point for interacting with DataFusion is the
69//! [`SessionContext`]. [`Expr`]s represent expressions such as `a + b`.
70//!
71//! [`SessionContext`]: execution::context::SessionContext
72//!
73//! ## DataFrame
74//!
75//! To execute a query against data stored
76//! in a CSV file using a [`DataFrame`]:
77//!
78//! ```rust
79//! # use datafusion::prelude::*;
80//! # use datafusion::error::Result;
81//! # use datafusion::functions_aggregate::expr_fn::min;
82//! # use datafusion::arrow::array::RecordBatch;
83//!
84//! # #[tokio::main]
85//! # async fn main() -> Result<()> {
86//! let ctx = SessionContext::new();
87//!
88//! // create the dataframe
89//! let df = ctx.read_csv("tests/data/example.csv", CsvReadOptions::new()).await?;
90//!
91//! // create a plan
92//! let df = df.filter(col("a").lt_eq(col("b")))?
93//!            .aggregate(vec![col("a")], vec![min(col("b"))])?
94//!            .limit(0, Some(100))?;
95//!
96//! // execute the plan
97//! let results: Vec<RecordBatch> = df.collect().await?;
98//!
99//! // format the results
100//! let pretty_results = arrow::util::pretty::pretty_format_batches(&results)?
101//!    .to_string();
102//!
103//! let expected = vec![
104//!     "+---+----------------+",
105//!     "| a | min(?table?.b) |",
106//!     "+---+----------------+",
107//!     "| 1 | 2              |",
108//!     "+---+----------------+"
109//! ];
110//!
111//! assert_eq!(pretty_results.trim().lines().collect::<Vec<_>>(), expected);
112//! # Ok(())
113//! # }
114//! ```
115//!
116//! ## SQL
117//!
118//! To execute a query against a CSV file using [SQL]:
119//!
120//! ```
121//! # use datafusion::prelude::*;
122//! # use datafusion::error::Result;
123//! # use datafusion::arrow::array::RecordBatch;
124//!
125//! # #[tokio::main]
126//! # async fn main() -> Result<()> {
127//! let ctx = SessionContext::new();
128//!
129//! ctx.register_csv("example", "tests/data/example.csv", CsvReadOptions::new()).await?;
130//!
131//! // create a plan
132//! let df = ctx.sql("SELECT a, MIN(b) FROM example WHERE a <= b GROUP BY a LIMIT 100").await?;
133//!
134//! // execute the plan
135//! let results: Vec<RecordBatch> = df.collect().await?;
136//!
137//! // format the results
138//! let pretty_results = arrow::util::pretty::pretty_format_batches(&results)?
139//!   .to_string();
140//!
141//! let expected = vec![
142//!     "+---+----------------+",
143//!     "| a | min(example.b) |",
144//!     "+---+----------------+",
145//!     "| 1 | 2              |",
146//!     "+---+----------------+"
147//! ];
148//!
149//! assert_eq!(pretty_results.trim().lines().collect::<Vec<_>>(), expected);
150//! # Ok(())
151//! # }
152//! ```
153//!
154//! ## More Examples
155//!
156//! There are many additional annotated examples of using DataFusion in the [datafusion-examples] directory.
157//!
158//! [datafusion-examples]: https://github.com/apache/datafusion/tree/main/datafusion-examples
159//!
160//! # Architecture
161//!
162//! <!-- NOTE: The goal of this section is to provide a high level
163//! overview of how DataFusion is organized and then link to other
164//! sections of the docs with more details -->
165//!
166//! You can find a formal description of DataFusion's architecture in our
167//! [SIGMOD 2024 Paper].
168//!
169//! [SIGMOD 2024 Paper]: https://dl.acm.org/doi/10.1145/3626246.3653368
170//!
171//! ## Design Goals
172//! DataFusion's Architecture Goals are:
173//!
174//! 1. Work β€œout of the box”: Provide a very fast, world class query engine with
175//!    minimal setup or required configuration.
176//!
177//! 2. Customizable everything: All behavior should be customizable by
178//!    implementing traits.
179//!
180//! 3. Architecturally boring πŸ₯±: Follow industrial best practice rather than
181//!    trying cutting edge, but unproven, techniques.
182//!
183//! With these principles, users start with a basic, high-performance engine
184//! and specialize it over time to suit their needs and available engineering
185//! capacity.
186//!
187//! ## Overview  Presentations
188//!
189//! The following presentations offer high level overviews of the
190//! different components and how they interact together.
191//!
192//! - [Apr 2023]: The Apache DataFusion Architecture talks
193//!   - _Query Engine_: [recording](https://youtu.be/NVKujPxwSBA) and [slides](https://docs.google.com/presentation/d/1D3GDVas-8y0sA4c8EOgdCvEjVND4s2E7I6zfs67Y4j8/edit#slide=id.p)
194//!   - _Logical Plan and Expressions_: [recording](https://youtu.be/EzZTLiSJnhY) and [slides](https://docs.google.com/presentation/d/1ypylM3-w60kVDW7Q6S99AHzvlBgciTdjsAfqNP85K30)
195//!   - _Physical Plan and Execution_: [recording](https://youtu.be/2jkWU3_w6z0) and [slides](https://docs.google.com/presentation/d/1cA2WQJ2qg6tx6y4Wf8FH2WVSm9JQ5UgmBWATHdik0hg)
196//! - [July 2022]: DataFusion and Arrow: Supercharge Your Data Analytical Tool with a Rusty Query Engine: [recording](https://www.youtube.com/watch?v=Rii1VTn3seQ) and [slides](https://docs.google.com/presentation/d/1q1bPibvu64k2b7LPi7Yyb0k3gA1BiUYiUbEklqW1Ckc/view#slide=id.g11054eeab4c_0_1165)
197//! - [March 2021]: The DataFusion architecture is described in _Query Engine Design and the Rust-Based DataFusion in Apache Arrow_: [recording](https://www.youtube.com/watch?v=K6eCAVEk4kU) (DataFusion content starts [~ 15 minutes in](https://www.youtube.com/watch?v=K6eCAVEk4kU&t=875s)) and [slides](https://www.slideshare.net/influxdata/influxdb-iox-tech-talks-query-engine-design-and-the-rustbased-datafusion-in-apache-arrow-244161934)
198//! - [February 2021]: How DataFusion is used within the Ballista Project is described in _Ballista: Distributed Compute with Rust and Apache Arrow_: [recording](https://www.youtube.com/watch?v=ZZHQaOap9pQ)
199//!
200//! ## Customization and Extension
201//!
202//! DataFusion is designed to be highly extensible, so you can
203//! start with a working, full featured engine, and then
204//! specialize any behavior for your use case. For example,
205//! some projects may add custom [`ExecutionPlan`] operators, or create their own
206//! query language that directly creates [`LogicalPlan`] rather than using the
207//! built in SQL planner, [`SqlToRel`].
208//!
209//! In order to achieve this, DataFusion supports extension at many points:
210//!
211//! * read from any datasource ([`TableProvider`])
212//! * define your own catalogs, schemas, and table lists ([`catalog`] and [`CatalogProvider`])
213//! * build your own query language or plans ([`LogicalPlanBuilder`])
214//! * declare and use user-defined functions ([`ScalarUDF`], and [`AggregateUDF`], [`WindowUDF`])
215//! * add custom plan rewrite passes ([`AnalyzerRule`], [`OptimizerRule`]  and [`PhysicalOptimizerRule`])
216//! * extend the planner to use user-defined logical and physical nodes ([`QueryPlanner`])
217//!
218//! You can find examples of each of them in the [datafusion-examples] directory.
219//!
220//! [`TableProvider`]: crate::datasource::TableProvider
221//! [`CatalogProvider`]: crate::catalog::CatalogProvider
222//! [`LogicalPlanBuilder`]: datafusion_expr::logical_plan::builder::LogicalPlanBuilder
223//! [`ScalarUDF`]: crate::logical_expr::ScalarUDF
224//! [`AggregateUDF`]: crate::logical_expr::AggregateUDF
225//! [`WindowUDF`]: crate::logical_expr::WindowUDF
226//! [`QueryPlanner`]: execution::context::QueryPlanner
227//! [`OptimizerRule`]: datafusion_optimizer::optimizer::OptimizerRule
228//! [`AnalyzerRule`]:  datafusion_optimizer::analyzer::AnalyzerRule
229//! [`PhysicalOptimizerRule`]: datafusion_physical_optimizer::PhysicalOptimizerRule
230//!
231//! ## Query Planning and Execution Overview
232//!
233//! ### SQL
234//!
235//! ```text
236//!                 Parsed with            SqlToRel creates
237//!                 sqlparser              initial plan
238//! β”Œβ”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”           β”Œβ”€β”€β”€β”€β”€β”€β”€β”€β”€β”             β”Œβ”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”
239//! β”‚   SELECT *    β”‚           β”‚Query {  β”‚             β”‚Project      β”‚
240//! β”‚   FROM ...    │──────────▢│..       │────────────▢│  TableScan  β”‚
241//! β”‚               β”‚           β”‚}        β”‚             β”‚    ...      β”‚
242//! β””β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”˜           β””β”€β”€β”€β”€β”€β”€β”€β”€β”€β”˜             β””β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”˜
243//!
244//!   SQL String                 sqlparser               LogicalPlan
245//!                              AST nodes
246//! ```
247//!
248//! 1. The query string is parsed to an Abstract Syntax Tree (AST)
249//!    [`Statement`] using [sqlparser].
250//!
251//! 2. The AST is converted to a [`LogicalPlan`] and logical expressions
252//!    [`Expr`]s to compute the desired result by [`SqlToRel`]. This phase
253//!    also includes name and type resolution ("binding").
254//!
255//! [`Statement`]: https://docs.rs/sqlparser/latest/sqlparser/ast/enum.Statement.html
256//!
257//! ### DataFrame
258//!
259//! When executing plans using the [`DataFrame`] API, the process is
260//! identical as with SQL, except the DataFrame API builds the
261//! [`LogicalPlan`] directly using [`LogicalPlanBuilder`]. Systems
262//! that have their own custom query languages typically also build
263//! [`LogicalPlan`] directly.
264//!
265//! ### Planning
266//!
267//! ```text
268//!             AnalyzerRules and      PhysicalPlanner          PhysicalOptimizerRules
269//!             OptimizerRules         creates ExecutionPlan    improve performance
270//!             rewrite plan
271//! β”Œβ”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”        β”Œβ”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”      β”Œβ”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”        β”Œβ”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”
272//! β”‚Project      β”‚        β”‚Project(x, y)β”‚      β”‚ProjectExec      β”‚        β”‚ProjectExec      β”‚
273//! β”‚  TableScan  │──...──▢│  TableScan  │─────▢│  ...            │──...──▢│  ...            β”‚
274//! β”‚    ...      β”‚        β”‚    ...      β”‚      β”‚   DataSourceExecβ”‚        β”‚   DataSourceExecβ”‚
275//! β””β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”˜        β””β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”˜      β””β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”˜        β””β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”˜
276//!
277//!  LogicalPlan            LogicalPlan         ExecutionPlan             ExecutionPlan
278//! ```
279//!
280//! To process large datasets with many rows as efficiently as
281//! possible, significant effort is spent planning and
282//! optimizing, in the following manner:
283//!
284//! 1. The [`LogicalPlan`] is checked and rewritten to enforce
285//!    semantic rules, such as type coercion, by [`AnalyzerRule`]s
286//!
287//! 2. The [`LogicalPlan`] is rewritten by [`OptimizerRule`]s, such as
288//!    projection and filter pushdown, to improve its efficiency.
289//!
290//! 3. The [`LogicalPlan`] is converted to an [`ExecutionPlan`] by a
291//!    [`PhysicalPlanner`]
292//!
293//! 4. The [`ExecutionPlan`] is rewritten by
294//!    [`PhysicalOptimizerRule`]s, such as sort and join selection, to
295//!    improve its efficiency.
296//!
297//! ## Data Sources
298//!
299//! ```text
300//! Planning       β”‚
301//! requests       β”‚            TableProvider::scan
302//! information    β”‚            creates an
303//! such as schema β”‚            ExecutionPlan
304//!                β”‚
305//!                β–Ό
306//!   β”Œβ”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”         β”Œβ”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”
307//!   β”‚                         β”‚         β”‚               β”‚
308//!   β”‚impl TableProvider       │────────▢│DataSourceExec β”‚
309//!   β”‚                         β”‚         β”‚               β”‚
310//!   β””β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”˜         β””β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”˜
311//!         TableProvider
312//!         (built in or user provided)    ExecutionPlan
313//! ```
314//!
315//! A [`TableProvider`] provides information for planning and
316//! an [`ExecutionPlan`] for execution. DataFusion includes two built-in
317//! table providers that support common file formats and require no runtime services,
318//! [`ListingTable`] and [`MemTable`]. You can add support for any other data
319//! source and/or file formats by implementing the [`TableProvider`] trait.
320//!
321//! See also:
322//!
323//! 1. [`ListingTable`]: Reads data from one or more Parquet, JSON, CSV, or AVRO
324//!    files in one or more local or remote directories. Supports HIVE style
325//!    partitioning, optional compression, directly reading from remote
326//!    object store, file metadata caching, and more.
327//!
328//! 2. [`MemTable`]: Reads data from in memory [`RecordBatch`]es.
329//!
330//! 3. [`StreamingTable`]: Reads data from potentially unbounded inputs.
331//!
332//! [`ListingTable`]: crate::datasource::listing::ListingTable
333//! [`MemTable`]: crate::datasource::memory::MemTable
334//! [`StreamingTable`]: crate::catalog::streaming::StreamingTable
335//!
336//! ## Plan Representations
337//!
338//! ### Logical Plans
339//! Logical planning yields [`LogicalPlan`] nodes and [`Expr`]
340//! representing expressions which are [`Schema`] aware and represent statements
341//! independent of how they are physically executed.
342//! A [`LogicalPlan`] is a Directed Acyclic Graph (DAG) of other
343//! [`LogicalPlan`]s, each potentially containing embedded [`Expr`]s.
344//!
345//! [`LogicalPlan`]s can be rewritten with [`TreeNode`] API, see the
346//! [`tree_node module`] for more details.
347//!
348//! [`Expr`]s can also be rewritten with [`TreeNode`] API and simplified using
349//! [`ExprSimplifier`]. Examples of working with and executing [`Expr`]s can be
350//! found in the [`expr_api`.rs] example
351//!
352//! [`TreeNode`]: datafusion_common::tree_node::TreeNode
353//! [`tree_node module`]: datafusion_expr::logical_plan::tree_node
354//! [`ExprSimplifier`]: crate::optimizer::simplify_expressions::ExprSimplifier
355//! [`expr_api`.rs]: https://github.com/apache/datafusion/blob/main/datafusion-examples/examples/expr_api.rs
356//!
357//! ### Physical Plans
358//!
359//! An [`ExecutionPlan`] (sometimes referred to as a "physical plan")
360//! is a plan that can be executed against data. It a DAG of other
361//! [`ExecutionPlan`]s each potentially containing expressions that implement the
362//! [`PhysicalExpr`] trait.
363//!
364//! Compared to a [`LogicalPlan`], an [`ExecutionPlan`] has additional concrete
365//! information about how to perform calculations (e.g. hash vs merge
366//! join), and how data flows during execution (e.g. partitioning and
367//! sortedness).
368//!
369//! [cp_solver] performs range propagation analysis on [`PhysicalExpr`]s and
370//! [`PruningPredicate`] can prove certain boolean [`PhysicalExpr`]s used for
371//! filtering can never be `true` using additional statistical information.
372//!
373//! [cp_solver]: crate::physical_expr::intervals::cp_solver
374//! [`PruningPredicate`]: datafusion_physical_optimizer::pruning::PruningPredicate
375//! [`PhysicalExpr`]: crate::physical_plan::PhysicalExpr
376//!
377//! ## Execution
378//!
379//! ```text
380//!            ExecutionPlan::execute             Calling next() on the
381//!            produces a stream                  stream produces the data
382//!
383//! β”Œβ”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”      β”Œβ”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”         β”Œβ”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”
384//! β”‚ProjectExec     β”‚      β”‚impl                     β”‚    β”Œβ”€β”€β”€β–Άβ”‚RecordBatch β”‚
385//! β”‚  ...           │─────▢│SendableRecordBatchStream│─────    β””β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”˜
386//! β”‚  DataSourceExecβ”‚      β”‚                         β”‚    β”‚    β”Œβ”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”
387//! β””β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”˜      β””β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”˜    β”œβ”€β”€β”€β–Άβ”‚RecordBatch β”‚
388//!               β–²                                        β”‚    β””β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”˜
389//! ExecutionPlan β”‚                                        β”‚         ...
390//!               β”‚                                        β”‚
391//!               β”‚                                        β”‚    β”Œβ”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”
392//!             PhysicalOptimizerRules                     β”œβ”€β”€β”€β–Άβ”‚RecordBatch β”‚
393//!             request information                        β”‚    β””β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”˜
394//!             such as partitioning                       β”‚    β”Œ ─ ─ ─ ─ ─ ─
395//!                                                        └───▢ None        β”‚
396//!                                                             β”” ─ ─ ─ ─ ─ ─
397//! ```
398//!
399//! [`ExecutionPlan`]s process data using the [Apache Arrow] memory
400//! format, making heavy use of functions from the [arrow]
401//! crate. Values are represented with [`ColumnarValue`], which are either
402//! [`ScalarValue`] (single constant values) or [`ArrayRef`] (Arrow
403//! Arrays).
404//!
405//! Calling [`execute`] produces 1 or more partitions of data,
406//! as a [`SendableRecordBatchStream`], which implements a pull based execution
407//! API. Calling [`next()`]`.await` will incrementally compute and return the next
408//! [`RecordBatch`]. Balanced parallelism is achieved using [Volcano style]
409//! "Exchange" operations implemented by [`RepartitionExec`].
410//!
411//! While some recent research such as [Morsel-Driven Parallelism] describes challenges
412//! with the pull style Volcano execution model on NUMA architectures, in practice DataFusion achieves
413//! similar scalability as systems that use push driven schedulers [such as DuckDB].
414//! See the [DataFusion paper in SIGMOD 2024] for more details.
415//!
416//! [`execute`]: physical_plan::ExecutionPlan::execute
417//! [`SendableRecordBatchStream`]: crate::physical_plan::SendableRecordBatchStream
418//! [`ColumnarValue`]: datafusion_expr::ColumnarValue
419//! [`ScalarValue`]: crate::scalar::ScalarValue
420//! [`ArrayRef`]: arrow::array::ArrayRef
421//! [`Stream`]: futures::stream::Stream
422//!
423//! See the [implementors of `ExecutionPlan`] for a list of physical operators available.
424//!
425//! [`RepartitionExec`]: https://docs.rs/datafusion/latest/datafusion/physical_plan/repartition/struct.RepartitionExec.html
426//! [Volcano style]: https://doi.org/10.1145/93605.98720
427//! [Morsel-Driven Parallelism]: https://db.in.tum.de/~leis/papers/morsels.pdf
428//! [DataFusion paper in SIGMOD 2024]: https://github.com/apache/datafusion/files/15149988/DataFusion_Query_Engine___SIGMOD_2024-FINAL-mk4.pdf
429//! [such as DuckDB]: https://github.com/duckdb/duckdb/issues/1583
430//! [implementors of `ExecutionPlan`]: https://docs.rs/datafusion/latest/datafusion/physical_plan/trait.ExecutionPlan.html#implementors
431//!
432//! ## Streaming Execution
433//!
434//! DataFusion is a "streaming" query engine which means [`ExecutionPlan`]s incrementally
435//! read from their input(s) and compute output one [`RecordBatch`] at a time
436//! by continually polling [`SendableRecordBatchStream`]s. Output and
437//! intermediate [`RecordBatch`]s each have approximately `batch_size` rows,
438//! which amortizes per-batch overhead of execution.
439//!
440//! Note that certain operations, sometimes called "pipeline breakers",
441//! (for example full sorts or hash aggregations) are fundamentally non streaming and
442//! must read their input fully before producing **any** output. As much as possible,
443//! other operators read a single [`RecordBatch`] from their input to produce a
444//! single [`RecordBatch`] as output.
445//!
446//! For example, given this SQL query:
447//!
448//! ```sql
449//! SELECT date_trunc('month', time) FROM data WHERE id IN (10,20,30);
450//! ```
451//!
452//! The diagram below shows the call sequence when a consumer calls [`next()`] to
453//! get the next [`RecordBatch`] of output. While it is possible that some
454//! steps run on different threads, typically tokio will use the same thread
455//! that called [`next()`] to read from the input, apply the filter, and
456//! return the results without interleaving any other operations. This results
457//! in excellent cache locality as the same CPU core that produces the data often
458//! consumes it immediately as well.
459//!
460//! ```text
461//!
462//! Step 3: FilterExec calls next()       Step 2: ProjectionExec calls
463//!         on input Stream                  next() on input Stream
464//!         β”Œ ─ ─ ─ ─ ─ ─ ─ ─ ─      β”Œ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ┐
465//!                            β”‚                                               Step 1: Consumer
466//!         β–Ό                        β–Ό                           β”‚               calls next()
467//! ┏━━━━━━━━━━━━━━━━┓     ┏━━━━━┻━━━━━━━━━━━━━┓      ┏━━━━━━━━━━━━━━━━━━━━━━━━┓
468//! ┃                ┃     ┃                   ┃      ┃                        β—€ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─
469//! ┃  DataSource    ┃     ┃                   ┃      ┃                        ┃
470//! ┃    (e.g.       ┃     ┃    FilterExec     ┃      ┃     ProjectionExec     ┃
471//! ┃ ParquetSource) ┃     ┃id IN (10, 20, 30) ┃      ┃date_bin('month', time) ┃
472//! ┃                ┃     ┃                   ┃      ┃                        ┣ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ β–Ά
473//! ┃                ┃     ┃                   ┃      ┃                        ┃
474//! ┗━━━━━━━━━━━━━━━━┛     ┗━━━━━━━━━━━┳━━━━━━━┛      ┗━━━━━━━━━━━━━━━━━━━━━━━━┛
475//!         β”‚                  β–²                                 β–²          Step 6: ProjectionExec
476//!                            β”‚     β”‚                           β”‚        computes date_trunc into a
477//!         β”” ─ ─ ─ ─ ─ ─ ─ ─ ─       ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─          new RecordBatch returned
478//!              β”Œβ”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”                β”Œβ”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”          from client
479//!              β”‚     RecordBatch     β”‚                β”‚ RecordBatch β”‚
480//!              β””β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”˜                β””β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”˜
481//!
482//!           Step 4: DataSource returns a        Step 5: FilterExec returns a new
483//!                single RecordBatch            RecordBatch with only matching rows
484//! ```
485//!
486//! [`next()`]: futures::StreamExt::next
487//!
488//! ## Thread Scheduling, CPU / IO Thread Pools, and [Tokio] [`Runtime`]s
489//!
490//! DataFusion automatically runs each plan with multiple CPU cores using
491//! a [Tokio] [`Runtime`] as a thread pool. While tokio is most commonly used
492//! for asynchronous network I/O, the combination of an efficient, work-stealing
493//! scheduler, and first class compiler support for automatic continuation
494//! generation (`async`) also makes it a compelling choice for CPU intensive
495//! applications as explained in the [Using Rustlang’s Async Tokio
496//! Runtime for CPU-Bound Tasks] blog.
497//!
498//! The number of cores used is determined by the `target_partitions`
499//! configuration setting, which defaults to the number of CPU cores.
500//! While preparing for execution, DataFusion tries to create this many distinct
501//! `async` [`Stream`]s for each [`ExecutionPlan`].
502//! The [`Stream`]s for certain [`ExecutionPlan`]s, such as [`RepartitionExec`]
503//! and [`CoalescePartitionsExec`], spawn [Tokio] [`task`]s, that run on
504//! threads managed by the [`Runtime`].
505//! Many DataFusion [`Stream`]s perform CPU intensive processing.
506//!
507//! ### Cooperative Scheduling
508//!
509//! DataFusion uses cooperative scheduling, which means that each [`Stream`]
510//! is responsible for yielding control back to the [`Runtime`] after
511//! some amount of work is done. Please see the [`coop`] module documentation
512//! for more details.
513//!
514//! [`coop`]: datafusion_physical_plan::coop
515//!
516//! ### Network I/O and CPU intensive tasks
517//!
518//! Using `async` for CPU intensive tasks makes it easy for [`TableProvider`]s
519//! to perform network I/O using standard Rust `async` during execution.
520//! However, this design also makes it very easy to mix CPU intensive and latency
521//! sensitive I/O work on the same thread pool ([`Runtime`]).
522//! Using the same (default) [`Runtime`] is convenient, and often works well for
523//! initial development and processing local files, but it can lead to problems
524//! under load and/or when reading from network sources such as AWS S3.
525//!
526//! ### Optimizing Latency: Throttled CPU / IO under Highly Concurrent Load
527//!
528//! If your system does not fully utilize either the CPU or network bandwidth
529//! during execution, or you see significantly higher tail (e.g. p99) latencies
530//! responding to network requests, **it is likely you need to use a different
531//! [`Runtime`] for DataFusion plans**. The [thread_pools example]
532//! has  an example of how to do so.
533//!
534//! As shown below, using the same [`Runtime`] for both CPU intensive processing
535//! and network requests can introduce significant delays in responding to
536//! those network requests. Delays in processing network requests can and does
537//! lead network flow control to throttle the available bandwidth in response.
538//! This effect can be especially pronounced when running multiple queries
539//! concurrently.
540//!
541//! ```text
542//!                                                                          Legend
543//!
544//!                                                                          ┏━━━━━━┓
545//!                            Processing network request                    ┃      ┃  CPU bound work
546//!                            is delayed due to processing                  ┗━━━━━━┛
547//!                            CPU bound work                                β”Œβ”€β”
548//!                                                                          β”‚ β”‚       Network request
549//!                                         β”‚β”‚                               β””β”€β”˜       processing
550//!
551//!                                         β”‚β”‚
552//!                                ─ ─ ─ ─ ─  ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─
553//!                               β”‚                                            β”‚
554//!
555//!                               β–Ό                                            β–Ό
556//! β”Œβ”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”           β”Œβ”€β”β”Œβ”€β”β”β”β”β”β”β”β”β”β”β”β”β”β”β”β”β”β”β”β”β”β”“β”β”β”β”β”β”β”β”β”β”β”β”β”β”β”β”β”β”β”β”β”“β”Œβ”€β”
557//! β”‚             β”‚thread 1   β”‚ β”‚β”‚ │┃     Decoding      ┃┃     Filtering     ┃│ β”‚
558//! β”‚             β”‚           β””β”€β”˜β””β”€β”˜β”—β”β”β”β”β”β”β”β”β”β”β”β”β”β”β”β”β”β”β”β”›β”—β”β”β”β”β”β”β”β”β”β”β”β”β”β”β”β”β”β”β”β”›β””β”€β”˜
559//! β”‚             β”‚           ┏━━━━━━━━━━━━━━┳━━━━━━━━━━━━━━━━━━━┳━━━━━━━━━━━━━━┓
560//! β”‚Tokio Runtimeβ”‚thread 2   ┃   Decoding   ┃     Filtering     ┃   Decoding   ┃       ...
561//! β”‚(thread pool)β”‚           ┗━━━━━━━━━━━━━━┻━━━━━━━━━━━━━━━━━━━┻━━━━━━━━━━━━━━┛
562//! β”‚             β”‚     ...                               ...
563//! β”‚             β”‚           β”β”β”β”β”β”β”β”β”β”β”β”β”β”β”β”β”β”β”β”β”³β”β”β”β”β”β”β”β”β”β”β”β”β”β”β”β”β”β”β”β”“β”Œβ”€β” ┏━━━━━━━━━━━━━━┓
564//! β”‚             β”‚thread N   ┃     Decoding      ┃     Filtering     ┃│ β”‚ ┃   Decoding   ┃
565//! β””β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”˜           β”—β”β”β”β”β”β”β”β”β”β”β”β”β”β”β”β”β”β”β”β”»β”β”β”β”β”β”β”β”β”β”β”β”β”β”β”β”β”β”β”β”›β””β”€β”˜ ┗━━━━━━━━━━━━━━┛
566//!                           ─────────────────────────────────────────────────────────────▢
567//!                                                                                           time
568//! ```
569//!
570//! The bottleneck resulting from network throttling can be avoided
571//! by using separate [`Runtime`]s for the different types of work, as shown
572//! in the diagram below.
573//!
574//! ```text
575//!                    A separate thread pool processes network       Legend
576//!                    requests, reducing the latency for
577//!                    processing each request                        ┏━━━━━━┓
578//!                                                                   ┃      ┃  CPU bound work
579//!                                         β”‚                         ┗━━━━━━┛
580//!                                          β”‚                        β”Œβ”€β”
581//!                               β”Œ ─ ─ ─ ─ β”˜                         β”‚ β”‚       Network request
582//!                                  β”Œ ─ ─ ─ β”˜                        β””β”€β”˜       processing
583//!                               β”‚
584//!                               β–Ό  β–Ό
585//! β”Œβ”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”           β”Œβ”€β”β”Œβ”€β”β”Œβ”€β”
586//! β”‚             β”‚thread 1   β”‚ β”‚β”‚ β”‚β”‚ β”‚
587//! β”‚             β”‚           β””β”€β”˜β””β”€β”˜β””β”€β”˜
588//! β”‚Tokio Runtimeβ”‚                                          ...
589//! β”‚(thread pool)β”‚thread 2
590//! β”‚             β”‚
591//! β”‚"IO Runtime" β”‚     ...
592//! β”‚             β”‚                                                   β”Œβ”€β”
593//! β”‚             β”‚thread N                                           β”‚ β”‚
594//! β””β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”˜                                                   β””β”€β”˜
595//!                           ─────────────────────────────────────────────────────────────▢
596//!                                                                                           time
597//!
598//! β”Œβ”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”           ┏━━━━━━━━━━━━━━━━━━━┓┏━━━━━━━━━━━━━━━━━━━┓
599//! β”‚             β”‚thread 1   ┃     Decoding      ┃┃     Filtering     ┃
600//! β”‚             β”‚           ┗━━━━━━━━━━━━━━━━━━━┛┗━━━━━━━━━━━━━━━━━━━┛
601//! β”‚Tokio Runtimeβ”‚           ┏━━━━━━━━━━━━━━┳━━━━━━━━━━━━━━━━━━━┳━━━━━━━━━━━━━━┓
602//! β”‚(thread pool)β”‚thread 2   ┃   Decoding   ┃     Filtering     ┃   Decoding   ┃       ...
603//! β”‚             β”‚           ┗━━━━━━━━━━━━━━┻━━━━━━━━━━━━━━━━━━━┻━━━━━━━━━━━━━━┛
604//! β”‚ CPU Runtime β”‚     ...                               ...
605//! β”‚             β”‚           ┏━━━━━━━━━━━━━━━━━━━┳━━━━━━━━━━━━━━━━━━━┳━━━━━━━━━━━━━━┓
606//! β”‚             β”‚thread N   ┃     Decoding      ┃     Filtering     ┃   Decoding   ┃
607//! β””β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”€β”˜           ┗━━━━━━━━━━━━━━━━━━━┻━━━━━━━━━━━━━━━━━━━┻━━━━━━━━━━━━━━┛
608//!                          ─────────────────────────────────────────────────────────────▢
609//!                                                                                           time
610//!```
611//!
612//! Note that DataFusion does not use [`tokio::task::spawn_blocking`] for
613//! CPU-bounded work, because `spawn_blocking` is designed for blocking **IO**,
614//! not designed CPU bound tasks. Among other challenges, spawned blocking
615//! tasks can't yield waiting for input (can't call `await`) so they
616//! can't be used to limit the number of concurrent CPU bound tasks or
617//! keep the processing pipeline to the same core.
618//!
619//! [Tokio]:  https://tokio.rs
620//! [`Runtime`]: tokio::runtime::Runtime
621//! [thread_pools example]: https://github.com/apache/datafusion/tree/main/datafusion-examples/examples/thread_pools.rs
622//! [`task`]: tokio::task
623//! [Using Rustlang’s Async Tokio Runtime for CPU-Bound Tasks]: https://thenewstack.io/using-rustlangs-async-tokio-runtime-for-cpu-bound-tasks/
624//! [`RepartitionExec`]: physical_plan::repartition::RepartitionExec
625//! [`CoalescePartitionsExec`]: physical_plan::coalesce_partitions::CoalescePartitionsExec
626//!
627//! ## State Management and Configuration
628//!
629//! [`ConfigOptions`] contain options to control DataFusion's
630//! execution.
631//!
632//! [`ConfigOptions`]: datafusion_common::config::ConfigOptions
633//!
634//! The state required to execute queries is managed by the following
635//! structures:
636//!
637//! 1. [`SessionContext`]: State needed to create [`LogicalPlan`]s such
638//!    as the table definitions and the function registries.
639//!
640//! 2. [`TaskContext`]: State needed for execution such as the
641//!    [`MemoryPool`], [`DiskManager`], and [`ObjectStoreRegistry`].
642//!
643//! 3. [`ExecutionProps`]: Per-execution properties and data (such as
644//!    starting timestamps, etc).
645//!
646//! [`SessionContext`]: crate::execution::context::SessionContext
647//! [`TaskContext`]: crate::execution::context::TaskContext
648//! [`ExecutionProps`]: crate::execution::context::ExecutionProps
649//!
650//! ### Resource Management
651//!
652//! The amount of memory and temporary local disk space used by
653//! DataFusion when running a plan can be controlled using the
654//! [`MemoryPool`] and [`DiskManager`]. Other runtime options can be
655//! found on [`RuntimeEnv`].
656//!
657//! [`DiskManager`]: crate::execution::DiskManager
658//! [`MemoryPool`]: crate::execution::memory_pool::MemoryPool
659//! [`RuntimeEnv`]: crate::execution::runtime_env::RuntimeEnv
660//! [`ObjectStoreRegistry`]: crate::datasource::object_store::ObjectStoreRegistry
661//!
662//! ## Crate Organization
663//!
664//! Most users interact with DataFusion via this crate (`datafusion`), which re-exports
665//! all functionality needed to build and execute queries.
666//!
667//! There are three other crates that provide additional functionality that
668//! must be used directly:
669//! * [`datafusion_proto`]: Plan serialization and deserialization
670//! * [`datafusion_substrait`]: Support for the substrait plan serialization format
671//! * [`datafusion_sqllogictest`] : The DataFusion SQL logic test runner
672//!
673//! [`datafusion_proto`]: https://crates.io/crates/datafusion-proto
674//! [`datafusion_substrait`]: https://crates.io/crates/datafusion-substrait
675//! [`datafusion_sqllogictest`]: https://crates.io/crates/datafusion-sqllogictest
676//!
677//! DataFusion is internally split into multiple sub crates to
678//! enforce modularity and improve compilation times. See the
679//! [list of modules](#modules) for all available sub-crates. Major ones are
680//!
681//! * [datafusion_common]: Common traits and types
682//! * [datafusion_catalog]: Catalog APIs such as [`SchemaProvider`] and [`CatalogProvider`]
683//! * [datafusion_datasource]: File and Data IO such as [`FileSource`] and [`DataSink`]
684//! * [datafusion_session]: [`Session`] and related structures
685//! * [datafusion_execution]: State and structures needed for execution
686//! * [datafusion_expr]: [`LogicalPlan`], [`Expr`] and related logical planning structure
687//! * [datafusion_functions]: Scalar function packages
688//! * [datafusion_functions_aggregate]: Aggregate functions such as `MIN`, `MAX`, `SUM`, etc
689//! * [datafusion_functions_nested]: Scalar function packages for `ARRAY`s, `MAP`s and `STRUCT`s
690//! * [datafusion_functions_table]: Table Functions such as `GENERATE_SERIES`
691//! * [datafusion_functions_window]: Window functions such as `ROW_NUMBER`, `RANK`, etc
692//! * [datafusion_optimizer]: [`OptimizerRule`]s and [`AnalyzerRule`]s
693//! * [datafusion_physical_expr]: [`PhysicalExpr`] and related expressions
694//! * [datafusion_physical_plan]: [`ExecutionPlan`] and related expressions
695//! * [datafusion_physical_optimizer]: [`ExecutionPlan`] and related expressions
696//! * [datafusion_sql]: SQL planner ([`SqlToRel`])
697//!
698//! [`SchemaProvider`]: datafusion_catalog::SchemaProvider
699//! [`CatalogProvider`]: datafusion_catalog::CatalogProvider
700//! [`Session`]: datafusion_session::Session
701//! [`FileSource`]: datafusion_datasource::file::FileSource
702//! [`DataSink`]: datafusion_datasource::sink::DataSink
703//!
704//! ## Citing DataFusion in Academic Papers
705//!
706//! You can use the following citation to reference DataFusion in academic papers:
707//!
708//! ```text
709//! @inproceedings{lamb2024apache
710//!   title={Apache Arrow DataFusion: A Fast, Embeddable, Modular Analytic Query Engine},
711//!   author={Lamb, Andrew and Shen, Yijie and Heres, Dani{\"e}l and Chakraborty, Jayjeet and Kabak, Mehmet Ozan and Hsieh, Liang-Chi and Sun, Chao},
712//!   booktitle={Companion of the 2024 International Conference on Management of Data},
713//!   pages={5--17},
714//!   year={2024}
715//! }
716//! ```
717//!
718//! [sqlparser]: https://docs.rs/sqlparser/latest/sqlparser
719//! [`SqlToRel`]: sql::planner::SqlToRel
720//! [`Expr`]: datafusion_expr::Expr
721//! [`LogicalPlan`]: datafusion_expr::LogicalPlan
722//! [`AnalyzerRule`]: datafusion_optimizer::analyzer::AnalyzerRule
723//! [`OptimizerRule`]: optimizer::optimizer::OptimizerRule
724//! [`ExecutionPlan`]: physical_plan::ExecutionPlan
725//! [`PhysicalPlanner`]: physical_planner::PhysicalPlanner
726//! [`PhysicalOptimizerRule`]: datafusion_physical_optimizer::PhysicalOptimizerRule
727//! [`Schema`]: arrow::datatypes::Schema
728//! [`PhysicalExpr`]: physical_plan::PhysicalExpr
729//! [`RecordBatch`]: arrow::array::RecordBatch
730//! [`RecordBatchReader`]: arrow::record_batch::RecordBatchReader
731//! [`Array`]: arrow::array::Array
732
733/// DataFusion crate version
734pub const DATAFUSION_VERSION: &str = env!("CARGO_PKG_VERSION");
735
736extern crate core;
737extern crate sqlparser;
738
739pub mod dataframe;
740pub mod datasource;
741pub mod error;
742pub mod execution;
743pub mod physical_planner;
744pub mod prelude;
745pub mod scalar;
746
747// Re-export dependencies that are part of DataFusion public API (e.g. via DataFusionError)
748pub use arrow;
749pub use object_store;
750
751#[cfg(feature = "parquet")]
752pub use parquet;
753
754#[cfg(feature = "avro")]
755pub use datafusion_datasource_avro::apache_avro;
756
757// re-export DataFusion sub-crates at the top level. Use `pub use *`
758// so that the contents of the subcrates appears in rustdocs
759// for details, see https://github.com/apache/datafusion/issues/6648
760
761/// re-export of [`datafusion_common`] crate
762pub mod common {
763    pub use datafusion_common::*;
764
765    /// re-export of [`datafusion_common_runtime`] crate
766    pub mod runtime {
767        pub use datafusion_common_runtime::*;
768    }
769}
770
771// Backwards compatibility
772pub use common::config;
773
774// NB datafusion execution is re-exported in the `execution` module
775
776/// re-export of [`datafusion_catalog`] crate
777pub mod catalog {
778    pub use datafusion_catalog::*;
779}
780
781/// re-export of [`datafusion_expr`] crate
782pub mod logical_expr {
783    pub use datafusion_expr::*;
784}
785
786/// re-export of [`datafusion_expr_common`] crate
787pub mod logical_expr_common {
788    pub use datafusion_expr_common::*;
789}
790
791/// re-export of [`datafusion_optimizer`] crate
792pub mod optimizer {
793    pub use datafusion_optimizer::*;
794}
795
796/// re-export of [`datafusion_physical_optimizer`] crate
797pub mod physical_optimizer {
798    pub use datafusion_physical_optimizer::*;
799}
800
801/// re-export of [`datafusion_physical_expr`] crate
802pub mod physical_expr_common {
803    pub use datafusion_physical_expr_common::*;
804}
805
806/// re-export of [`datafusion_physical_expr`] crate
807pub mod physical_expr {
808    pub use datafusion_physical_expr::*;
809}
810
811/// re-export of [`datafusion_physical_expr_adapter`] crate
812pub mod physical_expr_adapter {
813    pub use datafusion_physical_expr_adapter::*;
814}
815
816/// re-export of [`datafusion_physical_plan`] crate
817pub mod physical_plan {
818    pub use datafusion_physical_plan::*;
819}
820
821// Reexport testing macros for compatibility
822pub use datafusion_common::assert_batches_eq;
823pub use datafusion_common::assert_batches_sorted_eq;
824
825/// re-export of [`datafusion_sql`] crate
826pub mod sql {
827    pub use datafusion_sql::*;
828}
829
830/// re-export of [`datafusion_functions`] crate
831pub mod functions {
832    pub use datafusion_functions::*;
833}
834
835/// re-export of [`datafusion_functions_nested`] crate, if "nested_expressions" feature is enabled
836pub mod functions_nested {
837    #[cfg(feature = "nested_expressions")]
838    pub use datafusion_functions_nested::*;
839}
840
841/// re-export of [`datafusion_functions_aggregate`] crate
842pub mod functions_aggregate {
843    pub use datafusion_functions_aggregate::*;
844}
845
846/// re-export of [`datafusion_functions_window`] crate
847pub mod functions_window {
848    pub use datafusion_functions_window::*;
849}
850
851/// re-export of [`datafusion_functions_table`] crate
852pub mod functions_table {
853    pub use datafusion_functions_table::*;
854}
855
856/// re-export of variable provider for `@name` and `@@name` style runtime values.
857pub mod variable {
858    pub use datafusion_expr::var_provider::{VarProvider, VarType};
859}
860
861#[cfg(not(target_arch = "wasm32"))]
862pub mod test;
863
864mod schema_equivalence;
865pub mod test_util;
866
867#[cfg(doctest)]
868doc_comment::doctest!("../../../README.md", readme_example_test);
869
870// Instructions for Documentation Examples
871//
872// The following commands test the examples from the user guide as part of
873// `cargo test --doc`
874//
875// # Adding new tests:
876//
877// Simply add code like this to your .md file and ensure your md file is
878// included in the lists below.
879//
880// ```rust
881// <code here will be tested>
882// ```
883//
884// Note that sometimes it helps to author the doctest as a standalone program
885// first, and then copy it into the user guide.
886//
887// # Debugging Test Failures
888//
889// Unfortunately, the line numbers reported by doctest do not correspond to the
890// line numbers of in the .md files. Thus, if a doctest fails, use the name of
891// the test to find the relevant file in the list below, and then find the
892// example in that file to fix.
893//
894// For example, if `user_guide_expressions(line 123)` fails,
895// go to `docs/source/user-guide/expressions.md` to find the relevant problem.
896//
897#[cfg(doctest)]
898doc_comment::doctest!(
899    "../../../docs/source/user-guide/concepts-readings-events.md",
900    user_guide_concepts_readings_events
901);
902
903#[cfg(doctest)]
904doc_comment::doctest!(
905    "../../../docs/source/user-guide/configs.md",
906    user_guide_configs
907);
908
909#[cfg(doctest)]
910doc_comment::doctest!(
911    "../../../docs/source/user-guide/crate-configuration.md",
912    user_guide_crate_configuration
913);
914
915#[cfg(doctest)]
916doc_comment::doctest!(
917    "../../../docs/source/user-guide/dataframe.md",
918    user_guide_dataframe
919);
920
921#[cfg(doctest)]
922doc_comment::doctest!(
923    "../../../docs/source/user-guide/example-usage.md",
924    user_guide_example_usage
925);
926
927#[cfg(doctest)]
928doc_comment::doctest!(
929    "../../../docs/source/user-guide/explain-usage.md",
930    user_guide_explain_usage
931);
932
933#[cfg(doctest)]
934doc_comment::doctest!(
935    "../../../docs/source/user-guide/expressions.md",
936    user_guide_expressions
937);
938
939#[cfg(doctest)]
940doc_comment::doctest!("../../../docs/source/user-guide/faq.md", user_guide_faq);
941
942#[cfg(doctest)]
943doc_comment::doctest!(
944    "../../../docs/source/user-guide/introduction.md",
945    user_guide_introduction
946);
947
948#[cfg(doctest)]
949doc_comment::doctest!(
950    "../../../docs/source/user-guide/cli/datasources.md",
951    user_guide_cli_datasource
952);
953
954#[cfg(doctest)]
955doc_comment::doctest!(
956    "../../../docs/source/user-guide/cli/installation.md",
957    user_guide_cli_installation
958);
959
960#[cfg(doctest)]
961doc_comment::doctest!(
962    "../../../docs/source/user-guide/cli/overview.md",
963    user_guide_cli_overview
964);
965
966#[cfg(doctest)]
967doc_comment::doctest!(
968    "../../../docs/source/user-guide/cli/usage.md",
969    user_guide_cli_usage
970);
971
972#[cfg(doctest)]
973doc_comment::doctest!(
974    "../../../docs/source/user-guide/features.md",
975    user_guide_features
976);
977
978#[cfg(doctest)]
979doc_comment::doctest!(
980    "../../../docs/source/user-guide/sql/aggregate_functions.md",
981    user_guide_sql_aggregate_functions
982);
983
984#[cfg(doctest)]
985doc_comment::doctest!(
986    "../../../docs/source/user-guide/sql/data_types.md",
987    user_guide_sql_data_types
988);
989
990#[cfg(doctest)]
991doc_comment::doctest!(
992    "../../../docs/source/user-guide/sql/ddl.md",
993    user_guide_sql_ddl
994);
995
996#[cfg(doctest)]
997doc_comment::doctest!(
998    "../../../docs/source/user-guide/sql/dml.md",
999    user_guide_sql_dml
1000);
1001
1002#[cfg(doctest)]
1003doc_comment::doctest!(
1004    "../../../docs/source/user-guide/sql/explain.md",
1005    user_guide_sql_exmplain
1006);
1007
1008#[cfg(doctest)]
1009doc_comment::doctest!(
1010    "../../../docs/source/user-guide/sql/information_schema.md",
1011    user_guide_sql_information_schema
1012);
1013
1014#[cfg(doctest)]
1015doc_comment::doctest!(
1016    "../../../docs/source/user-guide/sql/operators.md",
1017    user_guide_sql_operators
1018);
1019
1020#[cfg(doctest)]
1021doc_comment::doctest!(
1022    "../../../docs/source/user-guide/sql/prepared_statements.md",
1023    user_guide_prepared_statements
1024);
1025
1026#[cfg(doctest)]
1027doc_comment::doctest!(
1028    "../../../docs/source/user-guide/sql/scalar_functions.md",
1029    user_guide_sql_scalar_functions
1030);
1031
1032#[cfg(doctest)]
1033doc_comment::doctest!(
1034    "../../../docs/source/user-guide/sql/select.md",
1035    user_guide_sql_select
1036);
1037
1038#[cfg(doctest)]
1039doc_comment::doctest!(
1040    "../../../docs/source/user-guide/sql/special_functions.md",
1041    user_guide_sql_special_functions
1042);
1043
1044#[cfg(doctest)]
1045doc_comment::doctest!(
1046    "../../../docs/source/user-guide/sql/subqueries.md",
1047    user_guide_sql_subqueries
1048);
1049
1050#[cfg(doctest)]
1051doc_comment::doctest!(
1052    "../../../docs/source/user-guide/sql/window_functions.md",
1053    user_guide_sql_window_functions
1054);
1055
1056#[cfg(doctest)]
1057doc_comment::doctest!(
1058    "../../../docs/source/user-guide/sql/format_options.md",
1059    user_guide_sql_format_options
1060);
1061
1062#[cfg(doctest)]
1063doc_comment::doctest!(
1064    "../../../docs/source/library-user-guide/functions/adding-udfs.md",
1065    library_user_guide_functions_adding_udfs
1066);
1067
1068#[cfg(doctest)]
1069doc_comment::doctest!(
1070    "../../../docs/source/library-user-guide/functions/spark.md",
1071    library_user_guide_functions_spark
1072);
1073
1074#[cfg(doctest)]
1075doc_comment::doctest!(
1076    "../../../docs/source/library-user-guide/building-logical-plans.md",
1077    library_user_guide_building_logical_plans
1078);
1079
1080#[cfg(doctest)]
1081doc_comment::doctest!(
1082    "../../../docs/source/library-user-guide/catalogs.md",
1083    library_user_guide_catalogs
1084);
1085
1086#[cfg(doctest)]
1087doc_comment::doctest!(
1088    "../../../docs/source/library-user-guide/custom-table-providers.md",
1089    library_user_guide_custom_table_providers
1090);
1091
1092#[cfg(doctest)]
1093doc_comment::doctest!(
1094    "../../../docs/source/library-user-guide/extending-operators.md",
1095    library_user_guide_extending_operators
1096);
1097
1098#[cfg(doctest)]
1099doc_comment::doctest!(
1100    "../../../docs/source/library-user-guide/extensions.md",
1101    library_user_guide_extensions
1102);
1103
1104#[cfg(doctest)]
1105doc_comment::doctest!(
1106    "../../../docs/source/library-user-guide/index.md",
1107    library_user_guide_index
1108);
1109
1110#[cfg(doctest)]
1111doc_comment::doctest!(
1112    "../../../docs/source/library-user-guide/profiling.md",
1113    library_user_guide_profiling
1114);
1115
1116#[cfg(doctest)]
1117doc_comment::doctest!(
1118    "../../../docs/source/library-user-guide/query-optimizer.md",
1119    library_user_guide_query_optimizer
1120);
1121
1122#[cfg(doctest)]
1123doc_comment::doctest!(
1124    "../../../docs/source/library-user-guide/using-the-dataframe-api.md",
1125    library_user_guide_dataframe_api
1126);
1127
1128#[cfg(doctest)]
1129doc_comment::doctest!(
1130    "../../../docs/source/library-user-guide/using-the-sql-api.md",
1131    library_user_guide_sql_api
1132);
1133
1134#[cfg(doctest)]
1135doc_comment::doctest!(
1136    "../../../docs/source/library-user-guide/working-with-exprs.md",
1137    library_user_guide_working_with_exprs
1138);
1139
1140#[cfg(doctest)]
1141doc_comment::doctest!(
1142    "../../../docs/source/library-user-guide/upgrading.md",
1143    library_user_guide_upgrading
1144);
1145
1146#[cfg(doctest)]
1147doc_comment::doctest!(
1148    "../../../docs/source/contributor-guide/api-health.md",
1149    contributor_guide_api_health
1150);