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 240 241 242 243 244 245 246 247 248 249 250 251 252 253 254 255 256 257 258 259 260 261 262 263 264 265 266 267 268 269 270 271 272 273 274 275 276 277 278 279 280 281 282 283 284 285 286 287 288 289 290 291 292 293 294 295 296 297 298 299 300 301 302 303 304 305 306 307 308 309 310 311 312 313 314 315 316 317 318 319 320 321 322 323 324 325 326 327 328 329 330 331 332 333 334 335 336 337 338 339 340 341 342 343 344 345 346 347 348 349 350 351 352 353 354 355 356 357 358 359 360 361 362 363 364 365 366 367 368 369 370 371 372 373 374 375 376 377 378 379 380 381 382 383 384 385 386 387 388 389 390 391 392 393 394 395 396 397 398 399 400 401 402 403 404 405 406 407 408
// 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::{
DFSchema, Expr, FunctionRegistry, JoinType, LogicalPlan, Partitioning,
};
use std::sync::Arc;
use crate::physical_plan::SendableRecordBatchStream;
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;
/// # #[tokio::main]
/// # async fn main() -> Result<()> {
/// let mut ctx = ExecutionContext::new();
/// let df = ctx.read_csv("tests/example.csv", CsvReadOptions::new()).await?;
/// 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: Send + Sync {
/// Filter the DataFrame by column. Returns a new DataFrame only containing the
/// specified columns.
///
/// ```
/// # 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()).await?;
/// let df = df.select_columns(&["a", "b"])?;
/// # Ok(())
/// # }
/// ```
fn select_columns(&self, columns: &[&str]) -> Result<Arc<dyn DataFrame>>;
/// Create a projection based on arbitrary expressions.
///
/// ```
/// # 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()).await?;
/// 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;
/// # #[tokio::main]
/// # async fn main() -> Result<()> {
/// let mut ctx = ExecutionContext::new();
/// let df = ctx.read_csv("tests/example.csv", CsvReadOptions::new()).await?;
/// 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;
/// # #[tokio::main]
/// # async fn main() -> Result<()> {
/// let mut ctx = ExecutionContext::new();
/// let df = ctx.read_csv("tests/example.csv", CsvReadOptions::new()).await?;
///
/// // 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;
/// # #[tokio::main]
/// # async fn main() -> Result<()> {
/// let mut ctx = ExecutionContext::new();
/// let df = ctx.read_csv("tests/example.csv", CsvReadOptions::new()).await?;
/// let df = df.limit(100)?;
/// # Ok(())
/// # }
/// ```
fn limit(&self, n: usize) -> Result<Arc<dyn DataFrame>>;
/// Calculate the union two [`DataFrame`]s. The two [`DataFrame`]s must have exactly the same schema
///
/// ```
/// # 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()).await?;
/// let df = df.union(df.clone())?;
/// # Ok(())
/// # }
/// ```
fn union(&self, dataframe: Arc<dyn DataFrame>) -> Result<Arc<dyn DataFrame>>;
/// Calculate the union distinct two [`DataFrame`]s. The two [`DataFrame`]s must have exactly the same schema
///
/// ```
/// # 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()).await?;
/// let df = df.union(df.clone())?;
/// let df = df.distinct()?;
/// # Ok(())
/// # }
/// ```
fn distinct(&self) -> 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;
/// # #[tokio::main]
/// # async fn main() -> Result<()> {
/// let mut ctx = ExecutionContext::new();
/// let df = ctx.read_csv("tests/example.csv", CsvReadOptions::new()).await?;
/// 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>>;
/// Join this DataFrame with another DataFrame using the specified columns as join keys
///
/// ```
/// # use datafusion::prelude::*;
/// # use datafusion::error::Result;
/// # #[tokio::main]
/// # async fn main() -> Result<()> {
/// let mut ctx = ExecutionContext::new();
/// let left = ctx.read_csv("tests/example.csv", CsvReadOptions::new()).await?;
/// let right = ctx.read_csv("tests/example.csv", CsvReadOptions::new()).await?
/// .select(vec![
/// col("a").alias("a2"),
/// col("b").alias("b2"),
/// col("c").alias("c2")])?;
/// let join = left.join(right, JoinType::Inner, &["a", "b"], &["a2", "b2"])?;
/// let batches = join.collect().await?;
/// # Ok(())
/// # }
/// ```
fn join(
&self,
right: Arc<dyn DataFrame>,
join_type: JoinType,
left_cols: &[&str],
right_cols: &[&str],
) -> Result<Arc<dyn DataFrame>>;
// TODO: add join_using
/// Repartition a DataFrame based on a logical partitioning scheme.
///
/// ```
/// # 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()).await?;
/// let df1 = df.repartition(Partitioning::RoundRobinBatch(4))?;
/// # Ok(())
/// # }
/// ```
fn repartition(
&self,
partitioning_scheme: Partitioning,
) -> 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()).await?;
/// let batches = df.collect().await?;
/// # Ok(())
/// # }
/// ```
async fn collect(&self) -> Result<Vec<RecordBatch>>;
/// Print results.
///
/// ```
/// # 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()).await?;
/// df.show().await?;
/// # Ok(())
/// # }
/// ```
async fn show(&self) -> Result<()>;
/// Print results and limit rows.
///
/// ```
/// # 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()).await?;
/// df.show_limit(10).await?;
/// # Ok(())
/// # }
/// ```
async fn show_limit(&self, n: usize) -> Result<()>;
/// Executes this DataFrame and returns a stream over a single partition
///
/// ```
/// # 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()).await?;
/// let stream = df.execute_stream().await?;
/// # Ok(())
/// # }
/// ```
async fn execute_stream(&self) -> Result<SendableRecordBatchStream>;
/// Executes this DataFrame and collects all results into a vector of vector of RecordBatch
/// maintaining the input partitioning.
///
/// ```
/// # 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()).await?;
/// let batches = df.collect_partitioned().await?;
/// # Ok(())
/// # }
/// ```
async fn collect_partitioned(&self) -> Result<Vec<Vec<RecordBatch>>>;
/// Executes this DataFrame and returns one stream per partition.
///
/// ```
/// # 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()).await?;
/// let batches = df.execute_stream_partitioned().await?;
/// # Ok(())
/// # }
/// ```
async fn execute_stream_partitioned(&self) -> Result<Vec<SendableRecordBatchStream>>;
/// 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;
/// # #[tokio::main]
/// # async fn main() -> Result<()> {
/// let mut ctx = ExecutionContext::new();
/// let df = ctx.read_csv("tests/example.csv", CsvReadOptions::new()).await?;
/// let schema = df.schema();
/// # Ok(())
/// # }
/// ```
fn schema(&self) -> &DFSchema;
/// 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.
///
/// if `analyze` is specified, runs the plan and reports metrics
///
/// ```
/// # 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()).await?;
/// let batches = df.limit(100)?.explain(false, false)?.collect().await?;
/// # Ok(())
/// # }
/// ```
fn explain(&self, verbose: bool, analyze: bool) -> Result<Arc<dyn DataFrame>>;
/// Return a `FunctionRegistry` used to plan udf's calls
///
/// ```
/// # 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()).await?;
/// let f = df.registry();
/// // use f.udf("name", vec![...]) to use the udf
/// # Ok(())
/// # }
/// ```
fn registry(&self) -> Arc<dyn FunctionRegistry>;
/// Calculate the intersection of two [`DataFrame`]s. The two [`DataFrame`]s must have exactly the same schema
///
/// ```
/// # 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()).await?;
/// let df = df.intersect(df.clone())?;
/// # Ok(())
/// # }
/// ```
fn intersect(&self, dataframe: Arc<dyn DataFrame>) -> Result<Arc<dyn DataFrame>>;
/// Calculate the exception of two [`DataFrame`]s. The two [`DataFrame`]s must have exactly the same schema
///
/// ```
/// # 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()).await?;
/// let df = df.except(df.clone())?;
/// # Ok(())
/// # }
/// ```
fn except(&self, dataframe: Arc<dyn DataFrame>) -> Result<Arc<dyn DataFrame>>;
}