forked from apache/arrow
-
Notifications
You must be signed in to change notification settings - Fork 0
/
dataframe_impl.rs
303 lines (251 loc) · 9.5 KB
/
dataframe_impl.rs
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
// 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.
//! Implementation of DataFrame API
use std::sync::Arc;
use crate::arrow::record_batch::RecordBatch;
use crate::dataframe::*;
use crate::error::Result;
use crate::execution::context::{ExecutionContext, ExecutionContextState};
use crate::logical_plan::{col, Expr, FunctionRegistry, LogicalPlan, LogicalPlanBuilder};
use arrow::datatypes::Schema;
/// Implementation of DataFrame API
pub struct DataFrameImpl {
ctx_state: ExecutionContextState,
plan: LogicalPlan,
}
impl DataFrameImpl {
/// Create a new Table based on an existing logical plan
pub fn new(ctx_state: ExecutionContextState, plan: &LogicalPlan) -> Self {
Self {
ctx_state,
plan: plan.clone(),
}
}
}
impl DataFrame for DataFrameImpl {
/// Apply a projection based on a list of column names
fn select_columns(&self, columns: Vec<&str>) -> Result<Arc<dyn DataFrame>> {
let exprs = columns
.iter()
.map(|name| {
self.plan
.schema()
// take the index to ensure that the column exists in the schema
.index_of(name.to_owned())
.and_then(|_| Ok(col(name)))
.map_err(|e| e.into())
})
.collect::<Result<Vec<_>>>()?;
self.select(exprs)
}
/// Create a projection based on arbitrary expressions
fn select(&self, expr_list: Vec<Expr>) -> Result<Arc<dyn DataFrame>> {
let plan = LogicalPlanBuilder::from(&self.plan)
.project(expr_list)?
.build()?;
Ok(Arc::new(DataFrameImpl::new(self.ctx_state.clone(), &plan)))
}
/// Create a filter based on a predicate expression
fn filter(&self, predicate: Expr) -> Result<Arc<dyn DataFrame>> {
let plan = LogicalPlanBuilder::from(&self.plan)
.filter(predicate)?
.build()?;
Ok(Arc::new(DataFrameImpl::new(self.ctx_state.clone(), &plan)))
}
/// Perform an aggregate query
fn aggregate(
&self,
group_expr: Vec<Expr>,
aggr_expr: Vec<Expr>,
) -> Result<Arc<dyn DataFrame>> {
let plan = LogicalPlanBuilder::from(&self.plan)
.aggregate(group_expr, aggr_expr)?
.build()?;
Ok(Arc::new(DataFrameImpl::new(self.ctx_state.clone(), &plan)))
}
/// Limit the number of rows
fn limit(&self, n: usize) -> Result<Arc<dyn DataFrame>> {
let plan = LogicalPlanBuilder::from(&self.plan).limit(n)?.build()?;
Ok(Arc::new(DataFrameImpl::new(self.ctx_state.clone(), &plan)))
}
/// Sort by specified sorting expressions
fn sort(&self, expr: Vec<Expr>) -> Result<Arc<dyn DataFrame>> {
let plan = LogicalPlanBuilder::from(&self.plan).sort(expr)?.build()?;
Ok(Arc::new(DataFrameImpl::new(self.ctx_state.clone(), &plan)))
}
/// Convert to logical plan
fn to_logical_plan(&self) -> LogicalPlan {
self.plan.clone()
}
// Convert the logical plan represented by this DataFrame into a physical plan and
// execute it
fn collect(&self) -> Result<Vec<RecordBatch>> {
let ctx = ExecutionContext::from(self.ctx_state.clone());
let plan = ctx.optimize(&self.plan)?;
let plan = ctx.create_physical_plan(&plan)?;
Ok(ctx.collect(plan)?)
}
/// Returns the schema from the logical plan
fn schema(&self) -> &Schema {
self.plan.schema()
}
fn explain(&self, verbose: bool) -> Result<Arc<dyn DataFrame>> {
let plan = LogicalPlanBuilder::from(&self.plan)
.explain(verbose)?
.build()?;
Ok(Arc::new(DataFrameImpl::new(self.ctx_state.clone(), &plan)))
}
fn registry(&self) -> &dyn FunctionRegistry {
&self.ctx_state
}
}
#[cfg(test)]
mod tests {
use super::*;
use crate::datasource::csv::CsvReadOptions;
use crate::execution::context::ExecutionContext;
use crate::logical_plan::*;
use crate::{physical_plan::functions::ScalarFunctionImplementation, test};
use arrow::{array::ArrayRef, datatypes::DataType};
#[test]
fn select_columns() -> Result<()> {
// build plan using Table API
let t = test_table()?;
let t2 = t.select_columns(vec!["c1", "c2", "c11"])?;
let plan = t2.to_logical_plan();
// build query using SQL
let sql_plan = create_plan("SELECT c1, c2, c11 FROM aggregate_test_100")?;
// the two plans should be identical
assert_same_plan(&plan, &sql_plan);
Ok(())
}
#[test]
fn select_expr() -> Result<()> {
// build plan using Table API
let t = test_table()?;
let t2 = t.select(vec![col("c1"), col("c2"), col("c11")])?;
let plan = t2.to_logical_plan();
// build query using SQL
let sql_plan = create_plan("SELECT c1, c2, c11 FROM aggregate_test_100")?;
// the two plans should be identical
assert_same_plan(&plan, &sql_plan);
Ok(())
}
#[test]
fn aggregate() -> Result<()> {
// build plan using DataFrame API
let df = test_table()?;
let group_expr = vec![col("c1")];
let aggr_expr = vec![
min(col("c12")),
max(col("c12")),
avg(col("c12")),
sum(col("c12")),
count(col("c12")),
];
let df = df.aggregate(group_expr.clone(), aggr_expr.clone())?;
let plan = df.to_logical_plan();
// build same plan using SQL API
let sql = "SELECT c1, MIN(c12), MAX(c12), AVG(c12), SUM(c12), COUNT(c12) \
FROM aggregate_test_100 \
GROUP BY c1";
let sql_plan = create_plan(sql)?;
// the two plans should be identical
assert_same_plan(&plan, &sql_plan);
Ok(())
}
#[test]
fn limit() -> Result<()> {
// build query using Table API
let t = test_table()?;
let t2 = t.select_columns(vec!["c1", "c2", "c11"])?.limit(10)?;
let plan = t2.to_logical_plan();
// build query using SQL
let sql_plan =
create_plan("SELECT c1, c2, c11 FROM aggregate_test_100 LIMIT 10")?;
// the two plans should be identical
assert_same_plan(&plan, &sql_plan);
Ok(())
}
#[test]
fn explain() -> Result<()> {
// build query using Table API
let df = test_table()?;
let df = df
.select_columns(vec!["c1", "c2", "c11"])?
.limit(10)?
.explain(false)?;
let plan = df.to_logical_plan();
// build query using SQL
let sql_plan =
create_plan("EXPLAIN SELECT c1, c2, c11 FROM aggregate_test_100 LIMIT 10")?;
// the two plans should be identical
assert_same_plan(&plan, &sql_plan);
Ok(())
}
#[test]
fn registry() -> Result<()> {
let mut ctx = ExecutionContext::new();
register_aggregate_csv(&mut ctx)?;
// declare the udf
let my_fn: ScalarFunctionImplementation =
Arc::new(|_: &[ArrayRef]| unimplemented!("my_fn is not implemented"));
// create and register the udf
ctx.register_udf(create_udf(
"my_fn",
vec![DataType::Float64],
Arc::new(DataType::Float64),
my_fn,
));
// build query with a UDF using DataFrame API
let df = ctx.table("aggregate_test_100")?;
let f = df.registry();
let df = df.select(vec![f.udf("my_fn", vec![col("c12")])?])?;
let plan = df.to_logical_plan();
// build query using SQL
let sql_plan =
ctx.create_logical_plan("SELECT my_fn(c12) FROM aggregate_test_100")?;
// the two plans should be identical
assert_same_plan(&plan, &sql_plan);
Ok(())
}
/// Compare the formatted string representation of two plans for equality
fn assert_same_plan(plan1: &LogicalPlan, plan2: &LogicalPlan) {
assert_eq!(format!("{:?}", plan1), format!("{:?}", plan2));
}
/// Create a logical plan from a SQL query
fn create_plan(sql: &str) -> Result<LogicalPlan> {
let mut ctx = ExecutionContext::new();
register_aggregate_csv(&mut ctx)?;
ctx.create_logical_plan(sql)
}
fn test_table() -> Result<Arc<dyn DataFrame + 'static>> {
let mut ctx = ExecutionContext::new();
register_aggregate_csv(&mut ctx)?;
ctx.table("aggregate_test_100")
}
fn register_aggregate_csv(ctx: &mut ExecutionContext) -> Result<()> {
let schema = test::aggr_test_schema();
let testdata = test::arrow_testdata_path();
ctx.register_csv(
"aggregate_test_100",
&format!("{}/csv/aggregate_test_100.csv", testdata),
CsvReadOptions::new().schema(&schema),
)?;
Ok(())
}
}