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(
102 ctx.arg_types[0].as_list_element_type(),
103 left.iter().chain(right.iter()),
104 )
105 } else if ctx.arg_types[0].as_list_element_type() == &ctx.arg_types[1] {
106 // array[] || array
107 let Some(right) = right else {
108 return left.map(|left| left.to_owned_scalar());
109 };
110 ListValue::from_datum_iter(
111 &ctx.arg_types[1],
112 left.iter()
113 .flat_map(|list| list.iter())
114 .chain([Some(right.into())]),
115 )
116 } else if &ctx.arg_types[0] == ctx.arg_types[1].as_list_element_type() {
117 // array || array[]
118 let Some(left) = left else {
119 return right.map(|right| right.to_owned_scalar());
120 };
121 ListValue::from_datum_iter(
122 &ctx.arg_types[0],
123 [Some(left.into())]
124 .into_iter()
125 .chain(right.iter().flat_map(|list| list.iter())),
126 )
127 } else {
128 unreachable!()
129 })
130}
131
132/// Appends a value as the back element of an array.
133/// The behavior is the same as PG.
134///
135/// Examples:
136///
137/// ```slt
138/// query T
139/// select array_append(array[66], 123);
140/// ----
141/// {66,123}
142///
143/// query T
144/// select array_append(array[66], null::int);
145/// ----
146/// {66,NULL}
147///
148/// query T
149/// select array_append(null::int[], 233);
150/// ----
151/// {233}
152///
153/// query T
154/// select array_append(null::int[], null::int);
155/// ----
156/// {NULL}
157/// ```
158#[function("array_append(anyarray, any) -> anyarray")]
159fn array_append(
160 left: Option<ListRef<'_>>,
161 right: Option<ScalarRefImpl<'_>>,
162 ctx: &Context,
163) -> ListValue {
164 ListValue::from_datum_iter(
165 &ctx.arg_types[1],
166 left.iter()
167 .flat_map(|list| list.iter())
168 .chain(std::iter::once(right)),
169 )
170}
171
172/// Prepends a value as the front element of an array.
173/// The behavior is the same as PG.
174///
175/// Examples:
176///
177/// ```slt
178/// query T
179/// select array_prepend(123, array[66]);
180/// ----
181/// {123,66}
182///
183/// query T
184/// select array_prepend(null::int, array[66]);
185/// ----
186/// {NULL,66}
187///
188/// query T
189/// select array_prepend(233, null::int[]);
190/// ----
191/// {233}
192///
193/// query T
194/// select array_prepend(null::int, null::int[]);
195/// ----
196/// {NULL}
197/// ```
198#[function("array_prepend(any, anyarray) -> anyarray")]
199fn array_prepend(
200 left: Option<ScalarRefImpl<'_>>,
201 right: Option<ListRef<'_>>,
202 ctx: &Context,
203) -> ListValue {
204 ListValue::from_datum_iter(
205 &ctx.arg_types[0],
206 std::iter::once(left).chain(right.iter().flat_map(|list| list.iter())),
207 )
208}