/Users/andrewlamb/Software/datafusion/datafusion/physical-plan/src/unnest.rs
Line | Count | Source (jump to first uncovered line) |
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 | | //! Define a plan for unnesting values in columns that contain a list type. |
19 | | |
20 | | use std::cmp::{self, Ordering}; |
21 | | use std::collections::HashMap; |
22 | | use std::{any::Any, sync::Arc}; |
23 | | |
24 | | use super::metrics::{self, ExecutionPlanMetricsSet, MetricBuilder, MetricsSet}; |
25 | | use super::{DisplayAs, ExecutionPlanProperties, PlanProperties}; |
26 | | use crate::{ |
27 | | DisplayFormatType, Distribution, ExecutionPlan, RecordBatchStream, |
28 | | SendableRecordBatchStream, |
29 | | }; |
30 | | |
31 | | use arrow::array::{ |
32 | | Array, ArrayRef, AsArray, FixedSizeListArray, LargeListArray, ListArray, |
33 | | PrimitiveArray, |
34 | | }; |
35 | | use arrow::compute::kernels::length::length; |
36 | | use arrow::compute::kernels::zip::zip; |
37 | | use arrow::compute::{cast, is_not_null, kernels, sum}; |
38 | | use arrow::datatypes::{DataType, Int64Type, Schema, SchemaRef}; |
39 | | use arrow::record_batch::RecordBatch; |
40 | | use arrow_array::{Int64Array, Scalar, StructArray}; |
41 | | use arrow_ord::cmp::lt; |
42 | | use datafusion_common::{ |
43 | | exec_datafusion_err, exec_err, internal_err, Result, UnnestOptions, |
44 | | }; |
45 | | use datafusion_execution::TaskContext; |
46 | | use datafusion_physical_expr::EquivalenceProperties; |
47 | | |
48 | | use async_trait::async_trait; |
49 | | use futures::{Stream, StreamExt}; |
50 | | use hashbrown::HashSet; |
51 | | use log::trace; |
52 | | |
53 | | /// Unnest the given columns (either with type struct or list) |
54 | | /// For list unnesting, each rows is vertically transformed into multiple rows |
55 | | /// For struct unnesting, each columns is horizontally transformed into multiple columns, |
56 | | /// Thus the original RecordBatch with dimension (n x m) may have new dimension (n' x m') |
57 | | /// |
58 | | /// See [`UnnestOptions`] for more details and an example. |
59 | | #[derive(Debug)] |
60 | | pub struct UnnestExec { |
61 | | /// Input execution plan |
62 | | input: Arc<dyn ExecutionPlan>, |
63 | | /// The schema once the unnest is applied |
64 | | schema: SchemaRef, |
65 | | /// indices of the list-typed columns in the input schema |
66 | | list_column_indices: Vec<ListUnnest>, |
67 | | /// indices of the struct-typed columns in the input schema |
68 | | struct_column_indices: Vec<usize>, |
69 | | /// Options |
70 | | options: UnnestOptions, |
71 | | /// Execution metrics |
72 | | metrics: ExecutionPlanMetricsSet, |
73 | | /// Cache holding plan properties like equivalences, output partitioning etc. |
74 | | cache: PlanProperties, |
75 | | } |
76 | | |
77 | | impl UnnestExec { |
78 | | /// Create a new [UnnestExec]. |
79 | 0 | pub fn new( |
80 | 0 | input: Arc<dyn ExecutionPlan>, |
81 | 0 | list_column_indices: Vec<ListUnnest>, |
82 | 0 | struct_column_indices: Vec<usize>, |
83 | 0 | schema: SchemaRef, |
84 | 0 | options: UnnestOptions, |
85 | 0 | ) -> Self { |
86 | 0 | let cache = Self::compute_properties(&input, Arc::clone(&schema)); |
87 | 0 |
|
88 | 0 | UnnestExec { |
89 | 0 | input, |
90 | 0 | schema, |
91 | 0 | list_column_indices, |
92 | 0 | struct_column_indices, |
93 | 0 | options, |
94 | 0 | metrics: Default::default(), |
95 | 0 | cache, |
96 | 0 | } |
97 | 0 | } |
98 | | |
99 | | /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. |
100 | 0 | fn compute_properties( |
101 | 0 | input: &Arc<dyn ExecutionPlan>, |
102 | 0 | schema: SchemaRef, |
103 | 0 | ) -> PlanProperties { |
104 | 0 | let eq_properties = EquivalenceProperties::new(schema); |
105 | 0 |
|
106 | 0 | PlanProperties::new( |
107 | 0 | eq_properties, |
108 | 0 | input.output_partitioning().clone(), |
109 | 0 | input.execution_mode(), |
110 | 0 | ) |
111 | 0 | } |
112 | | |
113 | | /// Input execution plan |
114 | 0 | pub fn input(&self) -> &Arc<dyn ExecutionPlan> { |
115 | 0 | &self.input |
116 | 0 | } |
117 | | |
118 | | /// indices of the list-typed columns in the input schema |
119 | 0 | pub fn list_column_indices(&self) -> &[ListUnnest] { |
120 | 0 | &self.list_column_indices |
121 | 0 | } |
122 | | |
123 | | /// indices of the struct-typed columns in the input schema |
124 | 0 | pub fn struct_column_indices(&self) -> &[usize] { |
125 | 0 | &self.struct_column_indices |
126 | 0 | } |
127 | | |
128 | 0 | pub fn options(&self) -> &UnnestOptions { |
129 | 0 | &self.options |
130 | 0 | } |
131 | | } |
132 | | |
133 | | impl DisplayAs for UnnestExec { |
134 | 0 | fn fmt_as( |
135 | 0 | &self, |
136 | 0 | t: DisplayFormatType, |
137 | 0 | f: &mut std::fmt::Formatter, |
138 | 0 | ) -> std::fmt::Result { |
139 | 0 | match t { |
140 | | DisplayFormatType::Default | DisplayFormatType::Verbose => { |
141 | 0 | write!(f, "UnnestExec") |
142 | 0 | } |
143 | 0 | } |
144 | 0 | } |
145 | | } |
146 | | |
147 | | impl ExecutionPlan for UnnestExec { |
148 | 0 | fn name(&self) -> &'static str { |
149 | 0 | "UnnestExec" |
150 | 0 | } |
151 | | |
152 | 0 | fn as_any(&self) -> &dyn Any { |
153 | 0 | self |
154 | 0 | } |
155 | | |
156 | 0 | fn properties(&self) -> &PlanProperties { |
157 | 0 | &self.cache |
158 | 0 | } |
159 | | |
160 | 0 | fn children(&self) -> Vec<&Arc<dyn ExecutionPlan>> { |
161 | 0 | vec![&self.input] |
162 | 0 | } |
163 | | |
164 | 0 | fn with_new_children( |
165 | 0 | self: Arc<Self>, |
166 | 0 | children: Vec<Arc<dyn ExecutionPlan>>, |
167 | 0 | ) -> Result<Arc<dyn ExecutionPlan>> { |
168 | 0 | Ok(Arc::new(UnnestExec::new( |
169 | 0 | Arc::clone(&children[0]), |
170 | 0 | self.list_column_indices.clone(), |
171 | 0 | self.struct_column_indices.clone(), |
172 | 0 | Arc::clone(&self.schema), |
173 | 0 | self.options.clone(), |
174 | 0 | ))) |
175 | 0 | } |
176 | | |
177 | 0 | fn required_input_distribution(&self) -> Vec<Distribution> { |
178 | 0 | vec![Distribution::UnspecifiedDistribution] |
179 | 0 | } |
180 | | |
181 | 0 | fn execute( |
182 | 0 | &self, |
183 | 0 | partition: usize, |
184 | 0 | context: Arc<TaskContext>, |
185 | 0 | ) -> Result<SendableRecordBatchStream> { |
186 | 0 | let input = self.input.execute(partition, context)?; |
187 | 0 | let metrics = UnnestMetrics::new(partition, &self.metrics); |
188 | 0 |
|
189 | 0 | Ok(Box::pin(UnnestStream { |
190 | 0 | input, |
191 | 0 | schema: Arc::clone(&self.schema), |
192 | 0 | list_type_columns: self.list_column_indices.clone(), |
193 | 0 | struct_column_indices: self.struct_column_indices.iter().copied().collect(), |
194 | 0 | options: self.options.clone(), |
195 | 0 | metrics, |
196 | 0 | })) |
197 | 0 | } |
198 | | |
199 | 0 | fn metrics(&self) -> Option<MetricsSet> { |
200 | 0 | Some(self.metrics.clone_inner()) |
201 | 0 | } |
202 | | } |
203 | | |
204 | | #[derive(Clone, Debug)] |
205 | | struct UnnestMetrics { |
206 | | /// total time for column unnesting |
207 | | elapsed_compute: metrics::Time, |
208 | | /// Number of batches consumed |
209 | | input_batches: metrics::Count, |
210 | | /// Number of rows consumed |
211 | | input_rows: metrics::Count, |
212 | | /// Number of batches produced |
213 | | output_batches: metrics::Count, |
214 | | /// Number of rows produced by this operator |
215 | | output_rows: metrics::Count, |
216 | | } |
217 | | |
218 | | impl UnnestMetrics { |
219 | 0 | fn new(partition: usize, metrics: &ExecutionPlanMetricsSet) -> Self { |
220 | 0 | let elapsed_compute = MetricBuilder::new(metrics).elapsed_compute(partition); |
221 | 0 |
|
222 | 0 | let input_batches = |
223 | 0 | MetricBuilder::new(metrics).counter("input_batches", partition); |
224 | 0 |
|
225 | 0 | let input_rows = MetricBuilder::new(metrics).counter("input_rows", partition); |
226 | 0 |
|
227 | 0 | let output_batches = |
228 | 0 | MetricBuilder::new(metrics).counter("output_batches", partition); |
229 | 0 |
|
230 | 0 | let output_rows = MetricBuilder::new(metrics).output_rows(partition); |
231 | 0 |
|
232 | 0 | Self { |
233 | 0 | input_batches, |
234 | 0 | input_rows, |
235 | 0 | output_batches, |
236 | 0 | output_rows, |
237 | 0 | elapsed_compute, |
238 | 0 | } |
239 | 0 | } |
240 | | } |
241 | | |
242 | | /// A stream that issues [RecordBatch]es with unnested column data. |
243 | | struct UnnestStream { |
244 | | /// Input stream |
245 | | input: SendableRecordBatchStream, |
246 | | /// Unnested schema |
247 | | schema: Arc<Schema>, |
248 | | /// represents all unnest operations to be applied to the input (input index, depth) |
249 | | /// e.g unnest(col1),unnest(unnest(col1)) where col1 has index 1 in original input schema |
250 | | /// then list_type_columns = [ListUnnest{1,1},ListUnnest{1,2}] |
251 | | list_type_columns: Vec<ListUnnest>, |
252 | | struct_column_indices: HashSet<usize>, |
253 | | /// Options |
254 | | options: UnnestOptions, |
255 | | /// Metrics |
256 | | metrics: UnnestMetrics, |
257 | | } |
258 | | |
259 | | impl RecordBatchStream for UnnestStream { |
260 | 0 | fn schema(&self) -> SchemaRef { |
261 | 0 | Arc::clone(&self.schema) |
262 | 0 | } |
263 | | } |
264 | | |
265 | | #[async_trait] |
266 | | impl Stream for UnnestStream { |
267 | | type Item = Result<RecordBatch>; |
268 | | |
269 | 0 | fn poll_next( |
270 | 0 | mut self: std::pin::Pin<&mut Self>, |
271 | 0 | cx: &mut std::task::Context<'_>, |
272 | 0 | ) -> std::task::Poll<Option<Self::Item>> { |
273 | 0 | self.poll_next_impl(cx) |
274 | 0 | } |
275 | | } |
276 | | |
277 | | impl UnnestStream { |
278 | | /// Separate implementation function that unpins the [`UnnestStream`] so |
279 | | /// that partial borrows work correctly |
280 | 0 | fn poll_next_impl( |
281 | 0 | &mut self, |
282 | 0 | cx: &mut std::task::Context<'_>, |
283 | 0 | ) -> std::task::Poll<Option<Result<RecordBatch>>> { |
284 | 0 | self.input |
285 | 0 | .poll_next_unpin(cx) |
286 | 0 | .map(|maybe_batch| match maybe_batch { |
287 | 0 | Some(Ok(batch)) => { |
288 | 0 | let timer = self.metrics.elapsed_compute.timer(); |
289 | 0 | let result = build_batch( |
290 | 0 | &batch, |
291 | 0 | &self.schema, |
292 | 0 | &self.list_type_columns, |
293 | 0 | &self.struct_column_indices, |
294 | 0 | &self.options, |
295 | 0 | ); |
296 | 0 | self.metrics.input_batches.add(1); |
297 | 0 | self.metrics.input_rows.add(batch.num_rows()); |
298 | 0 | if let Ok(ref batch) = result { |
299 | 0 | timer.done(); |
300 | 0 | self.metrics.output_batches.add(1); |
301 | 0 | self.metrics.output_rows.add(batch.num_rows()); |
302 | 0 | } |
303 | | |
304 | 0 | Some(result) |
305 | | } |
306 | 0 | other => { |
307 | 0 | trace!( |
308 | 0 | "Processed {} probe-side input batches containing {} rows and \ |
309 | 0 | produced {} output batches containing {} rows in {}", |
310 | | self.metrics.input_batches, |
311 | | self.metrics.input_rows, |
312 | | self.metrics.output_batches, |
313 | | self.metrics.output_rows, |
314 | | self.metrics.elapsed_compute, |
315 | | ); |
316 | 0 | other |
317 | | } |
318 | 0 | }) |
319 | 0 | } |
320 | | } |
321 | | |
322 | | /// Given a set of struct column indices to flatten |
323 | | /// try converting the column in input into multiple subfield columns |
324 | | /// For example |
325 | | /// struct_col: [a: struct(item: int, name: string), b: int] |
326 | | /// with a batch |
327 | | /// {a: {item: 1, name: "a"}, b: 2}, |
328 | | /// {a: {item: 3, name: "b"}, b: 4] |
329 | | /// will be converted into |
330 | | /// {a.item: 1, a.name: "a", b: 2}, |
331 | | /// {a.item: 3, a.name: "b", b: 4} |
332 | 1 | fn flatten_struct_cols( |
333 | 1 | input_batch: &[Arc<dyn Array>], |
334 | 1 | schema: &SchemaRef, |
335 | 1 | struct_column_indices: &HashSet<usize>, |
336 | 1 | ) -> Result<RecordBatch> { |
337 | | // horizontal expansion because of struct unnest |
338 | 1 | let columns_expanded = input_batch |
339 | 1 | .iter() |
340 | 1 | .enumerate() |
341 | 3 | .map(|(idx, column_data)| match struct_column_indices.get(&idx) { |
342 | 0 | Some(_) => match column_data.data_type() { |
343 | | DataType::Struct(_) => { |
344 | 0 | let struct_arr = |
345 | 0 | column_data.as_any().downcast_ref::<StructArray>().unwrap(); |
346 | 0 | Ok(struct_arr.columns().to_vec()) |
347 | | } |
348 | 0 | data_type => internal_err!( |
349 | 0 | "expecting column {} from input plan to be a struct, got {:?}", |
350 | 0 | idx, |
351 | 0 | data_type |
352 | 0 | ), |
353 | | }, |
354 | 3 | None => Ok(vec![Arc::clone(column_data)]), |
355 | 3 | }) |
356 | 1 | .collect::<Result<Vec<_>>>()?0 |
357 | 1 | .into_iter() |
358 | 1 | .flatten() |
359 | 1 | .collect(); |
360 | 1 | Ok(RecordBatch::try_new(Arc::clone(schema), columns_expanded)?0 ) |
361 | 1 | } |
362 | | |
363 | | #[derive(Debug, Copy, Clone, Eq, PartialEq, Hash)] |
364 | | pub struct ListUnnest { |
365 | | pub index_in_input_schema: usize, |
366 | | pub depth: usize, |
367 | | } |
368 | | |
369 | | /// This function is used to execute the unnesting on multiple columns all at once, but |
370 | | /// one level at a time, and is called n times, where n is the highest recursion level among |
371 | | /// the unnest exprs in the query. |
372 | | /// |
373 | | /// For example giving the following query: |
374 | | /// ```sql |
375 | | /// select unnest(colA, max_depth:=3) as P1, unnest(colA,max_depth:=2) as P2, unnest(colB, max_depth:=1) as P3 from temp; |
376 | | /// ``` |
377 | | /// Then the total times this function being called is 3 |
378 | | /// |
379 | | /// It needs to be aware of which level the current unnesting is, because if there exists |
380 | | /// multiple unnesting on the same column, but with different recursion levels, say |
381 | | /// **unnest(colA, max_depth:=3)** and **unnest(colA, max_depth:=2)**, then the unnesting |
382 | | /// of expr **unnest(colA, max_depth:=3)** will start at level 3, while unnesting for expr |
383 | | /// **unnest(colA, max_depth:=2)** has to start at level 2 |
384 | | /// |
385 | | /// Set *colA* as a 3-dimension columns and *colB* as an array (1-dimension). As stated, |
386 | | /// this function is called with the descending order of recursion depth |
387 | | /// |
388 | | /// Depth = 3 |
389 | | /// - colA(3-dimension) unnest into temp column temp_P1(2_dimension) (unnesting of P1 starts |
390 | | /// from this level) |
391 | | /// - colA(3-dimension) having indices repeated by the unnesting operation above |
392 | | /// - colB(1-dimension) having indices repeated by the unnesting operation above |
393 | | /// |
394 | | /// Depth = 2 |
395 | | /// - temp_P1(2-dimension) unnest into temp column temp_P1(1-dimension) |
396 | | /// - colA(3-dimension) unnest into temp column temp_P2(2-dimension) (unnesting of P2 starts |
397 | | /// from this level) |
398 | | /// - colB(1-dimension) having indices repeated by the unnesting operation above |
399 | | /// |
400 | | /// Depth = 1 |
401 | | /// - temp_P1(1-dimension) unnest into P1 |
402 | | /// - temp_P2(2-dimension) unnest into P2 |
403 | | /// - colB(1-dimension) unnest into P3 (unnesting of P3 starts from this level) |
404 | | /// |
405 | | /// The returned array will has the same size as the input batch |
406 | | /// and only contains original columns that are not being unnested. |
407 | 2 | fn list_unnest_at_level( |
408 | 2 | batch: &[ArrayRef], |
409 | 2 | list_type_unnests: &[ListUnnest], |
410 | 2 | temp_unnested_arrs: &mut HashMap<ListUnnest, ArrayRef>, |
411 | 2 | level_to_unnest: usize, |
412 | 2 | options: &UnnestOptions, |
413 | 2 | ) -> Result<(Vec<ArrayRef>, usize)> { |
414 | 2 | // extract unnestable columns at this level |
415 | 2 | let (arrs_to_unnest, list_unnest_specs): (Vec<Arc<dyn Array>>, Vec<_>) = |
416 | 2 | list_type_unnests |
417 | 2 | .iter() |
418 | 6 | .filter_map(|unnesting| { |
419 | 6 | if level_to_unnest == unnesting.depth { |
420 | 3 | return Some(( |
421 | 3 | Arc::clone(&batch[unnesting.index_in_input_schema]), |
422 | 3 | *unnesting, |
423 | 3 | )); |
424 | 3 | } |
425 | 3 | // this means the unnesting on this item has started at higher level |
426 | 3 | // and need to continue until depth reaches 1 |
427 | 3 | if level_to_unnest < unnesting.depth { |
428 | 1 | return Some(( |
429 | 1 | Arc::clone(temp_unnested_arrs.get(unnesting).unwrap()), |
430 | 1 | *unnesting, |
431 | 1 | )); |
432 | 2 | } |
433 | 2 | None |
434 | 6 | }) |
435 | 2 | .unzip(); |
436 | | |
437 | | // filter out so that list_arrays only contain column with the highest depth |
438 | | // at the same time, during iteration remove this depth so next time we don't have to unnest them again |
439 | 2 | let longest_length = find_longest_length(&arrs_to_unnest, options)?0 ; |
440 | 2 | let unnested_length = longest_length.as_primitive::<Int64Type>(); |
441 | 2 | let total_length = if unnested_length.is_empty() { |
442 | 0 | 0 |
443 | | } else { |
444 | 2 | sum(unnested_length).ok_or_else(|| { |
445 | 0 | exec_datafusion_err!("Failed to calculate the total unnested length") |
446 | 2 | })?0 as usize |
447 | | }; |
448 | 2 | if total_length == 0 { |
449 | 0 | return Ok((vec![], 0)); |
450 | 2 | } |
451 | | |
452 | | // Unnest all the list arrays |
453 | 2 | let unnested_temp_arrays = |
454 | 2 | unnest_list_arrays(arrs_to_unnest.as_ref(), unnested_length, total_length)?0 ; |
455 | | |
456 | | // Create the take indices array for other columns |
457 | 2 | let take_indices = create_take_indicies(unnested_length, total_length); |
458 | | |
459 | | // dimension of arrays in batch is untouch, but the values are repeated |
460 | | // as the side effect of unnesting |
461 | 2 | let ret = repeat_arrs_from_indices(batch, &take_indices)?0 ; |
462 | 2 | unnested_temp_arrays |
463 | 2 | .into_iter() |
464 | 2 | .zip(list_unnest_specs.iter()) |
465 | 4 | .for_each(|(flatten_arr, unnesting)| { |
466 | 4 | temp_unnested_arrs.insert(*unnesting, flatten_arr); |
467 | 4 | }); |
468 | 2 | Ok((ret, total_length)) |
469 | 2 | } |
470 | | struct UnnestingResult { |
471 | | arr: ArrayRef, |
472 | | depth: usize, |
473 | | } |
474 | | |
475 | | /// For each row in a `RecordBatch`, some list/struct columns need to be unnested. |
476 | | /// - For list columns: We will expand the values in each list into multiple rows, |
477 | | /// taking the longest length among these lists, and shorter lists are padded with NULLs. |
478 | | /// - For struct columns: We will expand the struct columns into multiple subfield columns. |
479 | | /// |
480 | | /// For columns that don't need to be unnested, repeat their values until reaching the longest length. |
481 | | /// |
482 | | /// Note: unnest has a big difference in behavior between Postgres and DuckDB |
483 | | /// |
484 | | /// Take this example |
485 | | /// |
486 | | /// 1. Postgres |
487 | | /// ```ignored |
488 | | /// create table temp ( |
489 | | /// i integer[][][], j integer[] |
490 | | /// ) |
491 | | /// insert into temp values ('{{{1,2},{3,4}},{{5,6},{7,8}}}', '{1,2}'); |
492 | | /// select unnest(i), unnest(j) from temp; |
493 | | /// ``` |
494 | | /// |
495 | | /// Result |
496 | | /// ```text |
497 | | /// 1 1 |
498 | | /// 2 2 |
499 | | /// 3 |
500 | | /// 4 |
501 | | /// 5 |
502 | | /// 6 |
503 | | /// 7 |
504 | | /// 8 |
505 | | /// ``` |
506 | | /// 2. DuckDB |
507 | | /// ```ignore |
508 | | /// create table temp (i integer[][][], j integer[]); |
509 | | /// insert into temp values ([[[1,2],[3,4]],[[5,6],[7,8]]], [1,2]); |
510 | | /// select unnest(i,recursive:=true), unnest(j,recursive:=true) from temp; |
511 | | /// ``` |
512 | | /// Result: |
513 | | /// ```text |
514 | | /// |
515 | | /// ┌────────────────────────────────────────────────┬────────────────────────────────────────────────┐ |
516 | | /// │ unnest(i, "recursive" := CAST('t' AS BOOLEAN)) │ unnest(j, "recursive" := CAST('t' AS BOOLEAN)) │ |
517 | | /// │ int32 │ int32 │ |
518 | | /// ├────────────────────────────────────────────────┼────────────────────────────────────────────────┤ |
519 | | /// │ 1 │ 1 │ |
520 | | /// │ 2 │ 2 │ |
521 | | /// │ 3 │ 1 │ |
522 | | /// │ 4 │ 2 │ |
523 | | /// │ 5 │ 1 │ |
524 | | /// │ 6 │ 2 │ |
525 | | /// │ 7 │ 1 │ |
526 | | /// │ 8 │ 2 │ |
527 | | /// └────────────────────────────────────────────────┴────────────────────────────────────────────────┘ |
528 | | /// ``` |
529 | | /// |
530 | | /// The following implementation refer to DuckDB's implementation |
531 | 1 | fn build_batch( |
532 | 1 | batch: &RecordBatch, |
533 | 1 | schema: &SchemaRef, |
534 | 1 | list_type_columns: &[ListUnnest], |
535 | 1 | struct_column_indices: &HashSet<usize>, |
536 | 1 | options: &UnnestOptions, |
537 | 1 | ) -> Result<RecordBatch> { |
538 | 1 | let transformed = match list_type_columns.len() { |
539 | 0 | 0 => flatten_struct_cols(batch.columns(), schema, struct_column_indices), |
540 | | _ => { |
541 | 1 | let mut temp_unnested_result = HashMap::new(); |
542 | 1 | let max_recursion = list_type_columns |
543 | 1 | .iter() |
544 | 3 | .fold(0, |highest_depth, ListUnnest { depth, .. }| { |
545 | 3 | cmp::max(highest_depth, *depth) |
546 | 3 | }); |
547 | 1 | |
548 | 1 | // This arr always has the same column count with the input batch |
549 | 1 | let mut flatten_arrs = vec![]; |
550 | | |
551 | | // original batch has the same columns |
552 | | // all unnesting results are written to temp_batch |
553 | 2 | for depth in (1..=max_recursion).rev()1 { |
554 | 2 | let input = match depth == max_recursion { |
555 | 1 | true => batch.columns(), |
556 | 1 | false => &flatten_arrs, |
557 | | }; |
558 | 2 | let (temp_result, num_rows) = list_unnest_at_level( |
559 | 2 | input, |
560 | 2 | list_type_columns, |
561 | 2 | &mut temp_unnested_result, |
562 | 2 | depth, |
563 | 2 | options, |
564 | 2 | )?0 ; |
565 | 2 | if num_rows == 0 { |
566 | 0 | return Ok(RecordBatch::new_empty(Arc::clone(schema))); |
567 | 2 | } |
568 | 2 | flatten_arrs = temp_result; |
569 | | } |
570 | 1 | let unnested_array_map: HashMap<usize, Vec<UnnestingResult>> = |
571 | 1 | temp_unnested_result.into_iter().fold( |
572 | 1 | HashMap::new(), |
573 | 1 | |mut acc, |
574 | | ( |
575 | | ListUnnest { |
576 | | index_in_input_schema, |
577 | | depth, |
578 | | }, |
579 | | flattened_array, |
580 | 3 | )| { |
581 | 3 | acc.entry(index_in_input_schema).or_default().push( |
582 | 3 | UnnestingResult { |
583 | 3 | arr: flattened_array, |
584 | 3 | depth, |
585 | 3 | }, |
586 | 3 | ); |
587 | 3 | acc |
588 | 3 | }, |
589 | 1 | ); |
590 | 1 | let output_order: HashMap<ListUnnest, usize> = list_type_columns |
591 | 1 | .iter() |
592 | 1 | .enumerate() |
593 | 3 | .map(|(order, unnest_def)| (*unnest_def, order)) |
594 | 1 | .collect(); |
595 | 1 | |
596 | 1 | // one original column may be unnested multiple times into separate columns |
597 | 1 | let mut multi_unnested_per_original_index = unnested_array_map |
598 | 1 | .into_iter() |
599 | 1 | .map( |
600 | 1 | // each item in unnested_columns is the result of unnesting the same input column |
601 | 1 | // we need to sort them to conform with the original expression order |
602 | 1 | // e.g unnest(unnest(col)) must goes before unnest(col) |
603 | 2 | |(original_index, mut unnested_columns)| { |
604 | 2 | unnested_columns.sort_by( |
605 | 2 | |UnnestingResult { depth: depth1, .. }, |
606 | | UnnestingResult { depth: depth2, .. }| |
607 | 1 | -> Ordering { |
608 | 1 | output_order |
609 | 1 | .get(&ListUnnest { |
610 | 1 | depth: *depth1, |
611 | 1 | index_in_input_schema: original_index, |
612 | 1 | }) |
613 | 1 | .unwrap() |
614 | 1 | .cmp( |
615 | 1 | output_order |
616 | 1 | .get(&ListUnnest { |
617 | 1 | depth: *depth2, |
618 | 1 | index_in_input_schema: original_index, |
619 | 1 | }) |
620 | 1 | .unwrap(), |
621 | 1 | ) |
622 | 2 | }, |
623 | 2 | ); |
624 | 2 | ( |
625 | 2 | original_index, |
626 | 2 | unnested_columns |
627 | 2 | .into_iter() |
628 | 3 | .map(|result| result.arr) |
629 | 2 | .collect::<Vec<_>>(), |
630 | 2 | ) |
631 | 2 | }, |
632 | 1 | ) |
633 | 1 | .collect::<HashMap<_, _>>(); |
634 | 1 | |
635 | 1 | let ret = flatten_arrs |
636 | 1 | .into_iter() |
637 | 1 | .enumerate() |
638 | 2 | .flat_map(|(col_idx, arr)| { |
639 | 2 | // convert original column into its unnested version(s) |
640 | 2 | // Plural because one column can be unnested with different recursion level |
641 | 2 | // and into separate output columns |
642 | 2 | match multi_unnested_per_original_index.remove(&col_idx) { |
643 | 2 | Some(unnested_arrays) => unnested_arrays, |
644 | 0 | None => vec![arr], |
645 | | } |
646 | 2 | }) |
647 | 1 | .collect::<Vec<_>>(); |
648 | 1 | |
649 | 1 | flatten_struct_cols(&ret, schema, struct_column_indices) |
650 | | } |
651 | | }; |
652 | 1 | transformed |
653 | 1 | } |
654 | | |
655 | | /// Find the longest list length among the given list arrays for each row. |
656 | | /// |
657 | | /// For example if we have the following two list arrays: |
658 | | /// |
659 | | /// ```ignore |
660 | | /// l1: [1, 2, 3], null, [], [3] |
661 | | /// l2: [4,5], [], null, [6, 7] |
662 | | /// ``` |
663 | | /// |
664 | | /// If `preserve_nulls` is false, the longest length array will be: |
665 | | /// |
666 | | /// ```ignore |
667 | | /// longest_length: [3, 0, 0, 2] |
668 | | /// ``` |
669 | | /// |
670 | | /// whereas if `preserve_nulls` is true, the longest length array will be: |
671 | | /// |
672 | | /// |
673 | | /// ```ignore |
674 | | /// longest_length: [3, 1, 1, 2] |
675 | | /// ``` |
676 | | /// |
677 | 10 | fn find_longest_length( |
678 | 10 | list_arrays: &[ArrayRef], |
679 | 10 | options: &UnnestOptions, |
680 | 10 | ) -> Result<ArrayRef> { |
681 | | // The length of a NULL list |
682 | 10 | let null_length = if options.preserve_nulls { |
683 | 6 | Scalar::new(Int64Array::from_value(1, 1)) |
684 | | } else { |
685 | 4 | Scalar::new(Int64Array::from_value(0, 1)) |
686 | | }; |
687 | 10 | let list_lengths: Vec<ArrayRef> = list_arrays |
688 | 10 | .iter() |
689 | 14 | .map(|list_array| { |
690 | 14 | let mut length_array = length(list_array)?0 ; |
691 | | // Make sure length arrays have the same type. Int64 is the most general one. |
692 | 14 | length_array = cast(&length_array, &DataType::Int64)?0 ; |
693 | 14 | length_array = |
694 | 14 | zip(&is_not_null(&length_array)?0 , &length_array, &null_length)?0 ; |
695 | 14 | Ok(length_array) |
696 | 14 | }) |
697 | 10 | .collect::<Result<_>>()?0 ; |
698 | | |
699 | 10 | let longest_length = list_lengths.iter().skip(1).try_fold( |
700 | 10 | Arc::clone(&list_lengths[0]), |
701 | 10 | |longest, current| {4 |
702 | 4 | let is_lt = lt(&longest, ¤t)?0 ; |
703 | 4 | zip(&is_lt, ¤t, &longest) |
704 | 10 | }4 , |
705 | 10 | )?0 ; |
706 | 10 | Ok(longest_length) |
707 | 10 | } |
708 | | |
709 | | /// Trait defining common methods used for unnesting, implemented by list array types. |
710 | | trait ListArrayType: Array { |
711 | | /// Returns a reference to the values of this list. |
712 | | fn values(&self) -> &ArrayRef; |
713 | | |
714 | | /// Returns the start and end offset of the values for the given row. |
715 | | fn value_offsets(&self, row: usize) -> (i64, i64); |
716 | | } |
717 | | |
718 | | impl ListArrayType for ListArray { |
719 | 5 | fn values(&self) -> &ArrayRef { |
720 | 5 | self.values() |
721 | 5 | } |
722 | | |
723 | 23 | fn value_offsets(&self, row: usize) -> (i64, i64) { |
724 | 23 | let offsets = self.value_offsets(); |
725 | 23 | (offsets[row].into(), offsets[row + 1].into()) |
726 | 23 | } |
727 | | } |
728 | | |
729 | | impl ListArrayType for LargeListArray { |
730 | 0 | fn values(&self) -> &ArrayRef { |
731 | 0 | self.values() |
732 | 0 | } |
733 | | |
734 | 0 | fn value_offsets(&self, row: usize) -> (i64, i64) { |
735 | 0 | let offsets = self.value_offsets(); |
736 | 0 | (offsets[row], offsets[row + 1]) |
737 | 0 | } |
738 | | } |
739 | | |
740 | | impl ListArrayType for FixedSizeListArray { |
741 | 1 | fn values(&self) -> &ArrayRef { |
742 | 1 | self.values() |
743 | 1 | } |
744 | | |
745 | 4 | fn value_offsets(&self, row: usize) -> (i64, i64) { |
746 | 4 | let start = self.value_offset(row) as i64; |
747 | 4 | (start, start + self.value_length() as i64) |
748 | 4 | } |
749 | | } |
750 | | |
751 | | /// Unnest multiple list arrays according to the length array. |
752 | 2 | fn unnest_list_arrays( |
753 | 2 | list_arrays: &[ArrayRef], |
754 | 2 | length_array: &PrimitiveArray<Int64Type>, |
755 | 2 | capacity: usize, |
756 | 2 | ) -> Result<Vec<ArrayRef>> { |
757 | 2 | let typed_arrays = list_arrays |
758 | 2 | .iter() |
759 | 4 | .map(|list_array| match list_array.data_type() { |
760 | 4 | DataType::List(_) => Ok(list_array.as_list::<i32>() as &dyn ListArrayType), |
761 | | DataType::LargeList(_) => { |
762 | 0 | Ok(list_array.as_list::<i64>() as &dyn ListArrayType) |
763 | | } |
764 | | DataType::FixedSizeList(_, _) => { |
765 | 0 | Ok(list_array.as_fixed_size_list() as &dyn ListArrayType) |
766 | | } |
767 | 0 | other => exec_err!("Invalid unnest datatype {other }"), |
768 | 4 | }) |
769 | 2 | .collect::<Result<Vec<_>>>()?0 ; |
770 | | |
771 | 2 | typed_arrays |
772 | 2 | .iter() |
773 | 4 | .map(|list_array| unnest_list_array(*list_array, length_array, capacity)) |
774 | 2 | .collect::<Result<_>>() |
775 | 2 | } |
776 | | |
777 | | /// Unnest a list array according the target length array. |
778 | | /// |
779 | | /// Consider a list array like this: |
780 | | /// |
781 | | /// ```ignore |
782 | | /// [1], [2, 3, 4], null, [5], [], |
783 | | /// ``` |
784 | | /// |
785 | | /// and the length array is: |
786 | | /// |
787 | | /// ```ignore |
788 | | /// [2, 3, 2, 1, 2] |
789 | | /// ``` |
790 | | /// |
791 | | /// If the length of a certain list is less than the target length, pad with NULLs. |
792 | | /// So the unnested array will look like this: |
793 | | /// |
794 | | /// ```ignore |
795 | | /// [1, null, 2, 3, 4, null, null, 5, null, null] |
796 | | /// ``` |
797 | | /// |
798 | 6 | fn unnest_list_array( |
799 | 6 | list_array: &dyn ListArrayType, |
800 | 6 | length_array: &PrimitiveArray<Int64Type>, |
801 | 6 | capacity: usize, |
802 | 6 | ) -> Result<ArrayRef> { |
803 | 6 | let values = list_array.values(); |
804 | 6 | let mut take_indicies_builder = PrimitiveArray::<Int64Type>::builder(capacity); |
805 | 36 | for row in 0..list_array.len()6 { |
806 | 36 | let mut value_length = 0; |
807 | 36 | if !list_array.is_null(row) { |
808 | 27 | let (start, end) = list_array.value_offsets(row); |
809 | 27 | value_length = end - start; |
810 | 63 | for i in start..end27 { |
811 | 63 | take_indicies_builder.append_value(i) |
812 | | } |
813 | 9 | } |
814 | 36 | let target_length = length_array.value(row); |
815 | 36 | debug_assert!( |
816 | 36 | value_length <= target_length, |
817 | 0 | "value length is beyond the longest length" |
818 | | ); |
819 | | // Pad with NULL values |
820 | 36 | for _ in value_length..target_length { |
821 | 26 | take_indicies_builder.append_null(); |
822 | 26 | } |
823 | | } |
824 | 6 | Ok(kernels::take::take( |
825 | 6 | &values, |
826 | 6 | &take_indicies_builder.finish(), |
827 | 6 | None, |
828 | 6 | )?0 ) |
829 | 6 | } |
830 | | |
831 | | /// Creates take indicies that will be used to expand all columns except for the list type |
832 | | /// [`columns`](UnnestExec::list_column_indices) that is being unnested. |
833 | | /// Every column value needs to be repeated multiple times according to the length array. |
834 | | /// |
835 | | /// If the length array looks like this: |
836 | | /// |
837 | | /// ```ignore |
838 | | /// [2, 3, 1] |
839 | | /// ``` |
840 | | /// Then `create_take_indicies` will return an array like this |
841 | | /// |
842 | | /// ```ignore |
843 | | /// [0, 0, 1, 1, 1, 2] |
844 | | /// ``` |
845 | | /// |
846 | 3 | fn create_take_indicies( |
847 | 3 | length_array: &PrimitiveArray<Int64Type>, |
848 | 3 | capacity: usize, |
849 | 3 | ) -> PrimitiveArray<Int64Type> { |
850 | 3 | // `find_longest_length()` guarantees this. |
851 | 3 | debug_assert!( |
852 | 3 | length_array.null_count() == 0, |
853 | 0 | "length array should not contain nulls" |
854 | | ); |
855 | 3 | let mut builder = PrimitiveArray::<Int64Type>::builder(capacity); |
856 | 13 | for (index, repeat) in length_array.iter().enumerate()3 { |
857 | 13 | // The length array should not contain nulls, so unwrap is safe |
858 | 13 | let repeat = repeat.unwrap(); |
859 | 33 | (0..repeat).for_each(|_| builder.append_value(index as i64)); |
860 | 13 | } |
861 | 3 | builder.finish() |
862 | 3 | } |
863 | | |
864 | | /// Create the batch given an arrays and a `indices` array |
865 | | /// that is used by the take kernel to copy values. |
866 | | /// |
867 | | /// For example if we have the following batch: |
868 | | /// |
869 | | /// ```ignore |
870 | | /// c1: [1], null, [2, 3, 4], null, [5, 6] |
871 | | /// c2: 'a', 'b', 'c', null, 'd' |
872 | | /// ``` |
873 | | /// |
874 | | /// then the `unnested_list_arrays` contains the unnest column that will replace `c1` in |
875 | | /// the final batch if `preserve_nulls` is true: |
876 | | /// |
877 | | /// ```ignore |
878 | | /// c1: 1, null, 2, 3, 4, null, 5, 6 |
879 | | /// ``` |
880 | | /// |
881 | | /// And the `indices` array contains the indices that are used by `take` kernel to |
882 | | /// repeat the values in `c2`: |
883 | | /// |
884 | | /// ```ignore |
885 | | /// 0, 1, 2, 2, 2, 3, 4, 4 |
886 | | /// ``` |
887 | | /// |
888 | | /// so that the final batch will look like: |
889 | | /// |
890 | | /// ```ignore |
891 | | /// c1: 1, null, 2, 3, 4, null, 5, 6 |
892 | | /// c2: 'a', 'b', 'c', 'c', 'c', null, 'd', 'd' |
893 | | /// ``` |
894 | | /// |
895 | 2 | fn repeat_arrs_from_indices( |
896 | 2 | batch: &[ArrayRef], |
897 | 2 | indices: &PrimitiveArray<Int64Type>, |
898 | 2 | ) -> Result<Vec<Arc<dyn Array>>> { |
899 | 2 | batch |
900 | 2 | .iter() |
901 | 4 | .map(|arr| Ok(kernels::take::take(arr, indices, None)?0 )) |
902 | 2 | .collect::<Result<_>>() |
903 | 2 | } |
904 | | |
905 | | #[cfg(test)] |
906 | | mod tests { |
907 | | use super::*; |
908 | | use arrow::{ |
909 | | datatypes::{Field, Int32Type}, |
910 | | util::pretty::pretty_format_batches, |
911 | | }; |
912 | | use arrow_array::{GenericListArray, OffsetSizeTrait, StringArray}; |
913 | | use arrow_buffer::{BooleanBufferBuilder, NullBuffer, OffsetBuffer}; |
914 | | |
915 | | // Create a GenericListArray with the following list values: |
916 | | // [A, B, C], [], NULL, [D], NULL, [NULL, F] |
917 | 4 | fn make_generic_array<OffsetSize>() -> GenericListArray<OffsetSize> |
918 | 4 | where |
919 | 4 | OffsetSize: OffsetSizeTrait, |
920 | 4 | { |
921 | 4 | let mut values = vec![]; |
922 | 4 | let mut offsets: Vec<OffsetSize> = vec![OffsetSize::zero()]; |
923 | 4 | let mut valid = BooleanBufferBuilder::new(6); |
924 | 4 | |
925 | 4 | // [A, B, C] |
926 | 4 | values.extend_from_slice(&[Some("A"), Some("B"), Some("C")]); |
927 | 4 | offsets.push(OffsetSize::from_usize(values.len()).unwrap()); |
928 | 4 | valid.append(true); |
929 | 4 | |
930 | 4 | // [] |
931 | 4 | offsets.push(OffsetSize::from_usize(values.len()).unwrap()); |
932 | 4 | valid.append(true); |
933 | 4 | |
934 | 4 | // NULL with non-zero value length |
935 | 4 | // Issue https://github.com/apache/datafusion/issues/9932 |
936 | 4 | values.push(Some("?")); |
937 | 4 | offsets.push(OffsetSize::from_usize(values.len()).unwrap()); |
938 | 4 | valid.append(false); |
939 | 4 | |
940 | 4 | // [D] |
941 | 4 | values.push(Some("D")); |
942 | 4 | offsets.push(OffsetSize::from_usize(values.len()).unwrap()); |
943 | 4 | valid.append(true); |
944 | 4 | |
945 | 4 | // Another NULL with zero value length |
946 | 4 | offsets.push(OffsetSize::from_usize(values.len()).unwrap()); |
947 | 4 | valid.append(false); |
948 | 4 | |
949 | 4 | // [NULL, F] |
950 | 4 | values.extend_from_slice(&[None, Some("F")]); |
951 | 4 | offsets.push(OffsetSize::from_usize(values.len()).unwrap()); |
952 | 4 | valid.append(true); |
953 | 4 | |
954 | 4 | let field = Arc::new(Field::new("item", DataType::Utf8, true)); |
955 | 4 | GenericListArray::<OffsetSize>::new( |
956 | 4 | field, |
957 | 4 | OffsetBuffer::new(offsets.into()), |
958 | 4 | Arc::new(StringArray::from(values)), |
959 | 4 | Some(NullBuffer::new(valid.finish())), |
960 | 4 | ) |
961 | 4 | } |
962 | | |
963 | | // Create a FixedSizeListArray with the following list values: |
964 | | // [A, B], NULL, [C, D], NULL, [NULL, F], [NULL, NULL] |
965 | 3 | fn make_fixed_list() -> FixedSizeListArray { |
966 | 3 | let values = Arc::new(StringArray::from_iter([ |
967 | 3 | Some("A"), |
968 | 3 | Some("B"), |
969 | 3 | None, |
970 | 3 | None, |
971 | 3 | Some("C"), |
972 | 3 | Some("D"), |
973 | 3 | None, |
974 | 3 | None, |
975 | 3 | None, |
976 | 3 | Some("F"), |
977 | 3 | None, |
978 | 3 | None, |
979 | 3 | ])); |
980 | 3 | let field = Arc::new(Field::new("item", DataType::Utf8, true)); |
981 | 3 | let valid = NullBuffer::from(vec![true, false, true, false, true, true]); |
982 | 3 | FixedSizeListArray::new(field, 2, values, Some(valid)) |
983 | 3 | } |
984 | | |
985 | 2 | fn verify_unnest_list_array( |
986 | 2 | list_array: &dyn ListArrayType, |
987 | 2 | lengths: Vec<i64>, |
988 | 2 | expected: Vec<Option<&str>>, |
989 | 2 | ) -> datafusion_common::Result<()> { |
990 | 2 | let length_array = Int64Array::from(lengths); |
991 | 2 | let unnested_array = unnest_list_array(list_array, &length_array, 3 * 6)?0 ; |
992 | 2 | let strs = unnested_array.as_string::<i32>().iter().collect::<Vec<_>>(); |
993 | 2 | assert_eq!(strs, expected); |
994 | 2 | Ok(()) |
995 | 2 | } |
996 | | |
997 | | #[test] |
998 | 1 | fn test_build_batch_list_arr_recursive() -> datafusion_common::Result<()> { |
999 | 1 | // col1 | col2 |
1000 | 1 | // [[1,2,3],null,[4,5]] | ['a','b'] |
1001 | 1 | // [[7,8,9,10], null, [11,12,13]] | ['c','d'] |
1002 | 1 | // null | ['e'] |
1003 | 1 | let list_arr1 = ListArray::from_iter_primitive::<Int32Type, _, _>(vec![ |
1004 | 1 | Some(vec![Some(1), Some(2), Some(3)]), |
1005 | 1 | None, |
1006 | 1 | Some(vec![Some(4), Some(5)]), |
1007 | 1 | Some(vec![Some(7), Some(8), Some(9), Some(10)]), |
1008 | 1 | None, |
1009 | 1 | Some(vec![Some(11), Some(12), Some(13)]), |
1010 | 1 | ]); |
1011 | 1 | |
1012 | 1 | let list_arr1_ref = Arc::new(list_arr1) as ArrayRef; |
1013 | 1 | let offsets = OffsetBuffer::from_lengths([3, 3, 0]); |
1014 | 1 | let mut nulls = BooleanBufferBuilder::new(3); |
1015 | 1 | nulls.append(true); |
1016 | 1 | nulls.append(true); |
1017 | 1 | nulls.append(false); |
1018 | 1 | // list<list<int32>> |
1019 | 1 | let col1_field = Field::new_list_field( |
1020 | 1 | DataType::List(Arc::new(Field::new_list_field( |
1021 | 1 | list_arr1_ref.data_type().to_owned(), |
1022 | 1 | true, |
1023 | 1 | ))), |
1024 | 1 | true, |
1025 | 1 | ); |
1026 | 1 | let col1 = ListArray::new( |
1027 | 1 | Arc::new(Field::new_list_field( |
1028 | 1 | list_arr1_ref.data_type().to_owned(), |
1029 | 1 | true, |
1030 | 1 | )), |
1031 | 1 | offsets, |
1032 | 1 | list_arr1_ref, |
1033 | 1 | Some(NullBuffer::new(nulls.finish())), |
1034 | 1 | ); |
1035 | 1 | |
1036 | 1 | let list_arr2 = StringArray::from(vec![ |
1037 | 1 | Some("a"), |
1038 | 1 | Some("b"), |
1039 | 1 | Some("c"), |
1040 | 1 | Some("d"), |
1041 | 1 | Some("e"), |
1042 | 1 | ]); |
1043 | 1 | |
1044 | 1 | let offsets = OffsetBuffer::from_lengths([2, 2, 1]); |
1045 | 1 | let mut nulls = BooleanBufferBuilder::new(3); |
1046 | 1 | nulls.append_n(3, true); |
1047 | 1 | let col2_field = Field::new( |
1048 | 1 | "col2", |
1049 | 1 | DataType::List(Arc::new(Field::new_list_field(DataType::Utf8, true))), |
1050 | 1 | true, |
1051 | 1 | ); |
1052 | 1 | let col2 = GenericListArray::<i32>::new( |
1053 | 1 | Arc::new(Field::new_list_field(DataType::Utf8, true)), |
1054 | 1 | OffsetBuffer::new(offsets.into()), |
1055 | 1 | Arc::new(list_arr2), |
1056 | 1 | Some(NullBuffer::new(nulls.finish())), |
1057 | 1 | ); |
1058 | 1 | // convert col1 and col2 to a record batch |
1059 | 1 | let schema = Arc::new(Schema::new(vec![col1_field, col2_field])); |
1060 | 1 | let out_schema = Arc::new(Schema::new(vec![ |
1061 | 1 | Field::new( |
1062 | 1 | "col1_unnest_placeholder_depth_1", |
1063 | 1 | DataType::List(Arc::new(Field::new("item", DataType::Int32, true))), |
1064 | 1 | true, |
1065 | 1 | ), |
1066 | 1 | Field::new("col1_unnest_placeholder_depth_2", DataType::Int32, true), |
1067 | 1 | Field::new("col2_unnest_placeholder_depth_1", DataType::Utf8, true), |
1068 | 1 | ])); |
1069 | 1 | let batch = RecordBatch::try_new( |
1070 | 1 | Arc::clone(&schema), |
1071 | 1 | vec![Arc::new(col1) as ArrayRef, Arc::new(col2) as ArrayRef], |
1072 | 1 | ) |
1073 | 1 | .unwrap(); |
1074 | 1 | let list_type_columns = vec![ |
1075 | 1 | ListUnnest { |
1076 | 1 | index_in_input_schema: 0, |
1077 | 1 | depth: 1, |
1078 | 1 | }, |
1079 | 1 | ListUnnest { |
1080 | 1 | index_in_input_schema: 0, |
1081 | 1 | depth: 2, |
1082 | 1 | }, |
1083 | 1 | ListUnnest { |
1084 | 1 | index_in_input_schema: 1, |
1085 | 1 | depth: 1, |
1086 | 1 | }, |
1087 | 1 | ]; |
1088 | 1 | let ret = build_batch( |
1089 | 1 | &batch, |
1090 | 1 | &out_schema, |
1091 | 1 | list_type_columns.as_ref(), |
1092 | 1 | &HashSet::default(), |
1093 | 1 | &UnnestOptions { |
1094 | 1 | preserve_nulls: true, |
1095 | 1 | }, |
1096 | 1 | )?0 ; |
1097 | 1 | let actual = |
1098 | 1 | format!("{}", pretty_format_batches(vec![ret].as_ref())?0 ).to_lowercase(); |
1099 | 1 | let expected = r#" |
1100 | 1 | +---------------------------------+---------------------------------+---------------------------------+ |
1101 | 1 | | col1_unnest_placeholder_depth_1 | col1_unnest_placeholder_depth_2 | col2_unnest_placeholder_depth_1 | |
1102 | 1 | +---------------------------------+---------------------------------+---------------------------------+ |
1103 | 1 | | [1, 2, 3] | 1 | a | |
1104 | 1 | | | 2 | b | |
1105 | 1 | | [4, 5] | 3 | | |
1106 | 1 | | [1, 2, 3] | | a | |
1107 | 1 | | | | b | |
1108 | 1 | | [4, 5] | | | |
1109 | 1 | | [1, 2, 3] | 4 | a | |
1110 | 1 | | | 5 | b | |
1111 | 1 | | [4, 5] | | | |
1112 | 1 | | [7, 8, 9, 10] | 7 | c | |
1113 | 1 | | | 8 | d | |
1114 | 1 | | [11, 12, 13] | 9 | | |
1115 | 1 | | | 10 | | |
1116 | 1 | | [7, 8, 9, 10] | | c | |
1117 | 1 | | | | d | |
1118 | 1 | | [11, 12, 13] | | | |
1119 | 1 | | [7, 8, 9, 10] | 11 | c | |
1120 | 1 | | | 12 | d | |
1121 | 1 | | [11, 12, 13] | 13 | | |
1122 | 1 | | | | e | |
1123 | 1 | +---------------------------------+---------------------------------+---------------------------------+ |
1124 | 1 | "# |
1125 | 1 | .trim(); |
1126 | 1 | assert_eq!(actual, expected); |
1127 | 1 | Ok(()) |
1128 | 1 | } |
1129 | | |
1130 | | #[test] |
1131 | 1 | fn test_unnest_list_array() -> datafusion_common::Result<()> { |
1132 | 1 | // [A, B, C], [], NULL, [D], NULL, [NULL, F] |
1133 | 1 | let list_array = make_generic_array::<i32>(); |
1134 | 1 | verify_unnest_list_array( |
1135 | 1 | &list_array, |
1136 | 1 | vec![3, 2, 1, 2, 0, 3], |
1137 | 1 | vec![ |
1138 | 1 | Some("A"), |
1139 | 1 | Some("B"), |
1140 | 1 | Some("C"), |
1141 | 1 | None, |
1142 | 1 | None, |
1143 | 1 | None, |
1144 | 1 | Some("D"), |
1145 | 1 | None, |
1146 | 1 | None, |
1147 | 1 | Some("F"), |
1148 | 1 | None, |
1149 | 1 | ], |
1150 | 1 | )?0 ; |
1151 | | |
1152 | | // [A, B], NULL, [C, D], NULL, [NULL, F], [NULL, NULL] |
1153 | 1 | let list_array = make_fixed_list(); |
1154 | 1 | verify_unnest_list_array( |
1155 | 1 | &list_array, |
1156 | 1 | vec![3, 1, 2, 0, 2, 3], |
1157 | 1 | vec![ |
1158 | 1 | Some("A"), |
1159 | 1 | Some("B"), |
1160 | 1 | None, |
1161 | 1 | None, |
1162 | 1 | Some("C"), |
1163 | 1 | Some("D"), |
1164 | 1 | None, |
1165 | 1 | Some("F"), |
1166 | 1 | None, |
1167 | 1 | None, |
1168 | 1 | None, |
1169 | 1 | ], |
1170 | 1 | )?0 ; |
1171 | | |
1172 | 1 | Ok(()) |
1173 | 1 | } |
1174 | | |
1175 | 8 | fn verify_longest_length( |
1176 | 8 | list_arrays: &[ArrayRef], |
1177 | 8 | preserve_nulls: bool, |
1178 | 8 | expected: Vec<i64>, |
1179 | 8 | ) -> datafusion_common::Result<()> { |
1180 | 8 | let options = UnnestOptions { preserve_nulls }; |
1181 | 8 | let longest_length = find_longest_length(list_arrays, &options)?0 ; |
1182 | 8 | let expected_array = Int64Array::from(expected); |
1183 | 8 | assert_eq!( |
1184 | 8 | longest_length |
1185 | 8 | .as_any() |
1186 | 8 | .downcast_ref::<Int64Array>() |
1187 | 8 | .unwrap(), |
1188 | 8 | &expected_array |
1189 | 8 | ); |
1190 | 8 | Ok(()) |
1191 | 8 | } |
1192 | | |
1193 | | #[test] |
1194 | 1 | fn test_longest_list_length() -> datafusion_common::Result<()> { |
1195 | 1 | // Test with single ListArray |
1196 | 1 | // [A, B, C], [], NULL, [D], NULL, [NULL, F] |
1197 | 1 | let list_array = Arc::new(make_generic_array::<i32>()) as ArrayRef; |
1198 | 1 | verify_longest_length(&[Arc::clone(&list_array)], false, vec![3, 0, 0, 1, 0, 2])?0 ; |
1199 | 1 | verify_longest_length(&[Arc::clone(&list_array)], true, vec![3, 0, 1, 1, 1, 2])?0 ; |
1200 | | |
1201 | | // Test with single LargeListArray |
1202 | | // [A, B, C], [], NULL, [D], NULL, [NULL, F] |
1203 | 1 | let list_array = Arc::new(make_generic_array::<i64>()) as ArrayRef; |
1204 | 1 | verify_longest_length(&[Arc::clone(&list_array)], false, vec![3, 0, 0, 1, 0, 2])?0 ; |
1205 | 1 | verify_longest_length(&[Arc::clone(&list_array)], true, vec![3, 0, 1, 1, 1, 2])?0 ; |
1206 | | |
1207 | | // Test with single FixedSizeListArray |
1208 | | // [A, B], NULL, [C, D], NULL, [NULL, F], [NULL, NULL] |
1209 | 1 | let list_array = Arc::new(make_fixed_list()) as ArrayRef; |
1210 | 1 | verify_longest_length(&[Arc::clone(&list_array)], false, vec![2, 0, 2, 0, 2, 2])?0 ; |
1211 | 1 | verify_longest_length(&[Arc::clone(&list_array)], true, vec![2, 1, 2, 1, 2, 2])?0 ; |
1212 | | |
1213 | | // Test with multiple list arrays |
1214 | | // [A, B, C], [], NULL, [D], NULL, [NULL, F] |
1215 | | // [A, B], NULL, [C, D], NULL, [NULL, F], [NULL, NULL] |
1216 | 1 | let list1 = Arc::new(make_generic_array::<i32>()) as ArrayRef; |
1217 | 1 | let list2 = Arc::new(make_fixed_list()) as ArrayRef; |
1218 | 1 | let list_arrays = vec![Arc::clone(&list1), Arc::clone(&list2)]; |
1219 | 1 | verify_longest_length(&list_arrays, false, vec![3, 0, 2, 1, 2, 2])?0 ; |
1220 | 1 | verify_longest_length(&list_arrays, true, vec![3, 1, 2, 1, 2, 2])?0 ; |
1221 | | |
1222 | 1 | Ok(()) |
1223 | 1 | } |
1224 | | |
1225 | | #[test] |
1226 | 1 | fn test_create_take_indicies() -> datafusion_common::Result<()> { |
1227 | 1 | let length_array = Int64Array::from(vec![2, 3, 1]); |
1228 | 1 | let take_indicies = create_take_indicies(&length_array, 6); |
1229 | 1 | let expected = Int64Array::from(vec![0, 0, 1, 1, 1, 2]); |
1230 | 1 | assert_eq!(take_indicies, expected); |
1231 | 1 | Ok(()) |
1232 | 1 | } |
1233 | | } |