risingwave_expr_impl/scalar/
trim_array.rs

1// Copyright 2025 RisingWave Labs
2//
3// Licensed under the Apache License, Version 2.0 (the "License");
4// you may not use this file except in compliance with the License.
5// You may obtain a copy of the License at
6//
7//     http://www.apache.org/licenses/LICENSE-2.0
8//
9// Unless required by applicable law or agreed to in writing, software
10// distributed under the License is distributed on an "AS IS" BASIS,
11// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12// See the License for the specific language governing permissions and
13// limitations under the License.
14
15use risingwave_common::array::{ListRef, ListValue};
16use risingwave_expr::{ExprError, Result, function};
17
18/// Trims an array by removing the last n elements. If the array is multidimensional, only the first
19/// dimension is trimmed.
20///
21/// Examples:
22///
23/// ```slt
24/// query T
25/// select trim_array(array[1,2,3,4,5,null], 4);
26/// ----
27/// {1,2}
28///
29/// query T
30/// select trim_array(array[1,2,3,4,5,null], 0);
31/// ----
32/// {1,2,3,4,5,NULL}
33///
34/// query T
35/// select trim_array(array[1,2,3,4,5,null], null);
36/// ----
37/// NULL
38///
39/// query T
40/// select trim_array(array[1,2,3,4,5,null], null::smallint);
41/// ----
42/// NULL
43///
44/// query T
45/// select trim_array(array[1,2,3,4,5,null], 6);
46/// ----
47/// {}
48///
49/// query T
50/// select trim_array(null::int[], 1);
51/// ----
52/// NULL
53///
54/// statement error
55/// select trim_array(array[1,2,3,4,5,null], 7);
56///
57/// statement error
58/// select trim_array(array[1,2,3,4,5,null], -1);
59///
60/// statement error
61/// select trim_array(array[1,2,3,4,5,null], null::bigint);
62///
63/// statement error
64/// select trim_array(array[1,2,3,4,5,null], 3.14);
65///
66/// statement error
67/// select trim_array(array[1,2,3,4,5,null], array[1]);
68///
69/// statement error
70/// select trim_array(array[1,2,3,4,5,null], true);
71/// ```
72#[function("trim_array(anyarray, int4) -> anyarray")]
73fn trim_array(array: ListRef<'_>, n: i32) -> Result<ListValue> {
74    let values = array.iter();
75    let len_to_trim: usize = n.try_into().map_err(|_| ExprError::InvalidParam {
76        name: "n",
77        reason: "less than zero".into(),
78    })?;
79    let len_to_retain =
80        values
81            .len()
82            .checked_sub(len_to_trim)
83            .ok_or_else(|| ExprError::InvalidParam {
84                name: "n",
85                reason: "more than array length".into(),
86            })?;
87    Ok(ListValue::from_datum_iter(
88        &array.data_type(),
89        values.take(len_to_retain),
90    ))
91}