datafusion_functions_nested/
repeat.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
18//! [`ScalarUDFImpl`] definitions for array_repeat function.
19
20use crate::utils::make_scalar_function;
21use arrow::array::{
22    new_null_array, Array, ArrayRef, Capacities, GenericListArray, ListArray,
23    MutableArrayData, OffsetSizeTrait, UInt64Array,
24};
25use arrow::buffer::OffsetBuffer;
26use arrow::compute;
27use arrow::compute::cast;
28use arrow::datatypes::DataType;
29use arrow::datatypes::{
30    DataType::{LargeList, List},
31    Field,
32};
33use datafusion_common::cast::{as_large_list_array, as_list_array, as_uint64_array};
34use datafusion_common::{exec_err, utils::take_function_args, Result};
35use datafusion_expr::{
36    ColumnarValue, Documentation, ScalarUDFImpl, Signature, Volatility,
37};
38use datafusion_macros::user_doc;
39use std::any::Any;
40use std::sync::Arc;
41
42make_udf_expr_and_func!(
43    ArrayRepeat,
44    array_repeat,
45    element count, // arg name
46    "returns an array containing element `count` times.", // doc
47    array_repeat_udf // internal function name
48);
49
50#[user_doc(
51    doc_section(label = "Array Functions"),
52    description = "Returns an array containing element `count` times.",
53    syntax_example = "array_repeat(element, count)",
54    sql_example = r#"```sql
55> select array_repeat(1, 3);
56+---------------------------------+
57| array_repeat(Int64(1),Int64(3)) |
58+---------------------------------+
59| [1, 1, 1]                       |
60+---------------------------------+
61> select array_repeat([1, 2], 2);
62+------------------------------------+
63| array_repeat(List([1,2]),Int64(2)) |
64+------------------------------------+
65| [[1, 2], [1, 2]]                   |
66+------------------------------------+
67```"#,
68    argument(
69        name = "element",
70        description = "Element expression. Can be a constant, column, or function, and any combination of array operators."
71    ),
72    argument(
73        name = "count",
74        description = "Value of how many times to repeat the element."
75    )
76)]
77#[derive(Debug)]
78pub struct ArrayRepeat {
79    signature: Signature,
80    aliases: Vec<String>,
81}
82
83impl Default for ArrayRepeat {
84    fn default() -> Self {
85        Self::new()
86    }
87}
88
89impl ArrayRepeat {
90    pub fn new() -> Self {
91        Self {
92            signature: Signature::user_defined(Volatility::Immutable),
93            aliases: vec![String::from("list_repeat")],
94        }
95    }
96}
97
98impl ScalarUDFImpl for ArrayRepeat {
99    fn as_any(&self) -> &dyn Any {
100        self
101    }
102
103    fn name(&self) -> &str {
104        "array_repeat"
105    }
106
107    fn signature(&self) -> &Signature {
108        &self.signature
109    }
110
111    fn return_type(&self, arg_types: &[DataType]) -> Result<DataType> {
112        Ok(List(Arc::new(Field::new_list_field(
113            arg_types[0].clone(),
114            true,
115        ))))
116    }
117
118    fn invoke_with_args(
119        &self,
120        args: datafusion_expr::ScalarFunctionArgs,
121    ) -> Result<ColumnarValue> {
122        make_scalar_function(array_repeat_inner)(&args.args)
123    }
124
125    fn aliases(&self) -> &[String] {
126        &self.aliases
127    }
128
129    fn coerce_types(&self, arg_types: &[DataType]) -> Result<Vec<DataType>> {
130        let [first_type, second_type] = take_function_args(self.name(), arg_types)?;
131
132        // Coerce the second argument to Int64/UInt64 if it's a numeric type
133        let second = match second_type {
134            DataType::Int8 | DataType::Int16 | DataType::Int32 | DataType::Int64 => {
135                DataType::Int64
136            }
137            DataType::UInt8 | DataType::UInt16 | DataType::UInt32 | DataType::UInt64 => {
138                DataType::UInt64
139            }
140            _ => return exec_err!("count must be an integer type"),
141        };
142
143        Ok(vec![first_type.clone(), second])
144    }
145
146    fn documentation(&self) -> Option<&Documentation> {
147        self.doc()
148    }
149}
150
151/// Array_repeat SQL function
152pub fn array_repeat_inner(args: &[ArrayRef]) -> Result<ArrayRef> {
153    let element = &args[0];
154    let count_array = &args[1];
155
156    let count_array = match count_array.data_type() {
157        DataType::Int64 => &cast(count_array, &DataType::UInt64)?,
158        DataType::UInt64 => count_array,
159        _ => return exec_err!("count must be an integer type"),
160    };
161
162    let count_array = as_uint64_array(count_array)?;
163
164    match element.data_type() {
165        List(_) => {
166            let list_array = as_list_array(element)?;
167            general_list_repeat::<i32>(list_array, count_array)
168        }
169        LargeList(_) => {
170            let list_array = as_large_list_array(element)?;
171            general_list_repeat::<i64>(list_array, count_array)
172        }
173        _ => general_repeat::<i32>(element, count_array),
174    }
175}
176
177/// For each element of `array[i]` repeat `count_array[i]` times.
178///
179/// Assumption for the input:
180///     1. `count[i] >= 0`
181///     2. `array.len() == count_array.len()`
182///
183/// For example,
184/// ```text
185/// array_repeat(
186///     [1, 2, 3], [2, 0, 1] => [[1, 1], [], [3]]
187/// )
188/// ```
189fn general_repeat<O: OffsetSizeTrait>(
190    array: &ArrayRef,
191    count_array: &UInt64Array,
192) -> Result<ArrayRef> {
193    let data_type = array.data_type();
194    let mut new_values = vec![];
195
196    let count_vec = count_array
197        .values()
198        .to_vec()
199        .iter()
200        .map(|x| *x as usize)
201        .collect::<Vec<_>>();
202
203    for (row_index, &count) in count_vec.iter().enumerate() {
204        let repeated_array = if array.is_null(row_index) {
205            new_null_array(data_type, count)
206        } else {
207            let original_data = array.to_data();
208            let capacity = Capacities::Array(count);
209            let mut mutable =
210                MutableArrayData::with_capacities(vec![&original_data], false, capacity);
211
212            for _ in 0..count {
213                mutable.extend(0, row_index, row_index + 1);
214            }
215
216            let data = mutable.freeze();
217            arrow::array::make_array(data)
218        };
219        new_values.push(repeated_array);
220    }
221
222    let new_values: Vec<_> = new_values.iter().map(|a| a.as_ref()).collect();
223    let values = compute::concat(&new_values)?;
224
225    Ok(Arc::new(GenericListArray::<O>::try_new(
226        Arc::new(Field::new_list_field(data_type.to_owned(), true)),
227        OffsetBuffer::from_lengths(count_vec),
228        values,
229        None,
230    )?))
231}
232
233/// Handle List version of `general_repeat`
234///
235/// For each element of `list_array[i]` repeat `count_array[i]` times.
236///
237/// For example,
238/// ```text
239/// array_repeat(
240///     [[1, 2, 3], [4, 5], [6]], [2, 0, 1] => [[[1, 2, 3], [1, 2, 3]], [], [[6]]]
241/// )
242/// ```
243fn general_list_repeat<O: OffsetSizeTrait>(
244    list_array: &GenericListArray<O>,
245    count_array: &UInt64Array,
246) -> Result<ArrayRef> {
247    let data_type = list_array.data_type();
248    let value_type = list_array.value_type();
249    let mut new_values = vec![];
250
251    let count_vec = count_array
252        .values()
253        .to_vec()
254        .iter()
255        .map(|x| *x as usize)
256        .collect::<Vec<_>>();
257
258    for (list_array_row, &count) in list_array.iter().zip(count_vec.iter()) {
259        let list_arr = match list_array_row {
260            Some(list_array_row) => {
261                let original_data = list_array_row.to_data();
262                let capacity = Capacities::Array(original_data.len() * count);
263                let mut mutable = MutableArrayData::with_capacities(
264                    vec![&original_data],
265                    false,
266                    capacity,
267                );
268
269                for _ in 0..count {
270                    mutable.extend(0, 0, original_data.len());
271                }
272
273                let data = mutable.freeze();
274                let repeated_array = arrow::array::make_array(data);
275
276                let list_arr = GenericListArray::<O>::try_new(
277                    Arc::new(Field::new_list_field(value_type.clone(), true)),
278                    OffsetBuffer::<O>::from_lengths(vec![original_data.len(); count]),
279                    repeated_array,
280                    None,
281                )?;
282                Arc::new(list_arr) as ArrayRef
283            }
284            None => new_null_array(data_type, count),
285        };
286        new_values.push(list_arr);
287    }
288
289    let lengths = new_values.iter().map(|a| a.len()).collect::<Vec<_>>();
290    let new_values: Vec<_> = new_values.iter().map(|a| a.as_ref()).collect();
291    let values = compute::concat(&new_values)?;
292
293    Ok(Arc::new(ListArray::try_new(
294        Arc::new(Field::new_list_field(data_type.to_owned(), true)),
295        OffsetBuffer::<i32>::from_lengths(lengths),
296        values,
297        None,
298    )?))
299}