Trait datafusion::dataframe::DataFrame
source · [−]pub trait DataFrame: Send + Sync {
Show 22 methods
fn select_columns(&self, columns: &[&str]) -> Result<Arc<dyn DataFrame>>;
fn select(&self, expr: Vec<Expr>) -> Result<Arc<dyn DataFrame>>;
fn filter(&self, expr: Expr) -> Result<Arc<dyn DataFrame>>;
fn aggregate(
&self,
group_expr: Vec<Expr>,
aggr_expr: Vec<Expr>
) -> Result<Arc<dyn DataFrame>>;
fn limit(&self, n: usize) -> Result<Arc<dyn DataFrame>>;
fn union(&self, dataframe: Arc<dyn DataFrame>) -> Result<Arc<dyn DataFrame>>;
fn distinct(&self) -> Result<Arc<dyn DataFrame>>;
fn sort(&self, expr: Vec<Expr>) -> Result<Arc<dyn DataFrame>>;
fn join(
&self,
right: Arc<dyn DataFrame>,
join_type: JoinType,
left_cols: &[&str],
right_cols: &[&str]
) -> Result<Arc<dyn DataFrame>>;
fn repartition(
&self,
partitioning_scheme: Partitioning
) -> Result<Arc<dyn DataFrame>>;
fn collect<'life0, 'async_trait>(
&'life0 self
) -> Pin<Box<dyn Future<Output = Result<Vec<RecordBatch>>> + Send + 'async_trait>>
where
'life0: 'async_trait,
Self: 'async_trait;
fn show<'life0, 'async_trait>(
&'life0 self
) -> Pin<Box<dyn Future<Output = Result<()>> + Send + 'async_trait>>
where
'life0: 'async_trait,
Self: 'async_trait;
fn show_limit<'life0, 'async_trait>(
&'life0 self,
n: usize
) -> Pin<Box<dyn Future<Output = Result<()>> + Send + 'async_trait>>
where
'life0: 'async_trait,
Self: 'async_trait;
fn execute_stream<'life0, 'async_trait>(
&'life0 self
) -> Pin<Box<dyn Future<Output = Result<SendableRecordBatchStream>> + Send + 'async_trait>>
where
'life0: 'async_trait,
Self: 'async_trait;
fn collect_partitioned<'life0, 'async_trait>(
&'life0 self
) -> Pin<Box<dyn Future<Output = Result<Vec<Vec<RecordBatch>>>> + Send + 'async_trait>>
where
'life0: 'async_trait,
Self: 'async_trait;
fn execute_stream_partitioned<'life0, 'async_trait>(
&'life0 self
) -> Pin<Box<dyn Future<Output = Result<Vec<SendableRecordBatchStream>>> + Send + 'async_trait>>
where
'life0: 'async_trait,
Self: 'async_trait;
fn schema(&self) -> &DFSchema;
fn to_logical_plan(&self) -> LogicalPlan;
fn explain(
&self,
verbose: bool,
analyze: bool
) -> Result<Arc<dyn DataFrame>>;
fn registry(&self) -> Arc<dyn FunctionRegistry>;
fn intersect(
&self,
dataframe: Arc<dyn DataFrame>
) -> Result<Arc<dyn DataFrame>>;
fn except(
&self,
dataframe: Arc<dyn DataFrame>
) -> Result<Arc<dyn DataFrame>>;
}
Expand description
DataFrame represents a logical set of rows with the same named columns. Similar to a Pandas DataFrame or Spark DataFrame
DataFrames are typically created by the read_csv
and read_parquet
methods on the
ExecutionContext 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.
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();
Required methods
Filter the DataFrame by column. Returns a new DataFrame only containing the specified columns.
let mut ctx = ExecutionContext::new();
let df = ctx.read_csv("tests/example.csv", CsvReadOptions::new()).await?;
let df = df.select_columns(&["a", "b"])?;
Create a projection based on arbitrary expressions.
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")])?;
Filter a DataFrame to only include rows that match the specified filter expression.
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")))?;
Perform an aggregate query with optional grouping expressions.
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"))])?;
Limit the number of rows returned from this DataFrame.
let mut ctx = ExecutionContext::new();
let df = ctx.read_csv("tests/example.csv", CsvReadOptions::new()).await?;
let df = df.limit(100)?;
Sort the DataFrame by the specified sorting expressions. Any expression can be turned into a sort expression by calling its sort method.
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)])?;
Join this DataFrame with another DataFrame using the specified columns as join keys
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?;
fn repartition(
&self,
partitioning_scheme: Partitioning
) -> Result<Arc<dyn DataFrame>>
fn repartition(
&self,
partitioning_scheme: Partitioning
) -> Result<Arc<dyn DataFrame>>
Repartition a DataFrame based on a logical partitioning scheme.
let mut ctx = ExecutionContext::new();
let df = ctx.read_csv("tests/example.csv", CsvReadOptions::new()).await?;
let df1 = df.repartition(Partitioning::RoundRobinBatch(4))?;
Executes this DataFrame and collects all results into a vector of RecordBatch.
let mut ctx = ExecutionContext::new();
let df = ctx.read_csv("tests/example.csv", CsvReadOptions::new()).await?;
let batches = df.collect().await?;
Print results.
let mut ctx = ExecutionContext::new();
let df = ctx.read_csv("tests/example.csv", CsvReadOptions::new()).await?;
df.show().await?;
Print results and limit rows.
let mut ctx = ExecutionContext::new();
let df = ctx.read_csv("tests/example.csv", CsvReadOptions::new()).await?;
df.show_limit(10).await?;
fn execute_stream<'life0, 'async_trait>(
&'life0 self
) -> Pin<Box<dyn Future<Output = Result<SendableRecordBatchStream>> + Send + 'async_trait>> where
'life0: 'async_trait,
Self: 'async_trait,
fn execute_stream<'life0, 'async_trait>(
&'life0 self
) -> Pin<Box<dyn Future<Output = Result<SendableRecordBatchStream>> + Send + 'async_trait>> where
'life0: 'async_trait,
Self: 'async_trait,
Executes this DataFrame and returns a stream over a single partition
let mut ctx = ExecutionContext::new();
let df = ctx.read_csv("tests/example.csv", CsvReadOptions::new()).await?;
let stream = df.execute_stream().await?;
fn collect_partitioned<'life0, 'async_trait>(
&'life0 self
) -> Pin<Box<dyn Future<Output = Result<Vec<Vec<RecordBatch>>>> + Send + 'async_trait>> where
'life0: 'async_trait,
Self: 'async_trait,
fn collect_partitioned<'life0, 'async_trait>(
&'life0 self
) -> Pin<Box<dyn Future<Output = Result<Vec<Vec<RecordBatch>>>> + Send + 'async_trait>> where
'life0: 'async_trait,
Self: 'async_trait,
Executes this DataFrame and collects all results into a vector of vector of RecordBatch maintaining the input partitioning.
let mut ctx = ExecutionContext::new();
let df = ctx.read_csv("tests/example.csv", CsvReadOptions::new()).await?;
let batches = df.collect_partitioned().await?;
fn execute_stream_partitioned<'life0, 'async_trait>(
&'life0 self
) -> Pin<Box<dyn Future<Output = Result<Vec<SendableRecordBatchStream>>> + Send + 'async_trait>> where
'life0: 'async_trait,
Self: 'async_trait,
fn execute_stream_partitioned<'life0, 'async_trait>(
&'life0 self
) -> Pin<Box<dyn Future<Output = Result<Vec<SendableRecordBatchStream>>> + Send + 'async_trait>> where
'life0: 'async_trait,
Self: 'async_trait,
Executes this DataFrame and returns one stream per partition.
let mut ctx = ExecutionContext::new();
let df = ctx.read_csv("tests/example.csv", CsvReadOptions::new()).await?;
let batches = df.execute_stream_partitioned().await?;
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.
let mut ctx = ExecutionContext::new();
let df = ctx.read_csv("tests/example.csv", CsvReadOptions::new()).await?;
let schema = df.schema();
fn to_logical_plan(&self) -> LogicalPlan
fn to_logical_plan(&self) -> LogicalPlan
Return the logical plan represented by this DataFrame.
Return a DataFrame with the explanation of its plan so far.
if analyze
is specified, runs the plan and reports metrics
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?;
fn registry(&self) -> Arc<dyn FunctionRegistry>
fn registry(&self) -> Arc<dyn FunctionRegistry>
Return a FunctionRegistry
used to plan udf’s calls
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