Skip to content

Commit 1b946f1

Browse files
authored
Merge branch 'main' into broadcast_nested_join_loop_support
2 parents 69fd2bb + da016db commit 1b946f1

12 files changed

Lines changed: 1107 additions & 31 deletions

File tree

docs/source/contributor-guide/spark_expressions_support.md

Lines changed: 384 additions & 0 deletions
Large diffs are not rendered by default.
Lines changed: 379 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,379 @@
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+
// Spark-compatible slice(array, start, length).
19+
//
20+
// Differs from datafusion-spark's SparkSlice in that we correctly return an
21+
// empty array when a negative start position lies before the beginning of the
22+
// array. The upstream implementation (as of datafusion-spark 53.1.0) produces
23+
// the first element instead. Once the upstream is fixed, this can be removed
24+
// in favour of datafusion_spark::function::array::slice::SparkSlice.
25+
26+
use arrow::array::{
27+
make_array, Array, ArrayRef, AsArray, Capacities, GenericListArray, Int64Array,
28+
MutableArrayData, NullBufferBuilder, OffsetSizeTrait,
29+
};
30+
use arrow::buffer::OffsetBuffer;
31+
use arrow::datatypes::{DataType, FieldRef};
32+
use datafusion::common::{cast::as_int64_array, exec_err, utils::take_function_args, Result};
33+
use datafusion::logical_expr::{
34+
ColumnarValue, ReturnFieldArgs, ScalarFunctionArgs, ScalarUDFImpl, Signature, TypeSignature,
35+
Volatility,
36+
};
37+
use std::any::Any;
38+
use std::sync::Arc;
39+
40+
#[derive(Debug, PartialEq, Eq, Hash)]
41+
pub struct SparkArraySlice {
42+
signature: Signature,
43+
}
44+
45+
impl Default for SparkArraySlice {
46+
fn default() -> Self {
47+
Self::new()
48+
}
49+
}
50+
51+
impl SparkArraySlice {
52+
pub fn new() -> Self {
53+
Self {
54+
signature: Signature::new(TypeSignature::Any(3), Volatility::Immutable),
55+
}
56+
}
57+
}
58+
59+
impl ScalarUDFImpl for SparkArraySlice {
60+
fn as_any(&self) -> &dyn Any {
61+
self
62+
}
63+
64+
fn name(&self) -> &str {
65+
"spark_array_slice"
66+
}
67+
68+
fn signature(&self) -> &Signature {
69+
&self.signature
70+
}
71+
72+
fn return_type(&self, _arg_types: &[DataType]) -> Result<DataType> {
73+
datafusion::common::internal_err!("return_field_from_args should be used instead")
74+
}
75+
76+
fn return_field_from_args(&self, args: ReturnFieldArgs) -> Result<FieldRef> {
77+
Ok(Arc::clone(&args.arg_fields[0]))
78+
}
79+
80+
fn invoke_with_args(&self, args: ScalarFunctionArgs) -> Result<ColumnarValue> {
81+
let row_count = args.number_rows;
82+
let arrays = args
83+
.args
84+
.iter()
85+
.map(|arg| match arg {
86+
ColumnarValue::Array(a) => Ok(Arc::clone(a)),
87+
ColumnarValue::Scalar(s) => s.to_array_of_size(row_count),
88+
})
89+
.collect::<Result<Vec<_>>>()?;
90+
let [array, start, length] = take_function_args(self.name(), &arrays)?;
91+
let start = as_int64_array(&start)?;
92+
let length = as_int64_array(&length)?;
93+
94+
let result = match array.data_type() {
95+
DataType::List(_) => slice_list::<i32>(array.as_list::<i32>(), start, length)?,
96+
DataType::LargeList(_) => slice_list::<i64>(array.as_list::<i64>(), start, length)?,
97+
other => {
98+
return exec_err!("{} does not support type '{other}'", self.name());
99+
}
100+
};
101+
Ok(ColumnarValue::Array(result))
102+
}
103+
}
104+
105+
fn slice_list<O: OffsetSizeTrait>(
106+
list_array: &GenericListArray<O>,
107+
start: &Int64Array,
108+
length: &Int64Array,
109+
) -> Result<ArrayRef> {
110+
let list_field = match list_array.data_type() {
111+
DataType::List(field) | DataType::LargeList(field) => field,
112+
other => {
113+
return exec_err!("expected List or LargeList, got {other:?}");
114+
}
115+
};
116+
117+
let values = list_array.values();
118+
let original_data = values.to_data();
119+
let row_count = list_array.len();
120+
let mut offsets = Vec::<O>::with_capacity(row_count + 1);
121+
let mut last_offset = O::zero();
122+
offsets.push(last_offset);
123+
let mut mutable = MutableArrayData::with_capacities(
124+
vec![&original_data],
125+
true,
126+
Capacities::Array(original_data.len()),
127+
);
128+
let mut nulls = NullBufferBuilder::new(row_count);
129+
130+
let row_offsets = list_array.offsets();
131+
let list_nulls = list_array.nulls();
132+
let start_nulls = start.nulls();
133+
let length_nulls = length.nulls();
134+
for row in 0..row_count {
135+
let is_row_null = list_nulls.is_some_and(|n| n.is_null(row))
136+
|| start_nulls.is_some_and(|n| n.is_null(row))
137+
|| length_nulls.is_some_and(|n| n.is_null(row));
138+
if is_row_null {
139+
offsets.push(last_offset);
140+
nulls.append_null();
141+
continue;
142+
}
143+
144+
let start_value = start.value(row);
145+
let length_value = length.value(row);
146+
147+
if start_value == 0 {
148+
return exec_err!("Unexpected value for start in function slice. Expected a positive or negative number, but got 0.");
149+
}
150+
if length_value < 0 {
151+
return exec_err!(
152+
"Unexpected value for length in function slice. Expected a non-negative number, but got {length_value}."
153+
);
154+
}
155+
156+
let row_start = row_offsets[row].as_usize();
157+
let row_end = row_offsets[row + 1].as_usize();
158+
let arr_len = (row_end - row_start) as i64;
159+
160+
let zero_based_start = if start_value > 0 {
161+
start_value - 1
162+
} else {
163+
start_value + arr_len
164+
};
165+
166+
let copied = if zero_based_start < 0 || zero_based_start >= arr_len || length_value == 0 {
167+
0
168+
} else {
169+
let take = std::cmp::min(length_value, arr_len - zero_based_start) as usize;
170+
let begin = row_start + zero_based_start as usize;
171+
mutable.extend(0, begin, begin + take);
172+
take
173+
};
174+
175+
last_offset += O::usize_as(copied);
176+
offsets.push(last_offset);
177+
nulls.append_non_null();
178+
}
179+
180+
Ok(Arc::new(GenericListArray::<O>::try_new(
181+
Arc::clone(list_field),
182+
OffsetBuffer::new(offsets.into()),
183+
make_array(mutable.freeze()),
184+
nulls.finish(),
185+
)?))
186+
}
187+
188+
#[cfg(test)]
189+
mod tests {
190+
use super::*;
191+
use arrow::array::{Int32Array, ListArray};
192+
use arrow::buffer::OffsetBuffer;
193+
use arrow::datatypes::{Field, Int32Type};
194+
195+
fn build_list(rows: Vec<Option<Vec<Option<i32>>>>) -> Arc<ListArray> {
196+
let mut offsets = vec![0i32];
197+
let mut values: Vec<Option<i32>> = Vec::new();
198+
let mut nulls = NullBufferBuilder::new(rows.len());
199+
for row in &rows {
200+
match row {
201+
Some(items) => {
202+
nulls.append_non_null();
203+
values.extend(items.iter().copied());
204+
}
205+
None => nulls.append_null(),
206+
}
207+
offsets.push(values.len() as i32);
208+
}
209+
let values = Arc::new(Int32Array::from(values)) as ArrayRef;
210+
let field = Arc::new(Field::new("item", DataType::Int32, true));
211+
Arc::new(ListArray::new(
212+
field,
213+
OffsetBuffer::new(offsets.into()),
214+
values,
215+
nulls.finish(),
216+
))
217+
}
218+
219+
fn run(
220+
list: Arc<ListArray>,
221+
start: Vec<Option<i64>>,
222+
length: Vec<Option<i64>>,
223+
) -> Vec<Option<Vec<Option<i32>>>> {
224+
let start = Int64Array::from(start);
225+
let length = Int64Array::from(length);
226+
let result = slice_list::<i32>(list.as_ref(), &start, &length).unwrap();
227+
let result = result.as_list::<i32>();
228+
(0..result.len())
229+
.map(|i| {
230+
if result.is_null(i) {
231+
None
232+
} else {
233+
let row = result.value(i);
234+
let row = row.as_primitive::<Int32Type>();
235+
Some(
236+
(0..row.len())
237+
.map(|j| {
238+
if row.is_null(j) {
239+
None
240+
} else {
241+
Some(row.value(j))
242+
}
243+
})
244+
.collect(),
245+
)
246+
}
247+
})
248+
.collect()
249+
}
250+
251+
#[test]
252+
fn positive_start() {
253+
let list = build_list(vec![Some(vec![
254+
Some(1),
255+
Some(2),
256+
Some(3),
257+
Some(4),
258+
Some(5),
259+
])]);
260+
assert_eq!(
261+
run(list, vec![Some(2)], vec![Some(3)]),
262+
vec![Some(vec![Some(2), Some(3), Some(4)])]
263+
);
264+
}
265+
266+
#[test]
267+
fn length_clamped_to_array_end() {
268+
let list = build_list(vec![Some(vec![Some(1), Some(2), Some(3)])]);
269+
assert_eq!(
270+
run(list, vec![Some(2)], vec![Some(100)]),
271+
vec![Some(vec![Some(2), Some(3)])]
272+
);
273+
}
274+
275+
#[test]
276+
fn length_zero_returns_empty() {
277+
let list = build_list(vec![Some(vec![Some(1), Some(2), Some(3)])]);
278+
assert_eq!(
279+
run(list, vec![Some(1)], vec![Some(0)]),
280+
vec![Some(Vec::new())]
281+
);
282+
}
283+
284+
#[test]
285+
fn start_past_end_returns_empty() {
286+
let list = build_list(vec![Some(vec![Some(1), Some(2), Some(3)])]);
287+
assert_eq!(
288+
run(list, vec![Some(10)], vec![Some(1)]),
289+
vec![Some(Vec::new())]
290+
);
291+
}
292+
293+
#[test]
294+
fn negative_start_counts_from_end() {
295+
let list = build_list(vec![Some(vec![
296+
Some(1),
297+
Some(2),
298+
Some(3),
299+
Some(4),
300+
Some(5),
301+
])]);
302+
assert_eq!(
303+
run(list, vec![Some(-2)], vec![Some(2)]),
304+
vec![Some(vec![Some(4), Some(5)])]
305+
);
306+
}
307+
308+
#[test]
309+
fn negative_start_overflows_returns_empty() {
310+
// Spark: slice([a], -2, 2) returns []. datafusion-spark returns [a] here.
311+
let list = build_list(vec![Some(vec![Some(1)])]);
312+
assert_eq!(
313+
run(list, vec![Some(-2)], vec![Some(2)]),
314+
vec![Some(Vec::new())]
315+
);
316+
}
317+
318+
#[test]
319+
fn negative_start_far_below_zero_returns_empty() {
320+
let list = build_list(vec![Some(vec![Some(1), Some(2), Some(3)])]);
321+
assert_eq!(
322+
run(list, vec![Some(-10)], vec![Some(2)]),
323+
vec![Some(Vec::new())]
324+
);
325+
}
326+
327+
#[test]
328+
fn negative_start_with_length_past_end() {
329+
let list = build_list(vec![Some(vec![
330+
Some(1),
331+
Some(2),
332+
Some(3),
333+
Some(4),
334+
Some(5),
335+
])]);
336+
assert_eq!(
337+
run(list, vec![Some(-2)], vec![Some(5)]),
338+
vec![Some(vec![Some(4), Some(5)])]
339+
);
340+
}
341+
342+
#[test]
343+
fn null_inputs_yield_null() {
344+
let list = build_list(vec![None, Some(vec![Some(1)]), Some(vec![Some(1)])]);
345+
assert_eq!(
346+
run(
347+
list,
348+
vec![Some(1), None, Some(1)],
349+
vec![Some(1), Some(1), None]
350+
),
351+
vec![None, None, None]
352+
);
353+
}
354+
355+
#[test]
356+
fn empty_array_input() {
357+
let list = build_list(vec![Some(Vec::new())]);
358+
assert_eq!(
359+
run(list, vec![Some(1)], vec![Some(2)]),
360+
vec![Some(Vec::new())]
361+
);
362+
}
363+
364+
#[test]
365+
fn start_zero_errors() {
366+
let list = build_list(vec![Some(vec![Some(1)])]);
367+
let start = Int64Array::from(vec![Some(0)]);
368+
let length = Int64Array::from(vec![Some(1)]);
369+
assert!(slice_list::<i32>(list.as_ref(), &start, &length).is_err());
370+
}
371+
372+
#[test]
373+
fn negative_length_errors() {
374+
let list = build_list(vec![Some(vec![Some(1)])]);
375+
let start = Int64Array::from(vec![Some(1)]);
376+
let length = Int64Array::from(vec![Some(-1)]);
377+
assert!(slice_list::<i32>(list.as_ref(), &start, &length).is_err());
378+
}
379+
}

native/spark-expr/src/array_funcs/mod.rs

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -18,6 +18,7 @@
1818
mod array_compact;
1919
mod array_insert;
2020
mod array_position;
21+
mod array_slice;
2122
mod arrays_overlap;
2223
mod arrays_zip;
2324
mod get_array_struct_fields;
@@ -27,6 +28,7 @@ mod size;
2728
pub use array_compact::SparkArrayCompact;
2829
pub use array_insert::ArrayInsert;
2930
pub use array_position::SparkArrayPositionFunc;
31+
pub use array_slice::SparkArraySlice;
3032
pub use arrays_overlap::SparkArraysOverlap;
3133
pub use arrays_zip::SparkArraysZipFunc;
3234
pub use get_array_struct_fields::GetArrayStructFields;

0 commit comments

Comments
 (0)