-
Notifications
You must be signed in to change notification settings - Fork 1.8k
feat: support named variables & defaults for CREATE FUNCTION
#18450
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
2e1b3f6
b9b3577
8d50405
b95f91a
a97ddb5
0330adb
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 |
|---|---|---|
|
|
@@ -104,13 +104,13 @@ impl<S: ContextProvider> SqlToRel<'_, S> { | |
| } | ||
|
|
||
| /// Create a placeholder expression | ||
| /// This is the same as Postgres's prepare statement syntax in which a placeholder starts with `$` sign and then | ||
| /// number 1, 2, ... etc. For example, `$1` is the first placeholder; $2 is the second one and so on. | ||
| /// Both named (`$foo`) and positional (`$1`, `$2`, ...) placeholder styles are supported. | ||
| fn create_placeholder_expr( | ||
| param: String, | ||
| param_data_types: &[FieldRef], | ||
| ) -> Result<Expr> { | ||
| // Parse the placeholder as a number because it is the only support from sqlparser and postgres | ||
| // Try to parse the placeholder as a number. If the placeholder does not have a valid | ||
| // positional value, assume we have a named placeholder. | ||
| let index = param[1..].parse::<usize>(); | ||
| let idx = match index { | ||
| Ok(0) => { | ||
|
|
@@ -123,12 +123,24 @@ impl<S: ContextProvider> SqlToRel<'_, S> { | |
| return if param_data_types.is_empty() { | ||
| Ok(Expr::Placeholder(Placeholder::new_with_field(param, None))) | ||
| } else { | ||
| // when PREPARE Statement, param_data_types length is always 0 | ||
| plan_err!("Invalid placeholder, not a number: {param}") | ||
| // FIXME: This branch is shared by params from PREPARE and CREATE FUNCTION, but | ||
| // only CREATE FUNCTION currently supports named params. For now, we rewrite | ||
| // these to positional params. | ||
|
Comment on lines
+126
to
+128
Contributor
Author
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. note I explored doing this without rewriting to positional params, but I couldn't see a path forward without either:
|
||
| let named_param_pos = param_data_types | ||
| .iter() | ||
| .position(|v| v.name() == ¶m[1..]); | ||
| match named_param_pos { | ||
| Some(pos) => Ok(Expr::Placeholder(Placeholder::new_with_field( | ||
| format!("${}", pos + 1), | ||
| param_data_types.get(pos).cloned(), | ||
| ))), | ||
| None => plan_err!("Unknown placeholder: {param}"), | ||
| } | ||
| }; | ||
| } | ||
| }; | ||
| // Check if the placeholder is in the parameter list | ||
| // FIXME: In the CREATE FUNCTION branch, param_type = None should raise an error | ||
| let param_type = param_data_types.get(idx); | ||
| // Data type of the parameter | ||
| debug!("type of param {param} param_data_types[idx]: {param_type:?}"); | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -1222,6 +1222,28 @@ impl<S: ContextProvider> SqlToRel<'_, S> { | |
| } | ||
| None => None, | ||
| }; | ||
| // Validate default arguments | ||
| let first_default = match args.as_ref() { | ||
| Some(arg) => arg.iter().position(|t| t.default_expr.is_some()), | ||
| None => None, | ||
| }; | ||
| let last_non_default = match args.as_ref() { | ||
| Some(arg) => arg | ||
| .iter() | ||
| .rev() | ||
| .position(|t| t.default_expr.is_none()) | ||
| .map(|reverse_pos| arg.len() - reverse_pos - 1), | ||
| None => None, | ||
| }; | ||
| if let (Some(pos_default), Some(pos_non_default)) = | ||
| (first_default, last_non_default) | ||
| { | ||
| if pos_non_default > pos_default { | ||
| return plan_err!( | ||
| "Non-default arguments cannot follow default arguments." | ||
| ); | ||
| } | ||
| } | ||
| // At the moment functions can't be qualified `schema.name` | ||
| let name = match &name.0[..] { | ||
| [] => exec_err!("Function should have name")?, | ||
|
|
@@ -1233,9 +1255,23 @@ impl<S: ContextProvider> SqlToRel<'_, S> { | |
| // | ||
| let arg_types = args.as_ref().map(|arg| { | ||
| arg.iter() | ||
| .map(|t| Arc::new(Field::new("", t.data_type.clone(), true))) | ||
| .map(|t| { | ||
| let name = match t.name.clone() { | ||
| Some(name) => name.value, | ||
| None => "".to_string(), | ||
| }; | ||
| Arc::new(Field::new(name, t.data_type.clone(), true)) | ||
| }) | ||
| .collect::<Vec<_>>() | ||
| }); | ||
| // Validate parameter style | ||
| if let Some(ref fields) = arg_types { | ||
| let count_positional = | ||
| fields.iter().filter(|f| f.name() == "").count(); | ||
| if !(count_positional == 0 || count_positional == fields.len()) { | ||
| return plan_err!("All function arguments must use either named or positional style."); | ||
| } | ||
| } | ||
|
Comment on lines
+1267
to
+1274
Contributor
Author
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. note I'm not sure this is actually necessary, but it made it easier to reason about the changes. If we think this is valuable I can look at relaxing this constraint. |
||
| let mut planner_context = PlannerContext::new() | ||
| .with_prepare_param_data_types(arg_types.unwrap_or_default()); | ||
|
|
||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Please add a test case with positional parameter that is not existing, e.g.
$5where there are less than 5 arguments.I have the feeling that it will fail with
index out of boundserror at https://github.com/apache/datafusion/pull/18450/files#diff-647d2e08b4d044bf63b35f9e23092ba9673b80b1568e8f3abffd7f909552ea1aR999You need to add a check similar to
if placeholder_position < defaults.len() {...}around it and return an error in theelseclauseUh oh!
There was an error while loading. Please reload this page.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks, addressed in 0330adb.
This case also revealed that the
DEFAULTsyntax is broken for positional params. I switched to=syntax and added a test that illustrates the problem in a97ddb5.Ref: https://github.com/apache/datafusion-sqlparser-rs/blob/308a7231bcbc5c1c8ab71fe38f17b1a21632a6c6/src/parser/mod.rs#L5536
EDIT: It seems that
=is the "canonical" syntax in any caseThere was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Proposed a fix for the
DEFAULTsyntax bug upstream: apache/datafusion-sqlparser-rs#2091