forked from apache/datafusion
-
Notifications
You must be signed in to change notification settings - Fork 1
Expand file tree
/
Copy pathmod.rs
More file actions
3056 lines (2758 loc) · 102 KB
/
mod.rs
File metadata and controls
3056 lines (2758 loc) · 102 KB
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
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
// 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.
#[cfg(feature = "parquet")]
mod parquet;
use std::any::Any;
use std::collections::HashMap;
use std::sync::Arc;
use crate::arrow::record_batch::RecordBatch;
use crate::arrow::util::pretty;
use crate::datasource::{provider_as_source, MemTable, TableProvider};
use crate::error::Result;
use crate::execution::context::{SessionState, TaskContext};
use crate::execution::FunctionRegistry;
use crate::logical_expr::utils::find_window_exprs;
use crate::logical_expr::{
col, Expr, JoinType, LogicalPlan, LogicalPlanBuilder, Partitioning, TableType,
};
use crate::physical_plan::{
collect, collect_partitioned, execute_stream, execute_stream_partitioned,
ExecutionPlan, SendableRecordBatchStream,
};
use crate::prelude::SessionContext;
use arrow::array::{Array, ArrayRef, Int64Array, StringArray};
use arrow::compute::{cast, concat};
use arrow::datatypes::{DataType, Field};
use arrow_schema::{Schema, SchemaRef};
use datafusion_common::config::{CsvOptions, FormatOptions, JsonOptions};
use datafusion_common::{
plan_err, Column, DFSchema, DataFusionError, ParamValues, SchemaError, UnnestOptions,
};
use datafusion_expr::{
avg, count, is_null, max, median, min, stddev, utils::COUNT_STAR_EXPANSION,
TableProviderFilterPushDown, UNNAMED_TABLE,
};
use async_trait::async_trait;
/// Contains options that control how data is
/// written out from a DataFrame
pub struct DataFrameWriteOptions {
/// Controls if existing data should be overwritten
overwrite: bool,
/// Controls if all partitions should be coalesced into a single output file
/// Generally will have slower performance when set to true.
single_file_output: bool,
/// Sets which columns should be used for hive-style partitioned writes by name.
/// Can be set to empty vec![] for non-partitioned writes.
partition_by: Vec<String>,
}
impl DataFrameWriteOptions {
/// Create a new DataFrameWriteOptions with default values
pub fn new() -> Self {
DataFrameWriteOptions {
overwrite: false,
single_file_output: false,
partition_by: vec![],
}
}
/// Set the overwrite option to true or false
pub fn with_overwrite(mut self, overwrite: bool) -> Self {
self.overwrite = overwrite;
self
}
/// Set the single_file_output value to true or false
pub fn with_single_file_output(mut self, single_file_output: bool) -> Self {
self.single_file_output = single_file_output;
self
}
/// Sets the partition_by columns for output partitioning
pub fn with_partition_by(mut self, partition_by: Vec<String>) -> Self {
self.partition_by = partition_by;
self
}
}
impl Default for DataFrameWriteOptions {
fn default() -> Self {
Self::new()
}
}
/// Represents a logical set of rows with the same named columns.
///
/// Similar to a [Pandas DataFrame] or [Spark DataFrame], a DataFusion DataFrame
/// represents a 2 dimensional table of rows and columns.
///
/// The typical workflow using DataFrames looks like
///
/// 1. Create a DataFrame via methods on [SessionContext], such as [`read_csv`]
/// and [`read_parquet`].
///
/// 2. Build a desired calculation by calling methods such as [`filter`],
/// [`select`], [`aggregate`], and [`limit`]
///
/// 3. Execute into [`RecordBatch`]es by calling [`collect`]
///
/// A `DataFrame` is a wrapper around a [`LogicalPlan`] and the [`SessionState`]
/// required for execution.
///
/// DataFrames are "lazy" in the sense that most methods do not actually compute
/// anything, they just build up a plan. Calling [`collect`] executes the plan
/// using the same DataFusion planning and execution process used to execute SQL
/// and other queries.
///
/// [Pandas DataFrame]: https://pandas.pydata.org/pandas-docs/stable/reference/api/pandas.DataFrame.html
/// [Spark DataFrame]: https://spark.apache.org/docs/latest/sql-programming-guide.html
/// [`read_csv`]: SessionContext::read_csv
/// [`read_parquet`]: SessionContext::read_parquet
/// [`filter`]: DataFrame::filter
/// [`select`]: DataFrame::select
/// [`aggregate`]: DataFrame::aggregate
/// [`limit`]: DataFrame::limit
/// [`collect`]: DataFrame::collect
///
/// # Example
/// ```
/// # use datafusion::prelude::*;
/// # use datafusion::error::Result;
/// # #[tokio::main]
/// # async fn main() -> Result<()> {
/// let ctx = SessionContext::new();
/// // Read the data from a csv file
/// let df = ctx.read_csv("tests/data/example.csv", CsvReadOptions::new()).await?;
/// // create a new dataframe that computes the equivalent of
/// // `SELECT a, MIN(b) FROM df WHERE a <= b GROUP BY a LIMIT 100;`
/// let df = df.filter(col("a").lt_eq(col("b")))?
/// .aggregate(vec![col("a")], vec![min(col("b"))])?
/// .limit(0, Some(100))?;
/// // Perform the actual computation
/// let results = df.collect();
/// # Ok(())
/// # }
/// ```
#[derive(Debug, Clone)]
pub struct DataFrame {
// Box the (large) SessionState to reduce the size of DataFrame on the stack
session_state: Box<SessionState>,
plan: LogicalPlan,
}
impl DataFrame {
/// Create a new `DataFrame ` based on an existing `LogicalPlan`
///
/// This is a low-level method and is not typically used by end users. See
/// [`SessionContext::read_csv`] and other methods for creating a
/// `DataFrame` from an existing datasource.
pub fn new(session_state: SessionState, plan: LogicalPlan) -> Self {
Self {
session_state: Box::new(session_state),
plan,
}
}
/// Consume the DataFrame and produce a physical plan
pub async fn create_physical_plan(self) -> Result<Arc<dyn ExecutionPlan>> {
self.session_state.create_physical_plan(&self.plan).await
}
/// 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 ctx = SessionContext::new();
/// let df = ctx.read_csv("tests/data/example.csv", CsvReadOptions::new()).await?;
/// let df = df.select_columns(&["a", "b"])?;
/// # Ok(())
/// # }
/// ```
pub fn select_columns(self, columns: &[&str]) -> Result<DataFrame> {
let fields = columns
.iter()
.map(|name| {
self.plan
.schema()
.qualified_field_with_unqualified_name(name)
})
.collect::<Result<Vec<_>>>()?;
let expr: Vec<Expr> = fields
.into_iter()
.map(|(qualifier, field)| Expr::Column(Column::from((qualifier, field))))
.collect();
self.select(expr)
}
/// Project arbitrary expressions (like SQL SELECT expressions) into a new
/// `DataFrame`.
///
/// The output `DataFrame` has one column for each element in `expr_list`.
///
/// # Example
/// ```
/// # use datafusion::prelude::*;
/// # use datafusion::error::Result;
/// # #[tokio::main]
/// # async fn main() -> Result<()> {
/// let ctx = SessionContext::new();
/// let df = ctx.read_csv("tests/data/example.csv", CsvReadOptions::new()).await?;
/// let df = df.select(vec![col("a") * col("b"), col("c")])?;
/// # Ok(())
/// # }
/// ```
pub fn select(self, expr_list: Vec<Expr>) -> Result<DataFrame> {
let window_func_exprs = find_window_exprs(&expr_list);
let plan = if window_func_exprs.is_empty() {
self.plan
} else {
LogicalPlanBuilder::window_plan(self.plan, window_func_exprs)?
};
let project_plan = LogicalPlanBuilder::from(plan).project(expr_list)?.build()?;
Ok(DataFrame {
session_state: self.session_state,
plan: project_plan,
})
}
/// Expand each list element of a column to multiple rows.
#[deprecated(since = "37.0.0", note = "use unnest_columns instead")]
pub fn unnest_column(self, column: &str) -> Result<DataFrame> {
self.unnest_columns(&[column])
}
/// Expand each list element of a column to multiple rows, with
/// behavior controlled by [`UnnestOptions`].
///
/// Please see the documentation on [`UnnestOptions`] for more
/// details about the meaning of unnest.
#[deprecated(since = "37.0.0", note = "use unnest_columns_with_options instead")]
pub fn unnest_column_with_options(
self,
column: &str,
options: UnnestOptions,
) -> Result<DataFrame> {
self.unnest_columns_with_options(&[column], options)
}
/// Expand multiple list columns into a set of rows.
///
/// See also:
///
/// 1. [`UnnestOptions`] documentation for the behavior of `unnest`
/// 2. [`Self::unnest_column_with_options`]
///
/// # Example
/// ```
/// # use datafusion::prelude::*;
/// # use datafusion::error::Result;
/// # #[tokio::main]
/// # async fn main() -> Result<()> {
/// let ctx = SessionContext::new();
/// let df = ctx.read_csv("tests/data/example.csv", CsvReadOptions::new()).await?;
/// let df = df.unnest_columns(&["a", "b"])?;
/// # Ok(())
/// # }
/// ```
pub fn unnest_columns(self, columns: &[&str]) -> Result<DataFrame> {
self.unnest_columns_with_options(columns, UnnestOptions::new())
}
/// Expand multiple list columns into a set of rows, with
/// behavior controlled by [`UnnestOptions`].
///
/// Please see the documentation on [`UnnestOptions`] for more
/// details about the meaning of unnest.
pub fn unnest_columns_with_options(
self,
columns: &[&str],
options: UnnestOptions,
) -> Result<DataFrame> {
let columns = columns.iter().map(|c| Column::from(*c)).collect();
let plan = LogicalPlanBuilder::from(self.plan)
.unnest_columns_with_options(columns, options)?
.build()?;
Ok(DataFrame {
session_state: self.session_state,
plan,
})
}
/// Return a DataFrame with only rows for which `predicate` evaluates to
/// `true`.
///
/// Rows for which `predicate` evaluates to `false` or `null`
/// are filtered out.
///
/// # Example
/// ```
/// # use datafusion::prelude::*;
/// # use datafusion::error::Result;
/// # #[tokio::main]
/// # async fn main() -> Result<()> {
/// let ctx = SessionContext::new();
/// let df = ctx.read_csv("tests/data/example.csv", CsvReadOptions::new()).await?;
/// let df = df.filter(col("a").lt_eq(col("b")))?;
/// # Ok(())
/// # }
/// ```
pub fn filter(self, predicate: Expr) -> Result<DataFrame> {
let plan = LogicalPlanBuilder::from(self.plan)
.filter(predicate)?
.build()?;
Ok(DataFrame {
session_state: self.session_state,
plan,
})
}
/// Return a new `DataFrame` that aggregates the rows of the current
/// `DataFrame`, first optionally grouping by the given expressions.
///
/// # Example
/// ```
/// # use datafusion::prelude::*;
/// # use datafusion::error::Result;
/// # #[tokio::main]
/// # async fn main() -> Result<()> {
/// let ctx = SessionContext::new();
/// let df = ctx.read_csv("tests/data/example.csv", CsvReadOptions::new()).await?;
///
/// // The following use is the equivalent of "SELECT MIN(b) GROUP BY a"
/// let _ = df.clone().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(())
/// # }
/// ```
pub fn aggregate(
self,
group_expr: Vec<Expr>,
aggr_expr: Vec<Expr>,
) -> Result<DataFrame> {
let plan = LogicalPlanBuilder::from(self.plan)
.aggregate(group_expr, aggr_expr)?
.build()?;
Ok(DataFrame {
session_state: self.session_state,
plan,
})
}
/// Return a new DataFrame that adds the result of evaluating one or more
/// window functions ([`Expr::WindowFunction`]) to the existing columns
pub fn window(self, window_exprs: Vec<Expr>) -> Result<DataFrame> {
let plan = LogicalPlanBuilder::from(self.plan)
.window(window_exprs)?
.build()?;
Ok(DataFrame {
session_state: self.session_state,
plan,
})
}
/// Returns a new `DataFrame` with a limited number of rows.
///
/// # Arguments
/// `skip` - Number of rows to skip before fetch any row
/// `fetch` - Maximum number of rows to return, after skipping `skip` rows.
///
/// # Example
/// ```
/// # use datafusion::prelude::*;
/// # use datafusion::error::Result;
/// # #[tokio::main]
/// # async fn main() -> Result<()> {
/// let ctx = SessionContext::new();
/// let df = ctx.read_csv("tests/data/example.csv", CsvReadOptions::new()).await?;
/// let df = df.limit(0, Some(100))?;
/// # Ok(())
/// # }
/// ```
pub fn limit(self, skip: usize, fetch: Option<usize>) -> Result<DataFrame> {
let plan = LogicalPlanBuilder::from(self.plan)
.limit(skip, fetch)?
.build()?;
Ok(DataFrame {
session_state: self.session_state,
plan,
})
}
/// Calculate the union of two [`DataFrame`]s, preserving duplicate rows.
///
/// The two [`DataFrame`]s must have exactly the same schema
///
/// # Example
/// ```
/// # use datafusion::prelude::*;
/// # use datafusion::error::Result;
/// # #[tokio::main]
/// # async fn main() -> Result<()> {
/// let ctx = SessionContext::new();
/// let df = ctx.read_csv("tests/data/example.csv", CsvReadOptions::new()).await?;
/// let d2 = df.clone();
/// let df = df.union(d2)?;
/// # Ok(())
/// # }
/// ```
pub fn union(self, dataframe: DataFrame) -> Result<DataFrame> {
let plan = LogicalPlanBuilder::from(self.plan)
.union(dataframe.plan)?
.build()?;
Ok(DataFrame {
session_state: self.session_state,
plan,
})
}
/// Calculate the distinct union of two [`DataFrame`]s.
///
/// The two [`DataFrame`]s must have exactly the same schema. Any duplicate
/// rows are discarded.
///
/// # Example
/// ```
/// # use datafusion::prelude::*;
/// # use datafusion::error::Result;
/// # #[tokio::main]
/// # async fn main() -> Result<()> {
/// let ctx = SessionContext::new();
/// let df = ctx.read_csv("tests/data/example.csv", CsvReadOptions::new()).await?;
/// let d2 = df.clone();
/// let df = df.union_distinct(d2)?;
/// # Ok(())
/// # }
/// ```
pub fn union_distinct(self, dataframe: DataFrame) -> Result<DataFrame> {
let plan = LogicalPlanBuilder::from(self.plan)
.union_distinct(dataframe.plan)?
.build()?;
Ok(DataFrame {
session_state: self.session_state,
plan,
})
}
/// Return a new `DataFrame` with all duplicated rows removed.
///
/// # Example
/// ```
/// # use datafusion::prelude::*;
/// # use datafusion::error::Result;
/// # #[tokio::main]
/// # async fn main() -> Result<()> {
/// let ctx = SessionContext::new();
/// let df = ctx.read_csv("tests/data/example.csv", CsvReadOptions::new()).await?;
/// let df = df.distinct()?;
/// # Ok(())
/// # }
/// ```
pub fn distinct(self) -> Result<DataFrame> {
let plan = LogicalPlanBuilder::from(self.plan).distinct()?.build()?;
Ok(DataFrame {
session_state: self.session_state,
plan,
})
}
/// Return a new `DataFrame` that has statistics for a DataFrame.
///
/// Only summarizes numeric datatypes at the moment and returns nulls for
/// non numeric datatypes. The output format is modeled after pandas
///
/// # Example
/// ```
/// # use datafusion::prelude::*;
/// # use datafusion::error::Result;
/// # use arrow::util::pretty;
/// # #[tokio::main]
/// # async fn main() -> Result<()> {
/// let ctx = SessionContext::new();
/// let df = ctx.read_csv("tests/tpch-csv/customer.csv", CsvReadOptions::new()).await?;
/// df.describe().await.unwrap();
///
/// # Ok(())
/// # }
/// ```
pub async fn describe(self) -> Result<Self> {
//the functions now supported
let supported_describe_functions =
vec!["count", "null_count", "mean", "std", "min", "max", "median"];
let original_schema_fields = self.schema().fields().iter();
//define describe column
let mut describe_schemas = vec![Field::new("describe", DataType::Utf8, false)];
describe_schemas.extend(original_schema_fields.clone().map(|field| {
if field.data_type().is_numeric() {
Field::new(field.name(), DataType::Float64, true)
} else {
Field::new(field.name(), DataType::Utf8, true)
}
}));
//collect recordBatch
let describe_record_batch = vec![
// count aggregation
self.clone().aggregate(
vec![],
original_schema_fields
.clone()
.map(|f| count(col(f.name())).alias(f.name()))
.collect::<Vec<_>>(),
),
// null_count aggregation
self.clone().aggregate(
vec![],
original_schema_fields
.clone()
.map(|f| count(is_null(col(f.name()))).alias(f.name()))
.collect::<Vec<_>>(),
),
// mean aggregation
self.clone().aggregate(
vec![],
original_schema_fields
.clone()
.filter(|f| f.data_type().is_numeric())
.map(|f| avg(col(f.name())).alias(f.name()))
.collect::<Vec<_>>(),
),
// std aggregation
self.clone().aggregate(
vec![],
original_schema_fields
.clone()
.filter(|f| f.data_type().is_numeric())
.map(|f| stddev(col(f.name())).alias(f.name()))
.collect::<Vec<_>>(),
),
// min aggregation
self.clone().aggregate(
vec![],
original_schema_fields
.clone()
.filter(|f| {
!matches!(f.data_type(), DataType::Binary | DataType::Boolean)
})
.map(|f| min(col(f.name())).alias(f.name()))
.collect::<Vec<_>>(),
),
// max aggregation
self.clone().aggregate(
vec![],
original_schema_fields
.clone()
.filter(|f| {
!matches!(f.data_type(), DataType::Binary | DataType::Boolean)
})
.map(|f| max(col(f.name())).alias(f.name()))
.collect::<Vec<_>>(),
),
// median aggregation
self.clone().aggregate(
vec![],
original_schema_fields
.clone()
.filter(|f| f.data_type().is_numeric())
.map(|f| median(col(f.name())).alias(f.name()))
.collect::<Vec<_>>(),
),
];
// first column with function names
let mut array_ref_vec: Vec<ArrayRef> = vec![Arc::new(StringArray::from(
supported_describe_functions.clone(),
))];
for field in original_schema_fields {
let mut array_datas = vec![];
for result in describe_record_batch.iter() {
let array_ref = match result {
Ok(df) => {
let batchs = df.clone().collect().await;
match batchs {
Ok(batchs)
if batchs.len() == 1
&& batchs[0]
.column_by_name(field.name())
.is_some() =>
{
let column =
batchs[0].column_by_name(field.name()).unwrap();
if field.data_type().is_numeric() {
cast(column, &DataType::Float64)?
} else {
cast(column, &DataType::Utf8)?
}
}
_ => Arc::new(StringArray::from(vec!["null"])),
}
}
//Handling error when only boolean/binary column, and in other cases
Err(err)
if err.to_string().contains(
"Error during planning: \
Aggregate requires at least one grouping \
or aggregate expression",
) =>
{
Arc::new(StringArray::from(vec!["null"]))
}
Err(other_err) => {
panic!("{other_err}")
}
};
array_datas.push(array_ref);
}
array_ref_vec.push(concat(
array_datas
.iter()
.map(|af| af.as_ref())
.collect::<Vec<_>>()
.as_slice(),
)?);
}
let describe_record_batch =
RecordBatch::try_new(Arc::new(Schema::new(describe_schemas)), array_ref_vec)?;
let provider = MemTable::try_new(
describe_record_batch.schema(),
vec![vec![describe_record_batch]],
)?;
let plan = LogicalPlanBuilder::scan(
UNNAMED_TABLE,
provider_as_source(Arc::new(provider)),
None,
)?
.build()?;
Ok(DataFrame {
session_state: self.session_state,
plan,
})
}
/// Sort the DataFrame by the specified sorting expressions.
///
/// Note that any expression can be turned into
/// a sort expression by calling its [sort](Expr::sort) method.
///
/// # Example
///
/// ```
/// # use datafusion::prelude::*;
/// # use datafusion::error::Result;
/// # #[tokio::main]
/// # async fn main() -> Result<()> {
/// let ctx = SessionContext::new();
/// let df = ctx.read_csv("tests/data/example.csv", CsvReadOptions::new()).await?;
/// let df = df.sort(vec![
/// col("a").sort(true, true), // a ASC, nulls first
/// col("b").sort(false, false), // b DESC, nulls last
/// ])?;
/// # Ok(())
/// # }
/// ```
pub fn sort(self, expr: Vec<Expr>) -> Result<DataFrame> {
let plan = LogicalPlanBuilder::from(self.plan).sort(expr)?.build()?;
Ok(DataFrame {
session_state: self.session_state,
plan,
})
}
/// Join this `DataFrame` with another `DataFrame` using explicitly specified
/// columns and an optional filter expression.
///
/// See [`join_on`](Self::join_on) for a more concise way to specify the
/// join condition. Since DataFusion will automatically identify and
/// optimize equality predicates there is no performance difference between
/// this function and `join_on`
///
/// `left_cols` and `right_cols` are used to form "equijoin" predicates (see
/// example below), which are then combined with the optional `filter`
/// expression.
///
/// Note that in case of outer join, the `filter` is applied to only matched rows.
///
/// # Example
/// ```
/// # use datafusion::prelude::*;
/// # use datafusion::error::Result;
/// # #[tokio::main]
/// # async fn main() -> Result<()> {
/// let ctx = SessionContext::new();
/// let left = ctx.read_csv("tests/data/example.csv", CsvReadOptions::new()).await?;
/// let right = ctx.read_csv("tests/data/example.csv", CsvReadOptions::new()).await?
/// .select(vec![
/// col("a").alias("a2"),
/// col("b").alias("b2"),
/// col("c").alias("c2")])?;
/// // Perform the equivalent of `left INNER JOIN right ON (a = a2 AND b = b2)`
/// // finding all pairs of rows from `left` and `right` where `a = a2` and `b = b2`.
/// let join = left.join(right, JoinType::Inner, &["a", "b"], &["a2", "b2"], None)?;
/// let batches = join.collect().await?;
/// # Ok(())
/// # }
/// ```
///
pub fn join(
self,
right: DataFrame,
join_type: JoinType,
left_cols: &[&str],
right_cols: &[&str],
filter: Option<Expr>,
) -> Result<DataFrame> {
let plan = LogicalPlanBuilder::from(self.plan)
.join(
right.plan,
join_type,
(left_cols.to_vec(), right_cols.to_vec()),
filter,
)?
.build()?;
Ok(DataFrame {
session_state: self.session_state,
plan,
})
}
/// Join this `DataFrame` with another `DataFrame` using the specified
/// expressions.
///
/// Note that DataFusion automatically optimizes joins, including
/// identifying and optimizing equality predicates.
///
/// # Example
/// ```
/// # use datafusion::prelude::*;
/// # use datafusion::error::Result;
/// # #[tokio::main]
/// # async fn main() -> Result<()> {
/// let ctx = SessionContext::new();
/// let left = ctx
/// .read_csv("tests/data/example.csv", CsvReadOptions::new())
/// .await?;
/// let right = ctx
/// .read_csv("tests/data/example.csv", CsvReadOptions::new())
/// .await?
/// .select(vec![
/// col("a").alias("a2"),
/// col("b").alias("b2"),
/// col("c").alias("c2"),
/// ])?;
///
/// // Perform the equivalent of `left INNER JOIN right ON (a != a2 AND b != b2)`
/// // finding all pairs of rows from `left` and `right` where
/// // where `a != a2` and `b != b2`.
/// let join_on = left.join_on(
/// right,
/// JoinType::Inner,
/// [col("a").not_eq(col("a2")), col("b").not_eq(col("b2"))],
/// )?;
/// let batches = join_on.collect().await?;
/// # Ok(())
/// # }
/// ```
pub fn join_on(
self,
right: DataFrame,
join_type: JoinType,
on_exprs: impl IntoIterator<Item = Expr>,
) -> Result<DataFrame> {
let expr = on_exprs.into_iter().reduce(Expr::and);
let plan = LogicalPlanBuilder::from(self.plan)
.join_on(right.plan, join_type, expr)?
.build()?;
Ok(DataFrame {
session_state: self.session_state,
plan,
})
}
/// Repartition a DataFrame based on a logical partitioning scheme.
///
/// # Example
/// ```
/// # use datafusion::prelude::*;
/// # use datafusion::error::Result;
/// # #[tokio::main]
/// # async fn main() -> Result<()> {
/// let ctx = SessionContext::new();
/// let df = ctx.read_csv("tests/data/example.csv", CsvReadOptions::new()).await?;
/// let df1 = df.repartition(Partitioning::RoundRobinBatch(4))?;
/// # Ok(())
/// # }
/// ```
pub fn repartition(self, partitioning_scheme: Partitioning) -> Result<DataFrame> {
let plan = LogicalPlanBuilder::from(self.plan)
.repartition(partitioning_scheme)?
.build()?;
Ok(DataFrame {
session_state: self.session_state,
plan,
})
}
/// Return the total number of rows in this `DataFrame`.
///
/// Note that this method will actually run a plan to calculate the count,
/// which may be slow for large or complicated DataFrames.
///
/// # Example
/// ```
/// # use datafusion::prelude::*;
/// # use datafusion::error::Result;
/// # #[tokio::main]
/// # async fn main() -> Result<()> {
/// let ctx = SessionContext::new();
/// let df = ctx.read_csv("tests/data/example.csv", CsvReadOptions::new()).await?;
/// let count = df.count().await?;
/// # Ok(())
/// # }
/// ```
pub async fn count(self) -> Result<usize> {
let rows = self
.aggregate(
vec![],
vec![datafusion_expr::count(Expr::Literal(COUNT_STAR_EXPANSION))],
)?
.collect()
.await?;
let len = *rows
.first()
.and_then(|r| r.columns().first())
.and_then(|c| c.as_any().downcast_ref::<Int64Array>())
.and_then(|a| a.values().first())
.ok_or(DataFusionError::Internal(
"Unexpected output when collecting for count()".to_string(),
))? as usize;
Ok(len)
}
/// Execute this `DataFrame` and buffer all resulting `RecordBatch`es into memory.
///
/// Prior to calling `collect`, modifying a DataFrame simply updates a plan
/// (no actual computation is performed). `collect` triggers the computation.
///
/// See [`Self::execute_stream`] to execute a DataFrame without buffering.
///
/// # Example
/// ```
/// # use datafusion::prelude::*;
/// # use datafusion::error::Result;
/// # #[tokio::main]
/// # async fn main() -> Result<()> {
/// let ctx = SessionContext::new();
/// let df = ctx.read_csv("tests/data/example.csv", CsvReadOptions::new()).await?;
/// let batches = df.collect().await?;
/// # Ok(())
/// # }
/// ```
pub async fn collect(self) -> Result<Vec<RecordBatch>> {
let task_ctx = Arc::new(self.task_ctx());
let plan = self.create_physical_plan().await?;
collect(plan, task_ctx).await
}
/// Execute the `DataFrame` and print the results to the console.
///
/// # Example
/// ```
/// # use datafusion::prelude::*;
/// # use datafusion::error::Result;
/// # #[tokio::main]
/// # async fn main() -> Result<()> {
/// let ctx = SessionContext::new();
/// let df = ctx.read_csv("tests/data/example.csv", CsvReadOptions::new()).await?;
/// df.show().await?;
/// # Ok(())
/// # }
/// ```
pub async fn show(self) -> Result<()> {
let results = self.collect().await?;
Ok(pretty::print_batches(&results)?)
}
/// Execute the `DataFrame` and print only the first `num` rows of the
/// result to the console.
///
/// # Example
/// ```
/// # use datafusion::prelude::*;
/// # use datafusion::error::Result;
/// # #[tokio::main]
/// # async fn main() -> Result<()> {
/// let ctx = SessionContext::new();
/// let df = ctx.read_csv("tests/data/example.csv", CsvReadOptions::new()).await?;
/// df.show_limit(10).await?;
/// # Ok(())
/// # }
/// ```
pub async fn show_limit(self, num: usize) -> Result<()> {
let results = self.limit(0, Some(num))?.collect().await?;
Ok(pretty::print_batches(&results)?)
}
/// Return a new [`TaskContext`] which would be used to execute this DataFrame
pub fn task_ctx(&self) -> TaskContext {
TaskContext::from(self.session_state.as_ref())
}
/// Executes this DataFrame and returns a stream over a single partition
///
/// See [Self::collect] to buffer the `RecordBatch`es in memory.
///
/// # Example
/// ```
/// # use datafusion::prelude::*;
/// # use datafusion::error::Result;
/// # #[tokio::main]
/// # async fn main() -> Result<()> {
/// let ctx = SessionContext::new();
/// let df = ctx.read_csv("tests/data/example.csv", CsvReadOptions::new()).await?;
/// let stream = df.execute_stream().await?;
/// # Ok(())
/// # }
/// ```
///
/// # Aborting Execution
///
/// Dropping the stream will abort the execution of the query, and free up
/// any allocated resources
pub async fn execute_stream(self) -> Result<SendableRecordBatchStream> {
let task_ctx = Arc::new(self.task_ctx());
let plan = self.create_physical_plan().await?;
execute_stream(plan, task_ctx)
}
/// Executes this DataFrame and collects all results into a vector of vector of RecordBatch
/// maintaining the input partitioning.
///
/// # Example
/// ```
/// # use datafusion::prelude::*;
/// # use datafusion::error::Result;
/// # #[tokio::main]
/// # async fn main() -> Result<()> {
/// let ctx = SessionContext::new();
/// let df = ctx.read_csv("tests/data/example.csv", CsvReadOptions::new()).await?;
/// let batches = df.collect_partitioned().await?;
/// # Ok(())
/// # }
/// ```
pub async fn collect_partitioned(self) -> Result<Vec<Vec<RecordBatch>>> {
let task_ctx = Arc::new(self.task_ctx());
let plan = self.create_physical_plan().await?;
collect_partitioned(plan, task_ctx).await
}
/// Executes this DataFrame and returns one stream per partition.
///
/// # Example
/// ```
/// # use datafusion::prelude::*;
/// # use datafusion::error::Result;
/// # #[tokio::main]
/// # async fn main() -> Result<()> {
/// let ctx = SessionContext::new();
/// let df = ctx.read_csv("tests/data/example.csv", CsvReadOptions::new()).await?;
/// let batches = df.execute_stream_partitioned().await?;
/// # Ok(())
/// # }
/// ```
/// # Aborting Execution
///
/// Dropping the stream will abort the execution of the query, and free up
/// any allocated resources
pub async fn execute_stream_partitioned(
self,