1 2 3 4 5 6 7 8 9 10 11 12 13 14 15 16 17 18 19 20 21 22 23 24 25 26 27 28 29 30 31 32 33 34 35 36 37 38 39 40 41 42 43 44 45 46 47 48 49 50 51 52 53 54 55 56 57 58 59 60 61 62 63 64 65 66 67 68 69 70 71 72 73 74 75 76 77 78 79 80 81 82 83 84 85 86 87 88 89 90 91 92 93 94 95 96 97 98 99 100 101 102 103 104 105 106 107 108 109 110 111 112 113 114 115 116 117 118 119 120 121 122 123 124 125 126 127 128 129 130 131 132 133 134 135 136 137 138 139 140 141 142 143 144 145 146 147 148 149 150 151 152 153 154 155 156 157 158 159 160 161 162 163 164 165 166 167 168 169 170 171 172 173 174 175 176 177 178 179 180 181 182 183 184 185 186 187 188 189 190 191 192 193 194 195 196 197 198 199 200 201 202 203 204 205 206 207 208 209 210 211 212 213 214 215 216 217 218 219 220 221 222 223 224 225 226 227 228 229 230 231 232 233 234 235 236 237 238 239
// 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.
#![warn(missing_docs, clippy::needless_borrow)]
//! [DataFusion](https://github.com/apache/arrow-datafusion)
//! is an extensible query execution framework that uses
//! [Apache Arrow](https://arrow.apache.org) as its in-memory format.
//!
//! DataFusion supports both an SQL and a DataFrame API for building logical query plans
//! as well as a query optimizer and execution engine capable of parallel execution
//! against partitioned data sources (CSV and Parquet) using threads.
//!
//! Below is an example of how to execute a query against data stored
//! in a CSV file using a [`DataFrame`](dataframe::DataFrame):
//!
//! ```rust
//! # use datafusion::prelude::*;
//! # use datafusion::error::Result;
//! # use datafusion::arrow::record_batch::RecordBatch;
//!
//! # #[tokio::main]
//! # async fn main() -> Result<()> {
//! let mut ctx = ExecutionContext::new();
//!
//! // create the dataframe
//! let df = ctx.read_csv("tests/example.csv", CsvReadOptions::new()).await?;
//!
//! // create a plan
//! let df = df.filter(col("a").lt_eq(col("b")))?
//! .aggregate(vec![col("a")], vec![min(col("b"))])?
//! .limit(100)?;
//!
//! // execute the plan
//! let results: Vec<RecordBatch> = df.collect().await?;
//!
//! // format the results
//! let pretty_results = arrow::util::pretty::pretty_format_batches(&results)?
//! .to_string();
//!
//! let expected = vec![
//! "+---+--------------------------+",
//! "| a | MIN(tests/example.csv.b) |",
//! "+---+--------------------------+",
//! "| 1 | 2 |",
//! "+---+--------------------------+"
//! ];
//!
//! assert_eq!(pretty_results.trim().lines().collect::<Vec<_>>(), expected);
//! # Ok(())
//! # }
//! ```
//!
//! and how to execute a query against a CSV using SQL:
//!
//! ```
//! # use datafusion::prelude::*;
//! # use datafusion::error::Result;
//! # use datafusion::arrow::record_batch::RecordBatch;
//!
//! # #[tokio::main]
//! # async fn main() -> Result<()> {
//! let mut ctx = ExecutionContext::new();
//!
//! ctx.register_csv("example", "tests/example.csv", CsvReadOptions::new()).await?;
//!
//! // create a plan
//! let df = ctx.sql("SELECT a, MIN(b) FROM example GROUP BY a LIMIT 100").await?;
//!
//! // execute the plan
//! let results: Vec<RecordBatch> = df.collect().await?;
//!
//! // format the results
//! let pretty_results = arrow::util::pretty::pretty_format_batches(&results)?
//! .to_string();
//!
//! let expected = vec![
//! "+---+----------------+",
//! "| a | MIN(example.b) |",
//! "+---+----------------+",
//! "| 1 | 2 |",
//! "+---+----------------+"
//! ];
//!
//! assert_eq!(pretty_results.trim().lines().collect::<Vec<_>>(), expected);
//! # Ok(())
//! # }
//! ```
//!
//! ## Parse, Plan, Optimize, Execute
//!
//! DataFusion is a fully fledged query engine capable of performing complex operations.
//! Specifically, when DataFusion receives an SQL query, there are different steps
//! that it passes through until a result is obtained. Broadly, they are:
//!
//! 1. The string is parsed to an Abstract syntax tree (AST) using [sqlparser](https://docs.rs/sqlparser/0.6.1/sqlparser/).
//! 2. The planner [`SqlToRel`](sql::planner::SqlToRel) converts logical expressions on the AST to logical expressions [`Expr`s](logical_plan::Expr).
//! 3. The planner [`SqlToRel`](sql::planner::SqlToRel) converts logical nodes on the AST to a [`LogicalPlan`](logical_plan::LogicalPlan).
//! 4. [`OptimizerRules`](optimizer::optimizer::OptimizerRule) are applied to the [`LogicalPlan`](logical_plan::LogicalPlan) to optimize it.
//! 5. The [`LogicalPlan`](logical_plan::LogicalPlan) is converted to an [`ExecutionPlan`](physical_plan::ExecutionPlan) by a [`PhysicalPlanner`](physical_plan::PhysicalPlanner)
//! 6. The [`ExecutionPlan`](physical_plan::ExecutionPlan) is executed against data through the [`ExecutionContext`](execution::context::ExecutionContext)
//!
//! With a [`DataFrame`](dataframe::DataFrame) API, steps 1-3 are not used as the DataFrame builds the [`LogicalPlan`](logical_plan::LogicalPlan) directly.
//!
//! Phases 1-5 are typically cheap when compared to phase 6, and thus DataFusion puts a
//! lot of effort to ensure that phase 6 runs efficiently and without errors.
//!
//! DataFusion's planning is divided in two main parts: logical planning and physical planning.
//!
//! ### Logical plan
//!
//! Logical planning yields [`logical plans`](logical_plan::LogicalPlan) and [`logical expressions`](logical_plan::Expr).
//! These are [`Schema`](arrow::datatypes::Schema)-aware traits that represent statements whose result is independent of how it should physically be executed.
//!
//! A [`LogicalPlan`](logical_plan::LogicalPlan) is a Directed Acyclic Graph (DAG) of other [`LogicalPlan`s](logical_plan::LogicalPlan) and each node contains logical expressions ([`Expr`s](logical_plan::Expr)).
//! All of these are located in [`logical_plan`](logical_plan).
//!
//! ### Physical plan
//!
//! A Physical plan ([`ExecutionPlan`](physical_plan::ExecutionPlan)) is a plan that can be executed against data.
//! Contrarily to a logical plan, the physical plan has concrete information about how the calculation
//! should be performed (e.g. what Rust functions are used) and how data should be loaded into memory.
//!
//! [`ExecutionPlan`](physical_plan::ExecutionPlan) uses the Arrow format as its in-memory representation of data, through the [arrow] crate.
//! We recommend going through [its documentation](arrow) for details on how the data is physically represented.
//!
//! A [`ExecutionPlan`](physical_plan::ExecutionPlan) is composed by nodes (implement the trait [`ExecutionPlan`](physical_plan::ExecutionPlan)),
//! and each node is composed by physical expressions ([`PhysicalExpr`](physical_plan::PhysicalExpr))
//! or aggreagate expressions ([`AggregateExpr`](physical_plan::AggregateExpr)).
//! All of these are located in the module [`physical_plan`](physical_plan).
//!
//! Broadly speaking,
//!
//! * an [`ExecutionPlan`](physical_plan::ExecutionPlan) receives a partition number and asyncronosly returns
//! an iterator over [`RecordBatch`](arrow::record_batch::RecordBatch)
//! (a node-specific struct that implements [`RecordBatchReader`](arrow::record_batch::RecordBatchReader))
//! * a [`PhysicalExpr`](physical_plan::PhysicalExpr) receives a [`RecordBatch`](arrow::record_batch::RecordBatch)
//! and returns an [`Array`](arrow::array::Array)
//! * an [`AggregateExpr`](physical_plan::AggregateExpr) receives [`RecordBatch`es](arrow::record_batch::RecordBatch)
//! and returns a [`RecordBatch`](arrow::record_batch::RecordBatch) of a single row(*)
//!
//! (*) Technically, it aggregates the results on each partition and then merges the results into a single partition.
//!
//! The following physical nodes are currently implemented:
//!
//! * Projection: [`ProjectionExec`](physical_plan::projection::ProjectionExec)
//! * Filter: [`FilterExec`](physical_plan::filter::FilterExec)
//! * Hash and Grouped aggregations: [`HashAggregateExec`](physical_plan::hash_aggregate::HashAggregateExec)
//! * Sort: [`SortExec`](physical_plan::sorts::sort::SortExec)
//! * Coalesce partitions: [`CoalescePartitionsExec`](physical_plan::coalesce_partitions::CoalescePartitionsExec)
//! * Limit: [`LocalLimitExec`](physical_plan::limit::LocalLimitExec) and [`GlobalLimitExec`](physical_plan::limit::GlobalLimitExec)
//! * Scan a CSV: [`CsvExec`](physical_plan::file_format::CsvExec)
//! * Scan a Parquet: [`ParquetExec`](physical_plan::file_format::ParquetExec)
//! * Scan from memory: [`MemoryExec`](physical_plan::memory::MemoryExec)
//! * Explain the plan: [`ExplainExec`](physical_plan::explain::ExplainExec)
//!
//! ## Customize
//!
//! DataFusion allows users to
//! * extend the planner to use user-defined logical and physical nodes ([`QueryPlanner`](execution::context::QueryPlanner))
//! * declare and use user-defined scalar functions ([`ScalarUDF`](physical_plan::udf::ScalarUDF))
//! * declare and use user-defined aggregate functions ([`AggregateUDF`](physical_plan::udaf::AggregateUDF))
//!
//! you can find examples of each of them in examples section.
//!
//! ## Examples
//!
//! Examples are located in [datafusion-examples directory](https://github.com/apache/arrow-datafusion/tree/master/datafusion-examples)
//!
//! Here's how to run them
//!
//! ```bash
//! git clone https://github.com/apache/arrow-datafusion
//! cd arrow-datafusion
//! # Download test data
//! git submodule update --init
//!
//! cargo run --example csv_sql
//!
//! PARQUET_TEST_DATA=./parquet-testing/data cargo run --example parquet_sql
//!
//! cargo run --example dataframe
//!
//! cargo run --example dataframe_in_memory
//!
//! cargo run --example simple_udaf
//!
//! cargo run --example simple_udf
//! ```
/// DataFusion crate version
pub const DATAFUSION_VERSION: &str = env!("CARGO_PKG_VERSION");
extern crate sqlparser;
pub mod avro_to_arrow;
pub mod catalog;
pub mod dataframe;
pub mod datasource;
pub mod error;
pub mod execution;
pub mod logical_plan;
pub mod optimizer;
pub mod physical_optimizer;
pub mod physical_plan;
pub mod prelude;
pub mod scalar;
pub mod sql;
pub mod variable;
// re-export dependencies from arrow-rs to minimise version maintenance for crate users
pub use arrow;
pub use parquet;
pub(crate) mod field_util;
#[cfg(feature = "row")]
pub(crate) mod row;
pub mod from_slice;
#[cfg(test)]
pub mod test;
pub mod test_util;
#[cfg(doctest)]
doc_comment::doctest!("../../README.md", readme_example_test);