vortex_layout/layouts/struct_/
writer.rs

1// SPDX-License-Identifier: Apache-2.0
2// SPDX-FileCopyrightText: Copyright the Vortex contributors
3
4use std::collections::VecDeque;
5use std::pin::Pin;
6use std::sync::Arc;
7use std::task::{Context, Poll, Waker};
8
9use arcref::ArcRef;
10use futures::future::try_join_all;
11use futures::{Stream, StreamExt, TryStreamExt};
12use itertools::Itertools;
13use parking_lot::Mutex;
14use vortex_array::{Array, ArrayContext, ToCanonical};
15use vortex_error::{VortexExpect as _, VortexResult, vortex_bail};
16use vortex_utils::aliases::DefaultHashBuilder;
17use vortex_utils::aliases::hash_set::HashSet;
18
19use crate::layouts::struct_::StructLayout;
20use crate::segments::SequenceWriter;
21use crate::{
22    IntoLayout as _, LayoutStrategy, SendableLayoutFuture, SendableSequentialStream,
23    SequentialStreamAdapter, SequentialStreamExt,
24};
25
26pub struct StructStrategy {
27    child: ArcRef<dyn LayoutStrategy>,
28}
29
30/// A [`LayoutStrategy`] that splits a StructArray batch into child layout writers
31impl StructStrategy {
32    pub fn new(child: ArcRef<dyn LayoutStrategy>) -> Self {
33        Self { child }
34    }
35}
36
37impl LayoutStrategy for StructStrategy {
38    fn write_stream(
39        &self,
40        ctx: &ArrayContext,
41        sequence_writer: SequenceWriter,
42        stream: SendableSequentialStream,
43    ) -> SendableLayoutFuture {
44        let dtype = stream.dtype().clone();
45        let Some(struct_dtype) = stream.dtype().as_struct().cloned() else {
46            // nothing we can do if dtype is not struct
47            return self.child.write_stream(ctx, sequence_writer, stream);
48        };
49        if HashSet::<_, DefaultHashBuilder>::from_iter(struct_dtype.names().iter()).len()
50            != struct_dtype.names().len()
51        {
52            return Box::pin(async { vortex_bail!("StructLayout must have unique field names") });
53        }
54
55        let stream = stream.map(|chunk| {
56            let (sequence_id, chunk) = chunk?;
57            if !chunk.all_valid()? {
58                vortex_bail!("Cannot push struct chunks with top level invalid values");
59            };
60            Ok((sequence_id, chunk))
61        });
62
63        // There are now fields so this is the layout leaf
64        if struct_dtype.nfields() == 0 {
65            return Box::pin(async move {
66                let row_count = stream
67                    .try_fold(
68                        0u64,
69                        |acc, (_, arr)| async move { Ok(acc + arr.len() as u64) },
70                    )
71                    .await?;
72                Ok(StructLayout::new(row_count, dtype, vec![]).into_layout())
73            });
74        }
75
76        // stream<struct_chunk> -> stream<vec<column_chunk>>
77        let columns_vec_stream = stream.map(|chunk| {
78            let (sequence_id, chunk) = chunk?;
79            let mut sequence_pointer = sequence_id.descend();
80            let struct_chunk = chunk.to_struct()?;
81            let columns: Vec<_> = (0..struct_chunk.struct_fields().nfields())
82                .map(|idx| {
83                    (
84                        sequence_pointer.advance(),
85                        struct_chunk.fields()[idx].to_array(),
86                    )
87                })
88                .collect();
89            Ok(columns)
90        });
91
92        // stream<vec<column_chunk>> -> vec<stream<column_chunk>>
93        let column_streams = transpose_stream(columns_vec_stream, struct_dtype.nfields());
94
95        let column_dtypes = (0..struct_dtype.nfields()).map(move |idx| {
96            struct_dtype
97                .field_by_index(idx)
98                .vortex_expect("bound checked")
99        });
100        let child = self.child.clone();
101        let ctx = ctx.clone();
102        let layout_futures = column_dtypes
103            .zip_eq(column_streams)
104            .map(move |(dtype, stream)| {
105                let column_stream = SequentialStreamAdapter::new(dtype, stream).sendable();
106                child.write_stream(&ctx, sequence_writer.clone(), column_stream)
107            });
108
109        Box::pin(async move {
110            let column_layouts = try_join_all(layout_futures).await?;
111            // TODO(os): transposed stream could count row counts as well,
112            // This must hold though, all columns must have the same row count of the struct layout
113            let row_count = column_layouts.first().map(|l| l.row_count()).unwrap_or(0);
114            Ok(StructLayout::new(row_count, dtype, column_layouts).into_layout())
115        })
116    }
117}
118
119fn transpose_stream<T, S>(stream: S, elements: usize) -> Vec<impl Stream<Item = VortexResult<T>>>
120where
121    S: Stream<Item = VortexResult<Vec<T>>> + Unpin,
122    T: Unpin + 'static,
123{
124    let state = Arc::new(Mutex::new(TransposeState {
125        upstream: stream,
126        buffers: (0..elements).map(|_| VecDeque::new()).collect(),
127        wakers: Vec::new(),
128        exhausted: false,
129    }));
130    (0..elements)
131        .map(|index| TransposedStream {
132            index,
133            state: state.clone(),
134        })
135        .collect()
136}
137
138struct TransposeState<T, S>
139where
140    S: Stream<Item = VortexResult<Vec<T>>> + Unpin,
141    T: Unpin,
142{
143    upstream: S,
144    // TODO(os): make these buffers bounded so transposed streams can not run ahead unbounded
145    buffers: Vec<VecDeque<VortexResult<T>>>,
146    wakers: Vec<Waker>,
147    exhausted: bool,
148}
149
150struct TransposedStream<T, S>
151where
152    S: Stream<Item = VortexResult<Vec<T>>> + Unpin,
153    T: Unpin,
154{
155    index: usize,
156    state: Arc<Mutex<TransposeState<T, S>>>,
157}
158
159impl<T, S> Stream for TransposedStream<T, S>
160where
161    S: Stream<Item = VortexResult<Vec<T>>> + Unpin,
162    T: Unpin,
163{
164    type Item = VortexResult<T>;
165    fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Option<Self::Item>> {
166        let mut guard = self.state.lock();
167        if let Some(item) = guard.buffers[self.index].pop_front() {
168            return Poll::Ready(Some(item));
169        }
170
171        // if we know upstream is exhausted we can skip polling it again.
172        if guard.exhausted {
173            return Poll::Ready(None);
174        }
175
176        match Pin::new(&mut guard.upstream).poll_next(cx) {
177            Poll::Pending => {
178                guard.wakers.push(cx.waker().clone());
179                Poll::Pending
180            }
181            Poll::Ready(None) => {
182                guard.exhausted = true;
183                Poll::Ready(None)
184            }
185            Poll::Ready(Some(Ok(vec_t))) => {
186                for (t, buffer) in vec_t.into_iter().zip_eq(guard.buffers.iter_mut()) {
187                    buffer.push_back(Ok(t));
188                }
189                let item = guard.buffers[self.index]
190                    .pop_front()
191                    .vortex_expect("just pushed");
192                let wakers = std::mem::take(&mut guard.wakers);
193
194                drop(guard);
195                for waker in wakers {
196                    waker.wake();
197                }
198
199                Poll::Ready(Some(item))
200            }
201            Poll::Ready(Some(Err(err))) => {
202                let shared_err = Arc::new(err);
203                for buffer in guard.buffers.iter_mut() {
204                    buffer.push_back(Err(shared_err.clone().into()));
205                }
206                Poll::Ready(Some(Err(shared_err.into())))
207            }
208        }
209    }
210}
211
212#[cfg(test)]
213mod tests {
214    use std::sync::Arc;
215
216    use arcref::ArcRef;
217    use futures::executor::block_on;
218    use futures::stream;
219    use vortex_array::arrays::{BoolArray, StructArray};
220    use vortex_array::validity::Validity;
221    use vortex_array::{ArrayContext, IntoArray as _};
222    use vortex_buffer::buffer;
223    use vortex_dtype::{DType, FieldNames, Nullability, PType, StructFields};
224
225    use crate::layouts::flat::writer::FlatLayoutStrategy;
226    use crate::layouts::struct_::writer::StructStrategy;
227    use crate::segments::{SequenceWriter, TestSegments};
228    use crate::sequence::SequenceId;
229    use crate::{LayoutStrategy, SequentialStreamAdapter, SequentialStreamExt};
230
231    #[test]
232    #[should_panic]
233    fn fails_on_duplicate_field() {
234        let strategy =
235            StructStrategy::new(ArcRef::new_arc(Arc::new(FlatLayoutStrategy::default())));
236        block_on(
237            strategy.write_stream(
238                &ArrayContext::empty(),
239                SequenceWriter::new(Box::new(TestSegments::default())),
240                SequentialStreamAdapter::new(
241                    DType::Struct(
242                        [
243                            ("a", DType::Primitive(PType::I32, Nullability::NonNullable)),
244                            ("a", DType::Primitive(PType::I32, Nullability::NonNullable)),
245                        ]
246                        .into_iter()
247                        .collect(),
248                        Nullability::NonNullable,
249                    ),
250                    stream::empty(),
251                )
252                .sendable(),
253            ),
254        )
255        .unwrap();
256    }
257
258    #[test]
259    fn fails_on_top_level_nulls() {
260        let strategy =
261            StructStrategy::new(ArcRef::new_arc(Arc::new(FlatLayoutStrategy::default())));
262        let res = block_on(
263            strategy.write_stream(
264                &ArrayContext::empty(),
265                SequenceWriter::new(Box::new(TestSegments::default())),
266                SequentialStreamAdapter::new(
267                    DType::Struct(
268                        [("a", DType::Primitive(PType::I32, Nullability::NonNullable))]
269                            .into_iter()
270                            .collect(),
271                        Nullability::Nullable,
272                    ),
273                    stream::once(async move {
274                        Ok((
275                            SequenceId::root().downgrade(),
276                            StructArray::try_new(
277                                ["a"].into(),
278                                vec![buffer![1, 2, 3].into_array()],
279                                3,
280                                Validity::Array(
281                                    BoolArray::from_iter(vec![true, true, false]).into_array(),
282                                ),
283                            )
284                            .unwrap()
285                            .into_array(),
286                        ))
287                    }),
288                )
289                .sendable(),
290            ),
291        );
292        assert!(
293            format!("{}", res.unwrap_err())
294                .starts_with("Cannot push struct chunks with top level invalid values"),
295        )
296    }
297
298    #[test]
299    fn write_empty_field_struct_array() {
300        let strategy =
301            StructStrategy::new(ArcRef::new_arc(Arc::new(FlatLayoutStrategy::default())));
302        let res = block_on(
303            strategy.write_stream(
304                &ArrayContext::empty(),
305                SequenceWriter::new(Box::new(TestSegments::default())),
306                SequentialStreamAdapter::new(
307                    DType::Struct(
308                        StructFields::new(FieldNames::default(), vec![]),
309                        Nullability::NonNullable,
310                    ),
311                    stream::iter([
312                        {
313                            Ok((
314                                SequenceId::root().downgrade(),
315                                StructArray::try_new(
316                                    FieldNames::default(),
317                                    vec![],
318                                    3,
319                                    Validity::NonNullable,
320                                )
321                                .unwrap()
322                                .into_array(),
323                            ))
324                        },
325                        {
326                            Ok((
327                                SequenceId::root().advance(),
328                                StructArray::try_new(
329                                    FieldNames::default(),
330                                    vec![],
331                                    5,
332                                    Validity::NonNullable,
333                                )
334                                .unwrap()
335                                .into_array(),
336                            ))
337                        },
338                    ]),
339                )
340                .sendable(),
341            ),
342        );
343
344        assert_eq!(res.unwrap().row_count(), 8);
345    }
346}