risingwave_expr_impl/scalar/array_concat.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_common::types::{ScalarRef, ScalarRefImpl};
17use risingwave_expr::expr::Context;
18use risingwave_expr::function;
19
20/// Concatenates the two arrays.
21///
22/// Examples:
23///
24/// ```slt
25/// # concat
26/// query T
27/// select array_cat(array[66], array[123]);
28/// ----
29/// {66,123}
30///
31/// query T
32/// select array_cat(array[66], null::int[]);
33/// ----
34/// {66}
35///
36/// query T
37/// select array_cat(null::int[], array[123]);
38/// ----
39/// {123}
40///
41/// query T
42/// select array_cat(null::int[], null::int[]);
43/// ----
44/// NULL
45///
46/// # append
47/// query T
48/// select array_cat(array[array[66]], array[233]);
49/// ----
50/// {{66},{233}}
51///
52/// query T
53/// select array_cat(array[array[66]], null::int[]);
54/// ----
55/// {{66}}
56///
57/// # different from PG
58/// query T
59/// select array_cat(null::int[][], array[233]);
60/// ----
61/// {{233}}
62///
63/// query T
64/// select array_cat(null::int[][], null::int[]);
65/// ----
66/// NULL
67///
68/// # prepend
69/// query T
70/// select array_cat(array[233], array[array[66]]);
71/// ----
72/// {{233},{66}}
73///
74/// query T
75/// select array_cat(null::int[], array[array[66]]);
76/// ----
77/// {{66}}
78///
79/// # different from PG
80/// query T
81/// select array_cat(array[233], null::int[][]);
82/// ----
83/// {{233}}
84///
85/// query T
86/// select array_cat(null::int[], null::int[][]);
87/// ----
88/// NULL
89/// ```
90#[function("array_cat(anyarray, anyarray) -> anyarray")]
91fn array_cat(
92 left: Option<ListRef<'_>>,
93 right: Option<ListRef<'_>>,
94 ctx: &Context,
95) -> Option<ListValue> {
96 Some(if ctx.arg_types[0] == ctx.arg_types[1] {
97 // array || array
98 let (Some(left), Some(right)) = (left, right) else {
99 return left.or(right).map(|list| list.to_owned_scalar());
100 };
101 ListValue::from_datum_iter(ctx.arg_types[0].as_list(), left.iter().chain(right.iter()))
102 } else if ctx.arg_types[0].as_list() == &ctx.arg_types[1] {
103 // array[] || array
104 let Some(right) = right else {
105 return left.map(|left| left.to_owned_scalar());
106 };
107 ListValue::from_datum_iter(
108 &ctx.arg_types[1],
109 left.iter()
110 .flat_map(|list| list.iter())
111 .chain([Some(right.into())]),
112 )
113 } else if &ctx.arg_types[0] == ctx.arg_types[1].as_list() {
114 // array || array[]
115 let Some(left) = left else {
116 return right.map(|right| right.to_owned_scalar());
117 };
118 ListValue::from_datum_iter(
119 &ctx.arg_types[0],
120 [Some(left.into())]
121 .into_iter()
122 .chain(right.iter().flat_map(|list| list.iter())),
123 )
124 } else {
125 unreachable!()
126 })
127}
128
129/// Appends a value as the back element of an array.
130/// The behavior is the same as PG.
131///
132/// Examples:
133///
134/// ```slt
135/// query T
136/// select array_append(array[66], 123);
137/// ----
138/// {66,123}
139///
140/// query T
141/// select array_append(array[66], null::int);
142/// ----
143/// {66,NULL}
144///
145/// query T
146/// select array_append(null::int[], 233);
147/// ----
148/// {233}
149///
150/// query T
151/// select array_append(null::int[], null::int);
152/// ----
153/// {NULL}
154/// ```
155#[function("array_append(anyarray, any) -> anyarray")]
156fn array_append(
157 left: Option<ListRef<'_>>,
158 right: Option<ScalarRefImpl<'_>>,
159 ctx: &Context,
160) -> ListValue {
161 ListValue::from_datum_iter(
162 &ctx.arg_types[1],
163 left.iter()
164 .flat_map(|list| list.iter())
165 .chain(std::iter::once(right)),
166 )
167}
168
169/// Prepends a value as the front element of an array.
170/// The behavior is the same as PG.
171///
172/// Examples:
173///
174/// ```slt
175/// query T
176/// select array_prepend(123, array[66]);
177/// ----
178/// {123,66}
179///
180/// query T
181/// select array_prepend(null::int, array[66]);
182/// ----
183/// {NULL,66}
184///
185/// query T
186/// select array_prepend(233, null::int[]);
187/// ----
188/// {233}
189///
190/// query T
191/// select array_prepend(null::int, null::int[]);
192/// ----
193/// {NULL}
194/// ```
195#[function("array_prepend(any, anyarray) -> anyarray")]
196fn array_prepend(
197 left: Option<ScalarRefImpl<'_>>,
198 right: Option<ListRef<'_>>,
199 ctx: &Context,
200) -> ListValue {
201 ListValue::from_datum_iter(
202 &ctx.arg_types[0],
203 std::iter::once(left).chain(right.iter().flat_map(|list| list.iter())),
204 )
205}