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
// 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. //! DataFrame API for building and executing query plans. use crate::arrow::record_batch::RecordBatch; use crate::error::Result; use crate::logical_plan::{Expr, FunctionRegistry, LogicalPlan}; use arrow::datatypes::Schema; use std::sync::Arc; use async_trait::async_trait; /// DataFrame represents a logical set of rows with the same named columns. /// Similar to a [Pandas DataFrame](https://pandas.pydata.org/pandas-docs/stable/reference/api/pandas.DataFrame.html) or /// [Spark DataFrame](https://spark.apache.org/docs/latest/sql-programming-guide.html) /// /// DataFrames are typically created by the `read_csv` and `read_parquet` methods on the /// [ExecutionContext](../execution/context/struct.ExecutionContext.html) and can then be modified /// by calling the transformation methods, such as `filter`, `select`, `aggregate`, and `limit` /// to build up a query definition. /// /// The query can be executed by calling the `collect` method. /// /// ``` /// # use datafusion::prelude::*; /// # use datafusion::error::Result; /// # fn main() -> Result<()> { /// let mut ctx = ExecutionContext::new(); /// let df = ctx.read_csv("tests/example.csv", CsvReadOptions::new())?; /// let df = df.filter(col("a").lt_eq(col("b")))? /// .aggregate(vec![col("a")], vec![min(col("b"))])? /// .limit(100)?; /// let results = df.collect(); /// # Ok(()) /// # } /// ``` #[async_trait] pub trait DataFrame { /// Filter the DataFrame by column. Returns a new DataFrame only containing the /// specified columns. /// /// ``` /// # use datafusion::prelude::*; /// # use datafusion::error::Result; /// # fn main() -> Result<()> { /// let mut ctx = ExecutionContext::new(); /// let df = ctx.read_csv("tests/example.csv", CsvReadOptions::new())?; /// let df = df.select_columns(vec!["a", "b"])?; /// # Ok(()) /// # } /// ``` fn select_columns(&self, columns: Vec<&str>) -> Result<Arc<dyn DataFrame>>; /// Create a projection based on arbitrary expressions. /// /// ``` /// # use datafusion::prelude::*; /// # use datafusion::error::Result; /// # fn main() -> Result<()> { /// let mut ctx = ExecutionContext::new(); /// let df = ctx.read_csv("tests/example.csv", CsvReadOptions::new())?; /// let df = df.select(vec![col("a") * col("b"), col("c")])?; /// # Ok(()) /// # } /// ``` fn select(&self, expr: Vec<Expr>) -> Result<Arc<dyn DataFrame>>; /// Filter a DataFrame to only include rows that match the specified filter expression. /// /// ``` /// # use datafusion::prelude::*; /// # use datafusion::error::Result; /// # fn main() -> Result<()> { /// let mut ctx = ExecutionContext::new(); /// let df = ctx.read_csv("tests/example.csv", CsvReadOptions::new())?; /// let df = df.filter(col("a").lt_eq(col("b")))?; /// # Ok(()) /// # } /// ``` fn filter(&self, expr: Expr) -> Result<Arc<dyn DataFrame>>; /// Perform an aggregate query with optional grouping expressions. /// /// ``` /// # use datafusion::prelude::*; /// # use datafusion::error::Result; /// # fn main() -> Result<()> { /// let mut ctx = ExecutionContext::new(); /// let df = ctx.read_csv("tests/example.csv", CsvReadOptions::new())?; /// /// // The following use is the equivalent of "SELECT MIN(b) GROUP BY a" /// let _ = df.aggregate(vec![col("a")], vec![min(col("b"))])?; /// /// // The following use is the equivalent of "SELECT MIN(b)" /// let _ = df.aggregate(vec![], vec![min(col("b"))])?; /// # Ok(()) /// # } /// ``` fn aggregate( &self, group_expr: Vec<Expr>, aggr_expr: Vec<Expr>, ) -> Result<Arc<dyn DataFrame>>; /// Limit the number of rows returned from this DataFrame. /// /// ``` /// # use datafusion::prelude::*; /// # use datafusion::error::Result; /// # fn main() -> Result<()> { /// let mut ctx = ExecutionContext::new(); /// let df = ctx.read_csv("tests/example.csv", CsvReadOptions::new())?; /// let df = df.limit(100)?; /// # Ok(()) /// # } /// ``` fn limit(&self, n: usize) -> Result<Arc<dyn DataFrame>>; /// Sort the DataFrame by the specified sorting expressions. Any expression can be turned into /// a sort expression by calling its [sort](../logical_plan/enum.Expr.html#method.sort) method. /// /// ``` /// # use datafusion::prelude::*; /// # use datafusion::error::Result; /// # fn main() -> Result<()> { /// let mut ctx = ExecutionContext::new(); /// let df = ctx.read_csv("tests/example.csv", CsvReadOptions::new())?; /// let df = df.sort(vec![col("a").sort(true, true), col("b").sort(false, false)])?; /// # Ok(()) /// # } /// ``` fn sort(&self, expr: Vec<Expr>) -> Result<Arc<dyn DataFrame>>; /// Executes this DataFrame and collects all results into a vector of RecordBatch. /// /// ``` /// # use datafusion::prelude::*; /// # use datafusion::error::Result; /// # #[tokio::main] /// # async fn main() -> Result<()> { /// let mut ctx = ExecutionContext::new(); /// let df = ctx.read_csv("tests/example.csv", CsvReadOptions::new())?; /// let batches = df.collect().await?; /// # Ok(()) /// # } /// ``` async fn collect(&self) -> Result<Vec<RecordBatch>>; /// Returns the schema describing the output of this DataFrame in terms of columns returned, /// where each column has a name, data type, and nullability attribute. /// ``` /// # use datafusion::prelude::*; /// # use datafusion::error::Result; /// # fn main() -> Result<()> { /// let mut ctx = ExecutionContext::new(); /// let df = ctx.read_csv("tests/example.csv", CsvReadOptions::new())?; /// let schema = df.schema(); /// # Ok(()) /// # } /// ``` fn schema(&self) -> &Schema; /// Return the logical plan represented by this DataFrame. fn to_logical_plan(&self) -> LogicalPlan; /// Return a DataFrame with the explanation of its plan so far. /// /// ``` /// # use datafusion::prelude::*; /// # use datafusion::error::Result; /// # #[tokio::main] /// # async fn main() -> Result<()> { /// let mut ctx = ExecutionContext::new(); /// let df = ctx.read_csv("tests/example.csv", CsvReadOptions::new())?; /// let batches = df.limit(100)?.explain(false)?.collect().await?; /// # Ok(()) /// # } /// ``` fn explain(&self, verbose: bool) -> Result<Arc<dyn DataFrame>>; /// Return a `FunctionRegistry` used to plan udf's calls /// /// ``` /// # use datafusion::prelude::*; /// # use datafusion::error::Result; /// # fn main() -> Result<()> { /// let mut ctx = ExecutionContext::new(); /// let df = ctx.read_csv("tests/example.csv", CsvReadOptions::new())?; /// let f = df.registry(); /// // use f.udf("name", vec![...]) to use the udf /// # Ok(()) /// # } /// ``` fn registry(&self) -> &dyn FunctionRegistry; }