Skip to main content

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::{Result, exec_err, internal_err};
21use datafusion_expr::{ColumnarValue, Documentation, ScalarFunctionArgs};
22use datafusion_expr::{ScalarUDFImpl, Signature, Volatility};
23use datafusion_macros::user_doc;
24use std::sync::Arc;
25
26#[user_doc(
27    doc_section(label = "Struct Functions"),
28    description = "Returns an Arrow struct using the specified input expressions optionally named.
29Fields in the returned struct use the optional name or the `cN` naming convention.
30For example: `c0`, `c1`, `c2`, etc.
31For information on comparing and ordering struct values (including `NULL` handling),
32see [Comparison and Ordering](struct_coercion.md#comparison-and-ordering).",
33    syntax_example = "struct(expression1[, ..., expression_n])",
34    sql_example = r#"For example, this query converts two columns `a` and `b` to a single column with
35a struct type of fields `field_a` and `c1`:
36```sql
37> select * from t;
38+---+---+
39| a | b |
40+---+---+
41| 1 | 2 |
42| 3 | 4 |
43+---+---+
44
45-- use default names `c0`, `c1`
46> select struct(a, b) from t;
47+-----------------+
48| struct(t.a,t.b) |
49+-----------------+
50| {c0: 1, c1: 2}  |
51| {c0: 3, c1: 4}  |
52+-----------------+
53
54-- name the first field `field_a`
55select struct(a as field_a, b) from t;
56+--------------------------------------------------+
57| named_struct(Utf8("field_a"),t.a,Utf8("c1"),t.b) |
58+--------------------------------------------------+
59| {field_a: 1, c1: 2}                              |
60| {field_a: 3, c1: 4}                              |
61+--------------------------------------------------+
62```"#,
63    argument(
64        name = "expression1, expression_n",
65        description = "Expression to include in the output struct. Can be a constant, column, or function, any combination of arithmetic or string operators."
66    )
67)]
68#[derive(Debug, PartialEq, Eq, Hash)]
69pub struct StructFunc {
70    signature: Signature,
71    aliases: Vec<String>,
72}
73
74impl Default for StructFunc {
75    fn default() -> Self {
76        Self::new()
77    }
78}
79
80impl StructFunc {
81    pub fn new() -> Self {
82        Self {
83            signature: Signature::variadic_any(Volatility::Immutable),
84            aliases: vec![String::from("row")],
85        }
86    }
87}
88
89impl ScalarUDFImpl for StructFunc {
90    fn name(&self) -> &str {
91        "struct"
92    }
93
94    fn aliases(&self) -> &[String] {
95        &self.aliases
96    }
97
98    fn signature(&self) -> &Signature {
99        &self.signature
100    }
101
102    fn return_type(&self, arg_types: &[DataType]) -> Result<DataType> {
103        if arg_types.is_empty() {
104            return exec_err!("struct requires at least one argument, got 0 instead");
105        }
106
107        let fields = arg_types
108            .iter()
109            .enumerate()
110            .map(|(pos, dt)| Field::new(format!("c{pos}"), dt.clone(), true))
111            .collect::<Vec<Field>>()
112            .into();
113
114        Ok(DataType::Struct(fields))
115    }
116
117    fn invoke_with_args(&self, args: ScalarFunctionArgs) -> Result<ColumnarValue> {
118        let DataType::Struct(fields) = args.return_type() else {
119            return internal_err!("incorrect struct return type");
120        };
121
122        assert_eq!(
123            fields.len(),
124            args.args.len(),
125            "return type field count != argument count"
126        );
127
128        let arrays = ColumnarValue::values_to_arrays(&args.args)?;
129        Ok(ColumnarValue::Array(Arc::new(StructArray::new(
130            fields.clone(),
131            arrays,
132            None,
133        ))))
134    }
135
136    fn documentation(&self) -> Option<&Documentation> {
137        self.doc()
138    }
139}