Skip to content

Commit 675881d

Browse files
authored
fix: insert placeholder type inference showing wrong type when there is function wrapped placeholder (unknown type) (#20744)
## Which issue does this PR close? - Closes #20544. ## Rationale for this change Please see #20544 for details. I wanted to open a PR to discuss if API change is ok for this fix ## What changes are included in this PR? This PR changes FieldRef to Option<FieldRef> for placeholders and adds test case for this. ## Are these changes tested? Yes. Existing tests and added test case for the reproduced error ## Are there any user-facing changes? Yes. this changes `prepare_param_data_types` field and its callers to use Option wrapped FieldRef instead of FieldRef.
1 parent 935382f commit 675881d

File tree

4 files changed

+47
-17
lines changed

4 files changed

+47
-17
lines changed

datafusion/sql/src/expr/value.rs

Lines changed: 8 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -45,7 +45,7 @@ impl<S: ContextProvider> SqlToRel<'_, S> {
4545
pub(crate) fn parse_value(
4646
&self,
4747
value: Value,
48-
param_data_types: &[FieldRef],
48+
param_data_types: &[Option<FieldRef>],
4949
) -> Result<Expr> {
5050
match value {
5151
Value::Number(n, _) => self.parse_sql_number(&n, false),
@@ -105,7 +105,7 @@ impl<S: ContextProvider> SqlToRel<'_, S> {
105105
/// Both named (`$foo`) and positional (`$1`, `$2`, ...) placeholder styles are supported.
106106
fn create_placeholder_expr(
107107
param: String,
108-
param_data_types: &[FieldRef],
108+
param_data_types: &[Option<FieldRef>],
109109
) -> Result<Expr> {
110110
// Try to parse the placeholder as a number. If the placeholder does not have a valid
111111
// positional value, assume we have a named placeholder.
@@ -124,13 +124,13 @@ impl<S: ContextProvider> SqlToRel<'_, S> {
124124
// FIXME: This branch is shared by params from PREPARE and CREATE FUNCTION, but
125125
// only CREATE FUNCTION currently supports named params. For now, we rewrite
126126
// these to positional params.
127-
let named_param_pos = param_data_types
128-
.iter()
129-
.position(|v| v.name() == &param[1..]);
127+
let named_param_pos = param_data_types.iter().position(|v| {
128+
v.as_ref().is_some_and(|field| field.name() == &param[1..])
129+
});
130130
match named_param_pos {
131131
Some(pos) => Ok(Expr::Placeholder(Placeholder::new_with_field(
132132
format!("${}", pos + 1),
133-
param_data_types.get(pos).cloned(),
133+
param_data_types.get(pos).and_then(|v| v.clone()),
134134
))),
135135
None => plan_err!("Unknown placeholder: {param}"),
136136
}
@@ -139,13 +139,12 @@ impl<S: ContextProvider> SqlToRel<'_, S> {
139139
};
140140
// Check if the placeholder is in the parameter list
141141
// FIXME: In the CREATE FUNCTION branch, param_type = None should raise an error
142-
let param_type = param_data_types.get(idx);
142+
let param_type = param_data_types.get(idx).and_then(|v| v.clone());
143143
// Data type of the parameter
144144
debug!("type of param {param} param_data_types[idx]: {param_type:?}");
145145

146146
Ok(Expr::Placeholder(Placeholder::new_with_field(
147-
param,
148-
param_type.cloned(),
147+
param, param_type,
149148
)))
150149
}
151150

datafusion/sql/src/planner.rs

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -257,7 +257,7 @@ impl IdentNormalizer {
257257
pub struct PlannerContext {
258258
/// Data types for numbered parameters ($1, $2, etc), if supplied
259259
/// in `PREPARE` statement
260-
prepare_param_data_types: Arc<Vec<FieldRef>>,
260+
prepare_param_data_types: Arc<Vec<Option<FieldRef>>>,
261261
/// Map of CTE name to logical plan of the WITH clause.
262262
/// Use `Arc<LogicalPlan>` to allow cheap cloning
263263
ctes: HashMap<String, Arc<LogicalPlan>>,
@@ -298,7 +298,7 @@ impl PlannerContext {
298298
/// Update the PlannerContext with provided prepare_param_data_types
299299
pub fn with_prepare_param_data_types(
300300
mut self,
301-
prepare_param_data_types: Vec<FieldRef>,
301+
prepare_param_data_types: Vec<Option<FieldRef>>,
302302
) -> Self {
303303
self.prepare_param_data_types = prepare_param_data_types.into();
304304
self
@@ -378,7 +378,7 @@ impl PlannerContext {
378378
}
379379

380380
/// Return the types of parameters (`$1`, `$2`, etc) if known
381-
pub fn prepare_param_data_types(&self) -> &[FieldRef] {
381+
pub fn prepare_param_data_types(&self) -> &[Option<FieldRef>] {
382382
&self.prepare_param_data_types
383383
}
384384

datafusion/sql/src/statement.rs

Lines changed: 20 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -790,8 +790,10 @@ impl<S: ContextProvider> SqlToRel<'_, S> {
790790
.collect::<Result<_>>()?;
791791

792792
// Create planner context with parameters
793-
let mut planner_context =
794-
PlannerContext::new().with_prepare_param_data_types(fields.clone());
793+
let mut planner_context = PlannerContext::new()
794+
.with_prepare_param_data_types(
795+
fields.iter().cloned().map(Some).collect(),
796+
);
795797

796798
// Build logical plan for inner statement of the prepare statement
797799
let plan = self.sql_statement_to_plan_with_context_impl(
@@ -808,7 +810,9 @@ impl<S: ContextProvider> SqlToRel<'_, S> {
808810
})
809811
.collect();
810812
fields.extend(param_types.iter().cloned());
811-
planner_context.with_prepare_param_data_types(param_types);
813+
planner_context.with_prepare_param_data_types(
814+
param_types.into_iter().map(Some).collect(),
815+
);
812816
}
813817

814818
Ok(LogicalPlan::Statement(PlanStatement::Prepare(Prepare {
@@ -1341,7 +1345,13 @@ impl<S: ContextProvider> SqlToRel<'_, S> {
13411345
}
13421346
}
13431347
let mut planner_context = PlannerContext::new()
1344-
.with_prepare_param_data_types(arg_types.unwrap_or_default());
1348+
.with_prepare_param_data_types(
1349+
arg_types
1350+
.unwrap_or_default()
1351+
.into_iter()
1352+
.map(Some)
1353+
.collect(),
1354+
);
13451355

13461356
let function_body = match function_body {
13471357
Some(r) => Some(self.sql_to_expr(
@@ -2340,7 +2350,12 @@ impl<S: ContextProvider> SqlToRel<'_, S> {
23402350
}
23412351
}
23422352
}
2343-
let prepare_param_data_types = prepare_param_data_types.into_values().collect();
2353+
let prepare_param_data_types = {
2354+
let len = prepare_param_data_types.keys().last().map_or(0, |&k| k + 1);
2355+
(0..len)
2356+
.map(|i| prepare_param_data_types.remove(&i))
2357+
.collect()
2358+
};
23442359

23452360
// Projection
23462361
let mut planner_context =

datafusion/sql/tests/cases/params.rs

Lines changed: 16 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -1041,6 +1041,22 @@ fn test_prepare_statement_unknown_hash_param() {
10411041
);
10421042
}
10431043

1044+
#[test]
1045+
fn test_insert_infer_with_function_wrapped_placeholder() {
1046+
let plan = logical_plan(
1047+
"INSERT INTO person (id, first_name, age) VALUES ($1, character_length($2), $3)",
1048+
)
1049+
.unwrap();
1050+
1051+
let actual_types = plan.get_parameter_types().unwrap();
1052+
let expected_types: HashMap<String, Option<DataType>> = HashMap::from([
1053+
("$1".to_string(), Some(DataType::UInt32)),
1054+
("$2".to_string(), None),
1055+
("$3".to_string(), Some(DataType::Int32)),
1056+
]);
1057+
assert_eq!(actual_types, expected_types);
1058+
}
1059+
10441060
#[test]
10451061
fn test_prepare_statement_bad_list_idx() {
10461062
let sql = "SELECT id from person where id = $foo";

0 commit comments

Comments
 (0)