-
Notifications
You must be signed in to change notification settings - Fork 1.5k
Add plugable handler for CREATE FUNCTION
#9333
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
c87b609
be5b233
76dc3bf
f265d68
f084201
a8d54b6
262f0f6
fe63c31
b975df7
9d1d715
7e7d896
0430c11
210b194
1d5d739
84b0fbd
b642570
b8f8991
5a9ad09
58479e3
83acc8c
383602c
00b8058
f27d800
d7e37ed
8a0f42f
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -73,9 +73,10 @@ use crate::datasource::{ | |
}; | ||
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, | ||
CreateCatalog, CreateCatalogSchema, CreateExternalTable, CreateFunction, | ||
CreateMemoryTable, CreateView, DropCatalogSchema, DropFunction, DropTable, DropView, | ||
Explain, LogicalPlan, LogicalPlanBuilder, SetVariable, TableSource, TableType, | ||
UNNAMED_TABLE, | ||
}; | ||
use crate::optimizer::OptimizerRule; | ||
use datafusion_sql::{ | ||
|
@@ -489,6 +490,8 @@ impl SessionContext { | |
DdlStatement::DropTable(cmd) => self.drop_table(cmd).await, | ||
DdlStatement::DropView(cmd) => self.drop_view(cmd).await, | ||
DdlStatement::DropCatalogSchema(cmd) => self.drop_schema(cmd).await, | ||
DdlStatement::CreateFunction(cmd) => self.create_function(cmd).await, | ||
DdlStatement::DropFunction(cmd) => self.drop_function(cmd).await, | ||
}, | ||
// TODO what about the other statements (like TransactionStart and TransactionEnd) | ||
LogicalPlan::Statement(Statement::SetVariable(stmt)) => { | ||
|
@@ -794,6 +797,55 @@ impl SessionContext { | |
Ok(false) | ||
} | ||
|
||
async fn create_function(&self, stmt: CreateFunction) -> Result<DataFrame> { | ||
let function = { | ||
let state = self.state.read().clone(); | ||
let function_factory = &state.function_factory; | ||
|
||
match function_factory { | ||
Some(f) => f.create(state.config(), stmt).await?, | ||
_ => Err(DataFusionError::Configuration( | ||
"Function factory has not been configured".into(), | ||
))?, | ||
} | ||
}; | ||
|
||
match function { | ||
RegisterFunction::Scalar(f) => { | ||
self.state.write().register_udf(f)?; | ||
} | ||
RegisterFunction::Aggregate(f) => { | ||
self.state.write().register_udaf(f)?; | ||
} | ||
RegisterFunction::Window(f) => { | ||
self.state.write().register_udwf(f)?; | ||
} | ||
RegisterFunction::Table(name, f) => self.register_udtf(&name, f), | ||
}; | ||
|
||
self.return_empty_dataframe() | ||
} | ||
|
||
async fn drop_function(&self, stmt: DropFunction) -> Result<DataFrame> { | ||
// we don't know function type at this point | ||
// decision has been made to drop all functions | ||
let mut dropped = false; | ||
dropped |= self.state.write().deregister_udf(&stmt.name)?.is_some(); | ||
dropped |= self.state.write().deregister_udaf(&stmt.name)?.is_some(); | ||
dropped |= self.state.write().deregister_udwf(&stmt.name)?.is_some(); | ||
|
||
// DROP FUNCTION IF EXISTS drops the specified function only if that | ||
// function exists and in this way, it avoids error. While the DROP FUNCTION | ||
// statement also performs the same function, it throws an | ||
// error if the function does not exist. | ||
|
||
if !stmt.if_exists && !dropped { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. It is better to add the There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. That is a good call -- I added a test to sqllogictest |
||
exec_err!("Function does not exist") | ||
} else { | ||
self.return_empty_dataframe() | ||
} | ||
} | ||
|
||
/// Registers a variable provider within this context. | ||
pub fn register_variable( | ||
&self, | ||
|
@@ -1261,7 +1313,30 @@ impl QueryPlanner for DefaultQueryPlanner { | |
.await | ||
} | ||
} | ||
/// A pluggable interface to handle `CREATE FUNCTION` statements | ||
/// and interact with [SessionState] to registers new udf, udaf or udwf. | ||
|
||
#[async_trait] | ||
pub trait FunctionFactory: Sync + Send { | ||
milenkovicm marked this conversation as resolved.
Show resolved
Hide resolved
|
||
/// Handles creation of user defined function specified in [CreateFunction] statement | ||
async fn create( | ||
&self, | ||
state: &SessionConfig, | ||
statement: CreateFunction, | ||
) -> Result<RegisterFunction>; | ||
} | ||
|
||
/// Type of function to create | ||
pub enum RegisterFunction { | ||
/// Scalar user defined function | ||
Scalar(Arc<ScalarUDF>), | ||
/// Aggregate user defined function | ||
Aggregate(Arc<AggregateUDF>), | ||
/// Window user defined function | ||
Window(Arc<WindowUDF>), | ||
/// Table user defined function | ||
Table(String, Arc<dyn TableFunctionImpl>), | ||
} | ||
/// Execution context for registering data sources and executing queries. | ||
/// See [`SessionContext`] for a higher level API. | ||
/// | ||
|
@@ -1306,6 +1381,12 @@ pub struct SessionState { | |
table_factories: HashMap<String, Arc<dyn TableProviderFactory>>, | ||
/// Runtime environment | ||
runtime_env: Arc<RuntimeEnv>, | ||
|
||
/// [FunctionFactory] to support pluggable user defined function handler. | ||
/// | ||
/// It will be invoked on `CREATE FUNCTION` statements. | ||
/// thus, changing dialect o PostgreSql is required | ||
function_factory: Option<Arc<dyn FunctionFactory>>, | ||
} | ||
|
||
impl Debug for SessionState { | ||
|
@@ -1392,6 +1473,7 @@ impl SessionState { | |
execution_props: ExecutionProps::new(), | ||
runtime_env: runtime, | ||
table_factories, | ||
function_factory: None, | ||
}; | ||
|
||
// register built in functions | ||
|
@@ -1568,6 +1650,15 @@ impl SessionState { | |
self | ||
} | ||
|
||
/// Registers a [`FunctionFactory`] to handle `CREATE FUNCTION` statements | ||
pub fn with_function_factory( | ||
mut self, | ||
function_factory: Arc<dyn FunctionFactory>, | ||
) -> Self { | ||
self.function_factory = Some(function_factory); | ||
self | ||
} | ||
|
||
/// Replace the extension [`SerializerRegistry`] | ||
pub fn with_serializer_registry( | ||
mut self, | ||
|
Uh oh!
There was an error while loading. Please reload this page.