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, ready};
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        exhausted: false,
128    }));
129    (0..elements)
130        .map(|index| TransposedStream {
131            index,
132            state: state.clone(),
133        })
134        .collect()
135}
136
137struct TransposeState<T, S>
138where
139    S: Stream<Item = VortexResult<Vec<T>>> + Unpin,
140    T: Unpin,
141{
142    upstream: S,
143    // TODO(os): make these buffers bounded so transposed streams can not run ahead unbounded
144    buffers: Vec<VecDeque<VortexResult<T>>>,
145    exhausted: bool,
146}
147
148struct TransposedStream<T, S>
149where
150    S: Stream<Item = VortexResult<Vec<T>>> + Unpin,
151    T: Unpin,
152{
153    index: usize,
154    state: Arc<Mutex<TransposeState<T, S>>>,
155}
156
157impl<T, S> Stream for TransposedStream<T, S>
158where
159    S: Stream<Item = VortexResult<Vec<T>>> + Unpin,
160    T: Unpin,
161{
162    type Item = VortexResult<T>;
163    fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Option<Self::Item>> {
164        let mut guard = self.state.lock();
165        if let Some(item) = guard.buffers[self.index].pop_front() {
166            return Poll::Ready(Some(item));
167        }
168
169        // if we know upstream is exhausted we can skip polling it again.
170        if guard.exhausted {
171            return Poll::Ready(None);
172        }
173
174        match ready!(Pin::new(&mut guard.upstream).poll_next(cx)) {
175            None => {
176                guard.exhausted = true;
177                Poll::Ready(None)
178            }
179            Some(Ok(vec_t)) => {
180                for (t, buffer) in vec_t.into_iter().zip_eq(guard.buffers.iter_mut()) {
181                    buffer.push_back(Ok(t));
182                }
183                Poll::Ready(Some(
184                    guard.buffers[self.index]
185                        .pop_front()
186                        .vortex_expect("just pushed"),
187                ))
188            }
189            Some(Err(err)) => {
190                let shared_err = Arc::new(err);
191                for buffer in guard.buffers.iter_mut() {
192                    buffer.push_back(Err(shared_err.clone().into()));
193                }
194                Poll::Ready(Some(Err(shared_err.into())))
195            }
196        }
197    }
198}
199
200#[cfg(test)]
201mod tests {
202    use std::sync::Arc;
203
204    use arcref::ArcRef;
205    use futures::executor::block_on;
206    use futures::stream;
207    use vortex_array::arrays::{BoolArray, StructArray};
208    use vortex_array::validity::Validity;
209    use vortex_array::{ArrayContext, IntoArray as _};
210    use vortex_buffer::buffer;
211    use vortex_dtype::{DType, FieldNames, Nullability, PType, StructFields};
212
213    use crate::layouts::flat::writer::FlatLayoutStrategy;
214    use crate::layouts::struct_::writer::StructStrategy;
215    use crate::segments::{SequenceWriter, TestSegments};
216    use crate::sequence::SequenceId;
217    use crate::{LayoutStrategy, SequentialStreamAdapter, SequentialStreamExt};
218
219    #[test]
220    #[should_panic]
221    fn fails_on_duplicate_field() {
222        let strategy =
223            StructStrategy::new(ArcRef::new_arc(Arc::new(FlatLayoutStrategy::default())));
224        block_on(
225            strategy.write_stream(
226                &ArrayContext::empty(),
227                SequenceWriter::new(Box::new(TestSegments::default())),
228                SequentialStreamAdapter::new(
229                    DType::Struct(
230                        [
231                            ("a", DType::Primitive(PType::I32, Nullability::NonNullable)),
232                            ("a", DType::Primitive(PType::I32, Nullability::NonNullable)),
233                        ]
234                        .into_iter()
235                        .collect(),
236                        Nullability::NonNullable,
237                    ),
238                    stream::empty(),
239                )
240                .sendable(),
241            ),
242        )
243        .unwrap();
244    }
245
246    #[test]
247    fn fails_on_top_level_nulls() {
248        let strategy =
249            StructStrategy::new(ArcRef::new_arc(Arc::new(FlatLayoutStrategy::default())));
250        let res = block_on(
251            strategy.write_stream(
252                &ArrayContext::empty(),
253                SequenceWriter::new(Box::new(TestSegments::default())),
254                SequentialStreamAdapter::new(
255                    DType::Struct(
256                        [("a", DType::Primitive(PType::I32, Nullability::NonNullable))]
257                            .into_iter()
258                            .collect(),
259                        Nullability::Nullable,
260                    ),
261                    stream::once(async move {
262                        Ok((
263                            SequenceId::root().downgrade(),
264                            StructArray::try_new(
265                                ["a"].into(),
266                                vec![buffer![1, 2, 3].into_array()],
267                                3,
268                                Validity::Array(
269                                    BoolArray::from_iter(vec![true, true, false]).into_array(),
270                                ),
271                            )
272                            .unwrap()
273                            .into_array(),
274                        ))
275                    }),
276                )
277                .sendable(),
278            ),
279        );
280        assert!(
281            format!("{}", res.unwrap_err())
282                .starts_with("Cannot push struct chunks with top level invalid values"),
283        )
284    }
285
286    #[test]
287    fn write_empty_field_struct_array() {
288        let strategy =
289            StructStrategy::new(ArcRef::new_arc(Arc::new(FlatLayoutStrategy::default())));
290        let res = block_on(
291            strategy.write_stream(
292                &ArrayContext::empty(),
293                SequenceWriter::new(Box::new(TestSegments::default())),
294                SequentialStreamAdapter::new(
295                    DType::Struct(
296                        StructFields::new(FieldNames::default(), vec![]),
297                        Nullability::NonNullable,
298                    ),
299                    stream::iter([
300                        {
301                            Ok((
302                                SequenceId::root().downgrade(),
303                                StructArray::try_new(
304                                    FieldNames::default(),
305                                    vec![],
306                                    3,
307                                    Validity::NonNullable,
308                                )
309                                .unwrap()
310                                .into_array(),
311                            ))
312                        },
313                        {
314                            Ok((
315                                SequenceId::root().advance(),
316                                StructArray::try_new(
317                                    FieldNames::default(),
318                                    vec![],
319                                    5,
320                                    Validity::NonNullable,
321                                )
322                                .unwrap()
323                                .into_array(),
324                            ))
325                        },
326                    ]),
327                )
328                .sendable(),
329            ),
330        );
331
332        assert_eq!(res.unwrap().row_count(), 8);
333    }
334}