datafusion_functions/core/
named_struct.rs1use arrow::array::StructArray;
19use arrow::datatypes::{DataType, Field, Fields};
20use datafusion_common::{exec_err, internal_err, Result};
21use datafusion_expr::{
22 ColumnarValue, Documentation, ReturnInfo, ReturnTypeArgs, ScalarFunctionArgs,
23};
24use datafusion_expr::{ScalarUDFImpl, Signature, Volatility};
25use datafusion_macros::user_doc;
26use std::any::Any;
27use std::sync::Arc;
28
29#[user_doc(
30 doc_section(label = "Struct Functions"),
31 description = "Returns an Arrow struct using the specified name and input expressions pairs.",
32 syntax_example = "named_struct(expression1_name, expression1_input[, ..., expression_n_name, expression_n_input])",
33 sql_example = r#"
34For example, this query converts two columns `a` and `b` to a single column with
35a struct type of fields `field_a` and `field_b`:
36```sql
37> select * from t;
38+---+---+
39| a | b |
40+---+---+
41| 1 | 2 |
42| 3 | 4 |
43+---+---+
44> select named_struct('field_a', a, 'field_b', b) from t;
45+-------------------------------------------------------+
46| named_struct(Utf8("field_a"),t.a,Utf8("field_b"),t.b) |
47+-------------------------------------------------------+
48| {field_a: 1, field_b: 2} |
49| {field_a: 3, field_b: 4} |
50+-------------------------------------------------------+
51```"#,
52 argument(
53 name = "expression_n_name",
54 description = "Name of the column field. Must be a constant string."
55 ),
56 argument(
57 name = "expression_n_input",
58 description = "Expression to include in the output struct. Can be a constant, column, or function, and any combination of arithmetic or string operators."
59 )
60)]
61#[derive(Debug)]
62pub struct NamedStructFunc {
63 signature: Signature,
64}
65
66impl Default for NamedStructFunc {
67 fn default() -> Self {
68 Self::new()
69 }
70}
71
72impl NamedStructFunc {
73 pub fn new() -> Self {
74 Self {
75 signature: Signature::variadic_any(Volatility::Immutable),
76 }
77 }
78}
79
80impl ScalarUDFImpl for NamedStructFunc {
81 fn as_any(&self) -> &dyn Any {
82 self
83 }
84
85 fn name(&self) -> &str {
86 "named_struct"
87 }
88
89 fn signature(&self) -> &Signature {
90 &self.signature
91 }
92
93 fn return_type(&self, _arg_types: &[DataType]) -> Result<DataType> {
94 internal_err!("named_struct: return_type called instead of return_type_from_args")
95 }
96
97 fn return_type_from_args(&self, args: ReturnTypeArgs) -> Result<ReturnInfo> {
98 if args.scalar_arguments.is_empty() {
100 return exec_err!(
101 "named_struct requires at least one pair of arguments, got 0 instead"
102 );
103 }
104
105 if args.scalar_arguments.len() % 2 != 0 {
106 return exec_err!(
107 "named_struct requires an even number of arguments, got {} instead",
108 args.scalar_arguments.len()
109 );
110 }
111
112 let names = args
113 .scalar_arguments
114 .iter()
115 .enumerate()
116 .step_by(2)
117 .map(|(i, sv)|
118 sv.and_then(|sv| sv.try_as_str().flatten().filter(|s| !s.is_empty()))
119 .map_or_else(
120 ||
121 exec_err!(
122 "{} requires {i}-th (0-indexed) field name as non-empty constant string",
123 self.name()
124 ),
125 Ok
126 )
127 )
128 .collect::<Result<Vec<_>>>()?;
129 let types = args.arg_types.iter().skip(1).step_by(2).collect::<Vec<_>>();
130
131 let return_fields = names
132 .into_iter()
133 .zip(types.into_iter())
134 .map(|(name, data_type)| Ok(Field::new(name, data_type.to_owned(), true)))
135 .collect::<Result<Vec<Field>>>()?;
136
137 Ok(ReturnInfo::new_nullable(DataType::Struct(Fields::from(
138 return_fields,
139 ))))
140 }
141
142 fn invoke_with_args(&self, args: ScalarFunctionArgs) -> Result<ColumnarValue> {
143 let DataType::Struct(fields) = args.return_type else {
144 return internal_err!("incorrect named_struct return type");
145 };
146
147 assert_eq!(
148 fields.len(),
149 args.args.len() / 2,
150 "return type field count != argument count / 2"
151 );
152
153 let values: Vec<ColumnarValue> = args
154 .args
155 .chunks_exact(2)
156 .map(|chunk| chunk[1].clone())
157 .collect();
158 let arrays = ColumnarValue::values_to_arrays(&values)?;
159 Ok(ColumnarValue::Array(Arc::new(StructArray::new(
160 fields.clone(),
161 arrays,
162 None,
163 ))))
164 }
165
166 fn documentation(&self) -> Option<&Documentation> {
167 self.doc()
168 }
169}