-
Notifications
You must be signed in to change notification settings - Fork 1.3k
/
Copy pathmod.rs
2557 lines (2281 loc) · 87 KB
/
mod.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
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.
//! [`SessionContext`] contains methods for registering data sources and executing queries
mod avro;
mod csv;
mod json;
#[cfg(feature = "parquet")]
mod parquet;
use crate::{
catalog::{CatalogList, MemoryCatalogList},
datasource::{
function::{TableFunction, TableFunctionImpl},
listing::{ListingOptions, ListingTable},
provider::TableProviderFactory,
},
datasource::{MemTable, ViewTable},
logical_expr::{PlanType, ToStringifiedPlan},
optimizer::optimizer::Optimizer,
physical_optimizer::optimizer::{PhysicalOptimizer, PhysicalOptimizerRule},
};
use datafusion_common::{
alias::AliasGenerator,
exec_err, not_impl_err, plan_datafusion_err, plan_err,
tree_node::{TreeNode, TreeNodeVisitor, VisitRecursion},
};
use datafusion_execution::registry::SerializerRegistry;
use datafusion_expr::{
logical_plan::{DdlStatement, Statement},
Expr, StringifiedPlan, UserDefinedLogicalNode, WindowUDF,
};
pub use datafusion_physical_expr::execution_props::ExecutionProps;
use datafusion_physical_expr::var_provider::is_system_variables;
use parking_lot::RwLock;
use std::collections::hash_map::Entry;
use std::string::String;
use std::sync::Arc;
use std::{
collections::{HashMap, HashSet},
fmt::Debug,
};
use std::{ops::ControlFlow, sync::Weak};
use arrow::datatypes::{DataType, SchemaRef};
use arrow::record_batch::RecordBatch;
use crate::catalog::{
schema::{MemorySchemaProvider, SchemaProvider},
{CatalogProvider, MemoryCatalogProvider},
};
use crate::dataframe::DataFrame;
use crate::datasource::{
listing::{ListingTableConfig, ListingTableUrl},
provider_as_source, TableProvider,
};
use crate::error::{DataFusionError, Result};
use crate::logical_expr::{
CreateCatalog, CreateCatalogSchema, CreateExternalTable, CreateMemoryTable,
CreateView, DropCatalogSchema, DropTable, DropView, Explain, LogicalPlan,
LogicalPlanBuilder, SetVariable, TableSource, TableType, UNNAMED_TABLE,
};
use crate::optimizer::OptimizerRule;
use datafusion_sql::{
parser::{CopyToSource, CopyToStatement},
planner::ParserOptions,
ResolvedTableReference, TableReference,
};
use sqlparser::dialect::dialect_from_str;
use crate::config::ConfigOptions;
use crate::execution::{runtime_env::RuntimeEnv, FunctionRegistry};
use crate::physical_plan::udaf::AggregateUDF;
use crate::physical_plan::udf::ScalarUDF;
use crate::physical_plan::ExecutionPlan;
use crate::physical_planner::DefaultPhysicalPlanner;
use crate::physical_planner::PhysicalPlanner;
use crate::variable::{VarProvider, VarType};
use async_trait::async_trait;
use chrono::{DateTime, Utc};
use datafusion_common::{OwnedTableReference, SchemaReference};
use datafusion_sql::{
parser::DFParser,
planner::{ContextProvider, SqlToRel},
};
use url::Url;
use crate::catalog::information_schema::{InformationSchemaProvider, INFORMATION_SCHEMA};
use crate::catalog::listing_schema::ListingSchemaProvider;
use crate::datasource::object_store::ObjectStoreUrl;
use datafusion_optimizer::{
analyzer::{Analyzer, AnalyzerRule},
OptimizerConfig,
};
use datafusion_sql::planner::object_name_to_table_reference;
use uuid::Uuid;
// backwards compatibility
use crate::datasource::provider::DefaultTableFactory;
use crate::execution::options::ArrowReadOptions;
pub use datafusion_execution::config::SessionConfig;
pub use datafusion_execution::TaskContext;
use super::options::ReadOptions;
/// DataFilePaths adds a method to convert strings and vector of strings to vector of [`ListingTableUrl`] URLs.
/// This allows methods such [`SessionContext::read_csv`] and [`SessionContext::read_avro`]
/// to take either a single file or multiple files.
pub trait DataFilePaths {
/// Parse to a vector of [`ListingTableUrl`] URLs.
fn to_urls(self) -> Result<Vec<ListingTableUrl>>;
}
impl DataFilePaths for &str {
fn to_urls(self) -> Result<Vec<ListingTableUrl>> {
Ok(vec![ListingTableUrl::parse(self)?])
}
}
impl DataFilePaths for String {
fn to_urls(self) -> Result<Vec<ListingTableUrl>> {
Ok(vec![ListingTableUrl::parse(self)?])
}
}
impl DataFilePaths for &String {
fn to_urls(self) -> Result<Vec<ListingTableUrl>> {
Ok(vec![ListingTableUrl::parse(self)?])
}
}
impl<P> DataFilePaths for Vec<P>
where
P: AsRef<str>,
{
fn to_urls(self) -> Result<Vec<ListingTableUrl>> {
self.iter()
.map(ListingTableUrl::parse)
.collect::<Result<Vec<ListingTableUrl>>>()
}
}
/// Main interface for executing queries with DataFusion. Maintains
/// the state of the connection between a user and an instance of the
/// DataFusion engine.
///
/// # Overview
///
/// [`SessionContext`] provides the following functionality:
///
/// * Create a DataFrame from a CSV or Parquet data source.
/// * Register a CSV or Parquet data source as a table that can be referenced from a SQL query.
/// * Register a custom data source that can be referenced from a SQL query.
/// * Execution a SQL query
///
/// # Example: DataFrame API
///
/// The following example demonstrates how to use the context to execute a query against a CSV
/// data source using the DataFrame API:
///
/// ```
/// use datafusion::prelude::*;
/// # use datafusion::{error::Result, assert_batches_eq};
/// # #[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")))?
/// .aggregate(vec![col("a")], vec![min(col("b"))])?
/// .limit(0, Some(100))?;
/// let results = df
/// .collect()
/// .await?;
/// assert_batches_eq!(
/// &[
/// "+---+----------------+",
/// "| a | MIN(?table?.b) |",
/// "+---+----------------+",
/// "| 1 | 2 |",
/// "+---+----------------+",
/// ],
/// &results
/// );
/// # Ok(())
/// # }
/// ```
///
/// # Example: SQL API
///
/// The following example demonstrates how to execute the same query using SQL:
///
/// ```
/// use datafusion::prelude::*;
/// # use datafusion::{error::Result, assert_batches_eq};
/// # #[tokio::main]
/// # async fn main() -> Result<()> {
/// let mut ctx = SessionContext::new();
/// ctx.register_csv("example", "tests/data/example.csv", CsvReadOptions::new()).await?;
/// let results = ctx
/// .sql("SELECT a, MIN(b) FROM example GROUP BY a LIMIT 100")
/// .await?
/// .collect()
/// .await?;
/// assert_batches_eq!(
/// &[
/// "+---+----------------+",
/// "| a | MIN(example.b) |",
/// "+---+----------------+",
/// "| 1 | 2 |",
/// "+---+----------------+",
/// ],
/// &results
/// );
/// # Ok(())
/// # }
/// ```
///
/// # `SessionContext`, `SessionState`, and `TaskContext`
///
/// A [`SessionContext`] can be created from a [`SessionConfig`] and
/// stores the state for a particular query session. A single
/// [`SessionContext`] can run multiple queries.
///
/// [`SessionState`] contains information available during query
/// planning (creating [`LogicalPlan`]s and [`ExecutionPlan`]s).
///
/// [`TaskContext`] contains the state available during query
/// execution [`ExecutionPlan::execute`]. It contains a subset of the
/// information in[`SessionState`] and is created from a
/// [`SessionContext`] or a [`SessionState`].
#[derive(Clone)]
pub struct SessionContext {
/// UUID for the session
session_id: String,
/// Session start time
session_start_time: DateTime<Utc>,
/// Shared session state for the session
state: Arc<RwLock<SessionState>>,
}
impl Default for SessionContext {
fn default() -> Self {
Self::new()
}
}
impl SessionContext {
/// Creates a new `SessionContext` using the default [`SessionConfig`].
pub fn new() -> Self {
Self::new_with_config(SessionConfig::new())
}
/// Finds any [`ListingSchemaProvider`]s and instructs them to reload tables from "disk"
pub async fn refresh_catalogs(&self) -> Result<()> {
let cat_names = self.catalog_names().clone();
for cat_name in cat_names.iter() {
let cat = self.catalog(cat_name.as_str()).ok_or_else(|| {
DataFusionError::Internal("Catalog not found!".to_string())
})?;
for schema_name in cat.schema_names() {
let schema = cat.schema(schema_name.as_str()).ok_or_else(|| {
DataFusionError::Internal("Schema not found!".to_string())
})?;
let lister = schema.as_any().downcast_ref::<ListingSchemaProvider>();
if let Some(lister) = lister {
lister.refresh(&self.state()).await?;
}
}
}
Ok(())
}
/// Creates a new `SessionContext` using the provided
/// [`SessionConfig`] and a new [`RuntimeEnv`].
///
/// See [`Self::new_with_config_rt`] for more details on resource
/// limits.
pub fn new_with_config(config: SessionConfig) -> Self {
let runtime = Arc::new(RuntimeEnv::default());
Self::new_with_config_rt(config, runtime)
}
/// Creates a new `SessionContext` using the provided
/// [`SessionConfig`] and a new [`RuntimeEnv`].
#[deprecated(since = "32.0.0", note = "Use SessionContext::new_with_config")]
pub fn with_config(config: SessionConfig) -> Self {
Self::new_with_config(config)
}
/// Creates a new `SessionContext` using the provided
/// [`SessionConfig`] and a [`RuntimeEnv`].
///
/// # Resource Limits
///
/// By default, each new `SessionContext` creates a new
/// `RuntimeEnv`, and therefore will not enforce memory or disk
/// limits for queries run on different `SessionContext`s.
///
/// To enforce resource limits (e.g. to limit the total amount of
/// memory used) across all DataFusion queries in a process,
/// all `SessionContext`'s should be configured with the
/// same `RuntimeEnv`.
pub fn new_with_config_rt(config: SessionConfig, runtime: Arc<RuntimeEnv>) -> Self {
let state = SessionState::new_with_config_rt(config, runtime);
Self::new_with_state(state)
}
/// Creates a new `SessionContext` using the provided
/// [`SessionConfig`] and a [`RuntimeEnv`].
#[deprecated(since = "32.0.0", note = "Use SessionState::new_with_config_rt")]
pub fn with_config_rt(config: SessionConfig, runtime: Arc<RuntimeEnv>) -> Self {
Self::new_with_config_rt(config, runtime)
}
/// Creates a new `SessionContext` using the provided [`SessionState`]
pub fn new_with_state(state: SessionState) -> Self {
Self {
session_id: state.session_id.clone(),
session_start_time: Utc::now(),
state: Arc::new(RwLock::new(state)),
}
}
/// Creates a new `SessionContext` using the provided [`SessionState`]
#[deprecated(since = "32.0.0", note = "Use SessionState::new_with_state")]
pub fn with_state(state: SessionState) -> Self {
Self::new_with_state(state)
}
/// Returns the time this `SessionContext` was created
pub fn session_start_time(&self) -> DateTime<Utc> {
self.session_start_time
}
/// Registers the [`RecordBatch`] as the specified table name
pub fn register_batch(
&self,
table_name: &str,
batch: RecordBatch,
) -> Result<Option<Arc<dyn TableProvider>>> {
let table = MemTable::try_new(batch.schema(), vec![vec![batch]])?;
self.register_table(
TableReference::Bare {
table: table_name.into(),
},
Arc::new(table),
)
}
/// Return the [RuntimeEnv] used to run queries with this `SessionContext`
pub fn runtime_env(&self) -> Arc<RuntimeEnv> {
self.state.read().runtime_env.clone()
}
/// Returns an id that uniquely identifies this `SessionContext`.
pub fn session_id(&self) -> String {
self.session_id.clone()
}
/// Return the [`TableProviderFactory`] that is registered for the
/// specified file type, if any.
pub fn table_factory(
&self,
file_type: &str,
) -> Option<Arc<dyn TableProviderFactory>> {
self.state.read().table_factories().get(file_type).cloned()
}
/// Return the `enable_ident_normalization` of this Session
pub fn enable_ident_normalization(&self) -> bool {
self.state
.read()
.config
.options()
.sql_parser
.enable_ident_normalization
}
/// Return a copied version of config for this Session
pub fn copied_config(&self) -> SessionConfig {
self.state.read().config.clone()
}
/// Creates a [`DataFrame`] from SQL query text.
///
/// Note: This API implements DDL statements such as `CREATE TABLE` and
/// `CREATE VIEW` and DML statements such as `INSERT INTO` with in-memory
/// default implementations. See [`Self::sql_with_options`].
///
/// # Example: Running SQL queries
///
/// See the example on [`Self`]
///
/// # Example: Creating a Table with SQL
///
/// ```
/// use datafusion::prelude::*;
/// # use datafusion::{error::Result, assert_batches_eq};
/// # #[tokio::main]
/// # async fn main() -> Result<()> {
/// let mut ctx = SessionContext::new();
/// ctx
/// .sql("CREATE TABLE foo (x INTEGER)")
/// .await?
/// .collect()
/// .await?;
/// assert!(ctx.table_exist("foo").unwrap());
/// # Ok(())
/// # }
/// ```
pub async fn sql(&self, sql: &str) -> Result<DataFrame> {
self.sql_with_options(sql, SQLOptions::new()).await
}
/// Creates a [`DataFrame`] from SQL query text, first validating
/// that the queries are allowed by `options`
///
/// # Example: Preventing Creating a Table with SQL
///
/// If you want to avoid creating tables, or modifying data or the
/// session, set [`SQLOptions`] appropriately:
///
/// ```
/// use datafusion::prelude::*;
/// # use datafusion::{error::Result};
/// # use datafusion::physical_plan::collect;
/// # #[tokio::main]
/// # async fn main() -> Result<()> {
/// let mut ctx = SessionContext::new();
/// let options = SQLOptions::new()
/// .with_allow_ddl(false);
/// let err = ctx.sql_with_options("CREATE TABLE foo (x INTEGER)", options)
/// .await
/// .unwrap_err();
/// assert!(
/// err.to_string().starts_with("Error during planning: DDL not supported: CreateMemoryTable")
/// );
/// # Ok(())
/// # }
/// ```
pub async fn sql_with_options(
&self,
sql: &str,
options: SQLOptions,
) -> Result<DataFrame> {
let plan = self.state().create_logical_plan(sql).await?;
options.verify_plan(&plan)?;
self.execute_logical_plan(plan).await
}
/// Execute the [`LogicalPlan`], return a [`DataFrame`]. This API
/// is not featured limited (so all SQL such as `CREATE TABLE` and
/// `COPY` will be run).
///
/// If you wish to limit the type of plan that can be run from
/// SQL, see [`Self::sql_with_options`] and
/// [`SQLOptions::verify_plan`].
pub async fn execute_logical_plan(&self, plan: LogicalPlan) -> Result<DataFrame> {
match plan {
LogicalPlan::Ddl(ddl) => match ddl {
DdlStatement::CreateExternalTable(cmd) => {
self.create_external_table(&cmd).await
}
DdlStatement::CreateMemoryTable(cmd) => {
self.create_memory_table(cmd).await
}
DdlStatement::CreateView(cmd) => self.create_view(cmd).await,
DdlStatement::CreateCatalogSchema(cmd) => {
self.create_catalog_schema(cmd).await
}
DdlStatement::CreateCatalog(cmd) => self.create_catalog(cmd).await,
DdlStatement::DropTable(cmd) => self.drop_table(cmd).await,
DdlStatement::DropView(cmd) => self.drop_view(cmd).await,
DdlStatement::DropCatalogSchema(cmd) => self.drop_schema(cmd).await,
},
// TODO what about the other statements (like TransactionStart and TransactionEnd)
LogicalPlan::Statement(Statement::SetVariable(stmt)) => {
self.set_variable(stmt).await
}
plan => Ok(DataFrame::new(self.state(), plan)),
}
}
// return an empty dataframe
fn return_empty_dataframe(&self) -> Result<DataFrame> {
let plan = LogicalPlanBuilder::empty(false).build()?;
Ok(DataFrame::new(self.state(), plan))
}
async fn create_external_table(
&self,
cmd: &CreateExternalTable,
) -> Result<DataFrame> {
let exist = self.table_exist(&cmd.name)?;
if exist {
match cmd.if_not_exists {
true => return self.return_empty_dataframe(),
false => {
return exec_err!("Table '{}' already exists", cmd.name);
}
}
}
let table_provider: Arc<dyn TableProvider> =
self.create_custom_table(cmd).await?;
self.register_table(&cmd.name, table_provider)?;
self.return_empty_dataframe()
}
async fn create_memory_table(&self, cmd: CreateMemoryTable) -> Result<DataFrame> {
let CreateMemoryTable {
name,
input,
if_not_exists,
or_replace,
constraints,
column_defaults,
} = cmd;
let input = Arc::try_unwrap(input).unwrap_or_else(|e| e.as_ref().clone());
let input = self.state().optimize(&input)?;
let table = self.table(&name).await;
match (if_not_exists, or_replace, table) {
(true, false, Ok(_)) => self.return_empty_dataframe(),
(false, true, Ok(_)) => {
self.deregister_table(&name)?;
let schema = Arc::new(input.schema().as_ref().into());
let physical = DataFrame::new(self.state(), input);
let batches: Vec<_> = physical.collect_partitioned().await?;
let table = Arc::new(
// pass constraints and column defaults to the mem table.
MemTable::try_new(schema, batches)?
.with_constraints(constraints)
.with_column_defaults(column_defaults.into_iter().collect()),
);
self.register_table(&name, table)?;
self.return_empty_dataframe()
}
(true, true, Ok(_)) => {
exec_err!("'IF NOT EXISTS' cannot coexist with 'REPLACE'")
}
(_, _, Err(_)) => {
let df_schema = input.schema();
let schema = Arc::new(df_schema.as_ref().into());
let physical = DataFrame::new(self.state(), input);
let batches: Vec<_> = physical.collect_partitioned().await?;
let table = Arc::new(
// pass constraints and column defaults to the mem table.
MemTable::try_new(schema, batches)?
.with_constraints(constraints)
.with_column_defaults(column_defaults.into_iter().collect()),
);
self.register_table(&name, table)?;
self.return_empty_dataframe()
}
(false, false, Ok(_)) => exec_err!("Table '{name}' already exists"),
}
}
async fn create_view(&self, cmd: CreateView) -> Result<DataFrame> {
let CreateView {
name,
input,
or_replace,
definition,
} = cmd;
let view = self.table(&name).await;
match (or_replace, view) {
(true, Ok(_)) => {
self.deregister_table(&name)?;
let table = Arc::new(ViewTable::try_new((*input).clone(), definition)?);
self.register_table(&name, table)?;
self.return_empty_dataframe()
}
(_, Err(_)) => {
let table = Arc::new(ViewTable::try_new((*input).clone(), definition)?);
self.register_table(&name, table)?;
self.return_empty_dataframe()
}
(false, Ok(_)) => exec_err!("Table '{name}' already exists"),
}
}
async fn create_catalog_schema(&self, cmd: CreateCatalogSchema) -> Result<DataFrame> {
let CreateCatalogSchema {
schema_name,
if_not_exists,
..
} = cmd;
// sqlparser doesnt accept database / catalog as parameter to CREATE SCHEMA
// so for now, we default to default catalog
let tokens: Vec<&str> = schema_name.split('.').collect();
let (catalog, schema_name) = match tokens.len() {
1 => {
let state = self.state.read();
let name = &state.config.options().catalog.default_catalog;
let catalog = state.catalog_list.catalog(name).ok_or_else(|| {
DataFusionError::Execution(format!(
"Missing default catalog '{name}'"
))
})?;
(catalog, tokens[0])
}
2 => {
let name = &tokens[0];
let catalog = self.catalog(name).ok_or_else(|| {
DataFusionError::Execution(format!("Missing catalog '{name}'"))
})?;
(catalog, tokens[1])
}
_ => return exec_err!("Unable to parse catalog from {schema_name}"),
};
let schema = catalog.schema(schema_name);
match (if_not_exists, schema) {
(true, Some(_)) => self.return_empty_dataframe(),
(true, None) | (false, None) => {
let schema = Arc::new(MemorySchemaProvider::new());
catalog.register_schema(schema_name, schema)?;
self.return_empty_dataframe()
}
(false, Some(_)) => exec_err!("Schema '{schema_name}' already exists"),
}
}
async fn create_catalog(&self, cmd: CreateCatalog) -> Result<DataFrame> {
let CreateCatalog {
catalog_name,
if_not_exists,
..
} = cmd;
let catalog = self.catalog(catalog_name.as_str());
match (if_not_exists, catalog) {
(true, Some(_)) => self.return_empty_dataframe(),
(true, None) | (false, None) => {
let new_catalog = Arc::new(MemoryCatalogProvider::new());
self.state
.write()
.catalog_list
.register_catalog(catalog_name, new_catalog);
self.return_empty_dataframe()
}
(false, Some(_)) => exec_err!("Catalog '{catalog_name}' already exists"),
}
}
async fn drop_table(&self, cmd: DropTable) -> Result<DataFrame> {
let DropTable {
name, if_exists, ..
} = cmd;
let result = self.find_and_deregister(&name, TableType::Base).await;
match (result, if_exists) {
(Ok(true), _) => self.return_empty_dataframe(),
(_, true) => self.return_empty_dataframe(),
(_, _) => exec_err!("Table '{name}' doesn't exist."),
}
}
async fn drop_view(&self, cmd: DropView) -> Result<DataFrame> {
let DropView {
name, if_exists, ..
} = cmd;
let result = self.find_and_deregister(&name, TableType::View).await;
match (result, if_exists) {
(Ok(true), _) => self.return_empty_dataframe(),
(_, true) => self.return_empty_dataframe(),
(_, _) => exec_err!("View '{name}' doesn't exist."),
}
}
async fn drop_schema(&self, cmd: DropCatalogSchema) -> Result<DataFrame> {
let DropCatalogSchema {
name,
if_exists: allow_missing,
cascade,
schema: _,
} = cmd;
let catalog = {
let state = self.state.read();
let catalog_name = match &name {
SchemaReference::Full { catalog, .. } => catalog.to_string(),
SchemaReference::Bare { .. } => {
state.config_options().catalog.default_catalog.to_string()
}
};
if let Some(catalog) = state.catalog_list.catalog(&catalog_name) {
catalog
} else if allow_missing {
return self.return_empty_dataframe();
} else {
return self.schema_doesnt_exist_err(name);
}
};
let dereg = catalog.deregister_schema(name.schema_name(), cascade)?;
match (dereg, allow_missing) {
(None, true) => self.return_empty_dataframe(),
(None, false) => self.schema_doesnt_exist_err(name),
(Some(_), _) => self.return_empty_dataframe(),
}
}
fn schema_doesnt_exist_err(
&self,
schemaref: SchemaReference<'_>,
) -> Result<DataFrame> {
exec_err!("Schema '{schemaref}' doesn't exist.")
}
async fn set_variable(&self, stmt: SetVariable) -> Result<DataFrame> {
let SetVariable {
variable, value, ..
} = stmt;
let mut state = self.state.write();
state.config.options_mut().set(&variable, &value)?;
drop(state);
self.return_empty_dataframe()
}
async fn create_custom_table(
&self,
cmd: &CreateExternalTable,
) -> Result<Arc<dyn TableProvider>> {
let state = self.state.read().clone();
let file_type = cmd.file_type.to_uppercase();
let factory =
&state
.table_factories
.get(file_type.as_str())
.ok_or_else(|| {
DataFusionError::Execution(format!(
"Unable to find factory for {}",
cmd.file_type
))
})?;
let table = (*factory).create(&state, cmd).await?;
Ok(table)
}
async fn find_and_deregister<'a>(
&self,
table_ref: impl Into<TableReference<'a>>,
table_type: TableType,
) -> Result<bool> {
let table_ref = table_ref.into();
let table = table_ref.table().to_owned();
let maybe_schema = {
let state = self.state.read();
let resolved = state.resolve_table_ref(table_ref);
state
.catalog_list
.catalog(&resolved.catalog)
.and_then(|c| c.schema(&resolved.schema))
};
if let Some(schema) = maybe_schema {
if let Some(table_provider) = schema.table(&table).await {
if table_provider.table_type() == table_type {
schema.deregister_table(&table)?;
return Ok(true);
}
}
}
Ok(false)
}
/// Registers a variable provider within this context.
pub fn register_variable(
&self,
variable_type: VarType,
provider: Arc<dyn VarProvider + Send + Sync>,
) {
self.state
.write()
.execution_props
.add_var_provider(variable_type, provider);
}
/// Register a table UDF with this context
pub fn register_udtf(&self, name: &str, fun: Arc<dyn TableFunctionImpl>) {
self.state.write().table_functions.insert(
name.to_owned(),
Arc::new(TableFunction::new(name.to_owned(), fun)),
);
}
/// Registers a scalar UDF within this context.
///
/// Note in SQL queries, function names are looked up using
/// lowercase unless the query uses quotes. For example,
///
/// - `SELECT MY_FUNC(x)...` will look for a function named `"my_func"`
/// - `SELECT "my_FUNC"(x)` will look for a function named `"my_FUNC"`
/// Any functions registered with the udf name or its aliases will be overwritten with this new function
pub fn register_udf(&self, f: ScalarUDF) {
let mut state = self.state.write();
let aliases = f.aliases();
for alias in aliases {
state
.scalar_functions
.insert(alias.to_string(), Arc::new(f.clone()));
}
state
.scalar_functions
.insert(f.name().to_string(), Arc::new(f));
}
/// Registers an aggregate UDF within this context.
///
/// Note in SQL queries, aggregate names are looked up using
/// lowercase unless the query uses quotes. For example,
///
/// - `SELECT MY_UDAF(x)...` will look for an aggregate named `"my_udaf"`
/// - `SELECT "my_UDAF"(x)` will look for an aggregate named `"my_UDAF"`
pub fn register_udaf(&self, f: AggregateUDF) {
self.state
.write()
.aggregate_functions
.insert(f.name().to_string(), Arc::new(f));
}
/// Registers a window UDF within this context.
///
/// Note in SQL queries, window function names are looked up using
/// lowercase unless the query uses quotes. For example,
///
/// - `SELECT MY_UDWF(x)...` will look for a window function named `"my_udwf"`
/// - `SELECT "my_UDWF"(x)` will look for a window function named `"my_UDWF"`
pub fn register_udwf(&self, f: WindowUDF) {
self.state
.write()
.window_functions
.insert(f.name().to_string(), Arc::new(f));
}
/// Creates a [`DataFrame`] for reading a data source.
///
/// For more control such as reading multiple files, you can use
/// [`read_table`](Self::read_table) with a [`ListingTable`].
async fn _read_type<'a, P: DataFilePaths>(
&self,
table_paths: P,
options: impl ReadOptions<'a>,
) -> Result<DataFrame> {
let table_paths = table_paths.to_urls()?;
let session_config = self.copied_config();
let listing_options = options.to_listing_options(&session_config);
let option_extension = listing_options.file_extension.clone();
if table_paths.is_empty() {
return exec_err!("No table paths were provided");
}
// check if the file extension matches the expected extension
for path in &table_paths {
let file_path = path.as_str();
if !file_path.ends_with(option_extension.clone().as_str())
&& !path.is_collection()
{
return exec_err!(
"File path '{file_path}' does not match the expected extension '{option_extension}'"
);
}
}
let resolved_schema = options
.get_resolved_schema(&session_config, self.state(), table_paths[0].clone())
.await?;
let config = ListingTableConfig::new_with_multi_paths(table_paths)
.with_listing_options(listing_options)
.with_schema(resolved_schema);
let provider = ListingTable::try_new(config)?;
self.read_table(Arc::new(provider))
}
/// Creates a [`DataFrame`] for reading an Arrow data source.
///
/// For more control such as reading multiple files, you can use
/// [`read_table`](Self::read_table) with a [`ListingTable`].
///
/// For an example, see [`read_csv`](Self::read_csv)
pub async fn read_arrow<P: DataFilePaths>(
&self,
table_paths: P,
options: ArrowReadOptions<'_>,
) -> Result<DataFrame> {
self._read_type(table_paths, options).await
}
/// Creates an empty DataFrame.
pub fn read_empty(&self) -> Result<DataFrame> {
Ok(DataFrame::new(
self.state(),
LogicalPlanBuilder::empty(true).build()?,
))
}
/// Creates a [`DataFrame`] for a [`TableProvider`] such as a
/// [`ListingTable`] or a custom user defined provider.
pub fn read_table(&self, provider: Arc<dyn TableProvider>) -> Result<DataFrame> {
Ok(DataFrame::new(
self.state(),
LogicalPlanBuilder::scan(UNNAMED_TABLE, provider_as_source(provider), None)?
.build()?,
))
}
/// Creates a [`DataFrame`] for reading a [`RecordBatch`]
pub fn read_batch(&self, batch: RecordBatch) -> Result<DataFrame> {
let provider = MemTable::try_new(batch.schema(), vec![vec![batch]])?;
Ok(DataFrame::new(
self.state(),
LogicalPlanBuilder::scan(
UNNAMED_TABLE,
provider_as_source(Arc::new(provider)),
None,
)?
.build()?,
))
}
/// Registers a [`ListingTable`] that can assemble multiple files
/// from locations in an [`ObjectStore`] instance into a single
/// table.
///
/// This method is `async` because it might need to resolve the schema.
///
/// [`ObjectStore`]: object_store::ObjectStore
pub async fn register_listing_table(
&self,
name: &str,
table_path: impl AsRef<str>,
options: ListingOptions,
provided_schema: Option<SchemaRef>,
sql_definition: Option<String>,
) -> Result<()> {
let table_path = ListingTableUrl::parse(table_path)?;
let resolved_schema = match (provided_schema, options.infinite_source) {
(Some(s), _) => s,
(None, false) => options.infer_schema(&self.state(), &table_path).await?,
(None, true) => {
return plan_err!(
"Schema inference for infinite data sources is not supported."
)
}
};
let config = ListingTableConfig::new(table_path)
.with_listing_options(options)
.with_schema(resolved_schema);
let table = ListingTable::try_new(config)?.with_definition(sql_definition);
self.register_table(
TableReference::Bare { table: name.into() },
Arc::new(table),
)?;
Ok(())
}
/// Registers an Arrow file as a table that can be referenced from
/// SQL statements executed against this context.
pub async fn register_arrow(
&self,
name: &str,
table_path: &str,
options: ArrowReadOptions<'_>,
) -> Result<()> {
let listing_options = options.to_listing_options(&self.copied_config());
self.register_listing_table(
name,
table_path,
listing_options,