vortex_layout/layouts/struct_/
writer.rs1use 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
30impl 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 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 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 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 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 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 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 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}