Struct datafusion::execution::context::ExecutionContext [−][src]
pub struct ExecutionContext {
pub state: Arc<Mutex<ExecutionContextState>>,
}
Expand description
ExecutionContext is the main interface for executing queries with DataFusion. The context provides the following functionality:
- Create 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
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::*;
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();
The following example demonstrates how to execute the same query using SQL:
use datafusion::prelude::*;
let mut ctx = ExecutionContext::new();
ctx.register_csv("example", "tests/example.csv", CsvReadOptions::new()).await?;
let results = ctx.sql("SELECT a, MIN(b) FROM example GROUP BY a LIMIT 100").await?;
Fields
state: Arc<Mutex<ExecutionContextState>>
Internal state for the context
Implementations
Creates a new execution context using the provided configuration.
Creates a dataframe that will execute a SQL query.
This method is async
because queries of type CREATE EXTERNAL TABLE
might require the schema to be inferred.
Creates a logical plan.
This function is intended for internal use and should not be called directly.
pub fn register_variable(
&mut self,
variable_type: VarType,
provider: Arc<dyn VarProvider + Send + Sync>
)
pub fn register_variable(
&mut self,
variable_type: VarType,
provider: Arc<dyn VarProvider + Send + Sync>
)
Registers a variable provider within this context.
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"
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"
Creates a DataFrame for reading an Avro data source.
Creates an empty DataFrame.
Creates a DataFrame for reading a CSV data source.
Creates a DataFrame for reading a Parquet data source.
Creates a DataFrame for reading a custom TableProvider.
pub async fn register_listing_table<'a>(
&'a mut self,
name: &'a str,
uri: &'a str,
options: ListingOptions,
provided_schema: Option<SchemaRef>
) -> Result<()>
pub async fn register_listing_table<'a>(
&'a mut self,
name: &'a str,
uri: &'a str,
options: ListingOptions,
provided_schema: Option<SchemaRef>
) -> Result<()>
Registers a table that uses the listing feature of the object store to find the files to be processed This is async because it might need to resolve the schema.
pub async fn register_csv(
&mut self,
name: &str,
uri: &str,
options: CsvReadOptions<'_>
) -> Result<()>
pub async fn register_csv(
&mut self,
name: &str,
uri: &str,
options: CsvReadOptions<'_>
) -> Result<()>
Registers a CSV data source so that it can be referenced from SQL statements executed against this context.
Registers a Parquet data source so that it can be referenced from SQL statements executed against this context.
pub async fn register_avro(
&mut self,
name: &str,
uri: &str,
options: AvroReadOptions<'_>
) -> Result<()>
pub async fn register_avro(
&mut self,
name: &str,
uri: &str,
options: AvroReadOptions<'_>
) -> Result<()>
Registers an Avro data source so that it can be referenced from SQL statements executed against this context.
pub fn register_catalog(
&self,
name: impl Into<String>,
catalog: Arc<dyn CatalogProvider>
) -> Option<Arc<dyn CatalogProvider>>
pub fn register_catalog(
&self,
name: impl Into<String>,
catalog: Arc<dyn CatalogProvider>
) -> Option<Arc<dyn CatalogProvider>>
Registers a named catalog using a custom CatalogProvider
so that
it can be referenced from SQL statements executed against this
context.
Returns the CatalogProvider
previously registered for this
name, if any
Retrieves a CatalogProvider
instance by name
pub fn register_object_store(
&self,
scheme: impl Into<String>,
object_store: Arc<dyn ObjectStore>
) -> Option<Arc<dyn ObjectStore>>
pub fn register_object_store(
&self,
scheme: impl Into<String>,
object_store: Arc<dyn ObjectStore>
) -> Option<Arc<dyn ObjectStore>>
Registers a object store with scheme using a custom ObjectStore
so that
an external file system or object storage system could be used against this context.
Returns the ObjectStore
previously registered for this scheme, if any
Retrieves a ObjectStore
instance by scheme
pub fn register_table<'a>(
&'a mut self,
table_ref: impl Into<TableReference<'a>>,
provider: Arc<dyn TableProvider>
) -> Result<Option<Arc<dyn TableProvider>>>
pub fn register_table<'a>(
&'a mut self,
table_ref: impl Into<TableReference<'a>>,
provider: Arc<dyn TableProvider>
) -> Result<Option<Arc<dyn TableProvider>>>
Registers a table using a custom TableProvider
so that
it can be referenced from SQL statements executed against this
context.
Returns the TableProvider
previously registered for this
reference, if any
pub fn deregister_table<'a>(
&'a mut self,
table_ref: impl Into<TableReference<'a>>
) -> Result<Option<Arc<dyn TableProvider>>>
pub fn deregister_table<'a>(
&'a mut self,
table_ref: impl Into<TableReference<'a>>
) -> Result<Option<Arc<dyn TableProvider>>>
Deregisters the given table.
Returns the registered provider, if any
Retrieves a DataFrame representing a table previously registered by calling the register_table function.
Returns an error if no table has been registered with the provided reference.
👎 Deprecated: Please use the catalog provider interface (ExecutionContext::catalog
) to examine available catalogs, schemas, and tables
Please use the catalog provider interface (ExecutionContext::catalog
) to examine available catalogs, schemas, and tables
Returns the set of available tables in the default catalog and schema.
Use table
to get a specific table.
Optimizes the logical plan by applying optimizer rules.
pub async fn create_physical_plan(
&self,
logical_plan: &LogicalPlan
) -> Result<Arc<dyn ExecutionPlan>>
pub async fn create_physical_plan(
&self,
logical_plan: &LogicalPlan
) -> Result<Arc<dyn ExecutionPlan>>
Creates a physical plan from a logical plan.
Executes a query and writes the results to a partitioned CSV file.
pub async fn write_parquet(
&self,
plan: Arc<dyn ExecutionPlan>,
path: impl AsRef<str>,
writer_properties: Option<WriterProperties>
) -> Result<()>
pub async fn write_parquet(
&self,
plan: Arc<dyn ExecutionPlan>,
path: impl AsRef<str>,
writer_properties: Option<WriterProperties>
) -> Result<()>
Executes a query and writes the results to a partitioned Parquet file.
Trait Implementations
Performs the conversion.
Auto Trait Implementations
impl RefUnwindSafe for ExecutionContext
impl Send for ExecutionContext
impl Sync for ExecutionContext
impl Unpin for ExecutionContext
impl UnwindSafe for ExecutionContext
Blanket Implementations
Mutably borrows from an owned value. Read more