datafusion_functions/core/
struct.rs

1// Licensed to the Apache Software Foundation (ASF) under one
2// or more contributor license agreements.  See the NOTICE file
3// distributed with this work for additional information
4// regarding copyright ownership.  The ASF licenses this file
5// to you under the Apache License, Version 2.0 (the
6// "License"); you may not use this file except in compliance
7// with the License.  You may obtain a copy of the License at
8//
9//   http://www.apache.org/licenses/LICENSE-2.0
10//
11// Unless required by applicable law or agreed to in writing,
12// software distributed under the License is distributed on an
13// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
14// KIND, either express or implied.  See the License for the
15// specific language governing permissions and limitations
16// under the License.
17
18use arrow::array::StructArray;
19use arrow::datatypes::{DataType, Field};
20use datafusion_common::{exec_err, internal_err, Result};
21use datafusion_expr::{ColumnarValue, Documentation, ScalarFunctionArgs};
22use datafusion_expr::{ScalarUDFImpl, Signature, Volatility};
23use datafusion_macros::user_doc;
24use std::any::Any;
25use std::sync::Arc;
26
27#[user_doc(
28    doc_section(label = "Struct Functions"),
29    description = "Returns an Arrow struct using the specified input expressions optionally named.
30Fields in the returned struct use the optional name or the `cN` naming convention.
31For example: `c0`, `c1`, `c2`, etc.",
32    syntax_example = "struct(expression1[, ..., expression_n])",
33    sql_example = r#"For example, this query converts two columns `a` and `b` to a single column with
34a struct type of fields `field_a` and `c1`:
35```sql
36> select * from t;
37+---+---+
38| a | b |
39+---+---+
40| 1 | 2 |
41| 3 | 4 |
42+---+---+
43
44-- use default names `c0`, `c1`
45> select struct(a, b) from t;
46+-----------------+
47| struct(t.a,t.b) |
48+-----------------+
49| {c0: 1, c1: 2}  |
50| {c0: 3, c1: 4}  |
51+-----------------+
52
53-- name the first field `field_a`
54select struct(a as field_a, b) from t;
55+--------------------------------------------------+
56| named_struct(Utf8("field_a"),t.a,Utf8("c1"),t.b) |
57+--------------------------------------------------+
58| {field_a: 1, c1: 2}                              |
59| {field_a: 3, c1: 4}                              |
60+--------------------------------------------------+
61```"#,
62    argument(
63        name = "expression1, expression_n",
64        description = "Expression to include in the output struct. Can be a constant, column, or function, any combination of arithmetic or string operators."
65    )
66)]
67#[derive(Debug)]
68pub struct StructFunc {
69    signature: Signature,
70    aliases: Vec<String>,
71}
72
73impl Default for StructFunc {
74    fn default() -> Self {
75        Self::new()
76    }
77}
78
79impl StructFunc {
80    pub fn new() -> Self {
81        Self {
82            signature: Signature::variadic_any(Volatility::Immutable),
83            aliases: vec![String::from("row")],
84        }
85    }
86}
87
88impl ScalarUDFImpl for StructFunc {
89    fn as_any(&self) -> &dyn Any {
90        self
91    }
92    fn name(&self) -> &str {
93        "struct"
94    }
95
96    fn aliases(&self) -> &[String] {
97        &self.aliases
98    }
99
100    fn signature(&self) -> &Signature {
101        &self.signature
102    }
103
104    fn return_type(&self, arg_types: &[DataType]) -> Result<DataType> {
105        if arg_types.is_empty() {
106            return exec_err!("struct requires at least one argument, got 0 instead");
107        }
108
109        let fields = arg_types
110            .iter()
111            .enumerate()
112            .map(|(pos, dt)| Field::new(format!("c{pos}"), dt.clone(), true))
113            .collect::<Vec<Field>>()
114            .into();
115
116        Ok(DataType::Struct(fields))
117    }
118
119    fn invoke_with_args(&self, args: ScalarFunctionArgs) -> Result<ColumnarValue> {
120        let DataType::Struct(fields) = args.return_type else {
121            return internal_err!("incorrect struct return type");
122        };
123
124        assert_eq!(
125            fields.len(),
126            args.args.len(),
127            "return type field count != argument count"
128        );
129
130        let arrays = ColumnarValue::values_to_arrays(&args.args)?;
131        Ok(ColumnarValue::Array(Arc::new(StructArray::new(
132            fields.clone(),
133            arrays,
134            None,
135        ))))
136    }
137
138    fn documentation(&self) -> Option<&Documentation> {
139        self.doc()
140    }
141}