1use std::sync::Arc;
5
6use futures::future::try_join_all;
7use termtree::Tree;
8use vortex_array::serde::ArrayParts;
9use vortex_error::VortexResult;
10use vortex_utils::aliases::hash_map::HashMap;
11
12use crate::LayoutRef;
13use crate::layouts::flat::FlatLayout;
14use crate::layouts::flat::FlatVTable;
15use crate::segments::SegmentId;
16use crate::segments::SegmentSource;
17
18pub(super) async fn display_tree_with_segment_sizes(
25 layout: LayoutRef,
26 segment_source: Arc<dyn SegmentSource>,
27) -> VortexResult<DisplayLayoutTree> {
28 let mut segments_to_fetch = Vec::new();
30 collect_segments_to_fetch(&layout, &mut segments_to_fetch)?;
31 segments_to_fetch.dedup();
32
33 let fetch_futures = segments_to_fetch.iter().map(|&segment_id| {
35 let segment_source = segment_source.clone();
36 async move {
37 let buffer = segment_source.request(segment_id).await?;
38 let parts = ArrayParts::try_from(buffer)?;
39 VortexResult::Ok((segment_id, parts.buffer_lengths()))
40 }
41 });
42 let results = try_join_all(fetch_futures).await?;
43 let segment_buffer_sizes: HashMap<SegmentId, Vec<usize>> = results.into_iter().collect();
44
45 Ok(DisplayLayoutTree {
46 layout,
47 segment_buffer_sizes: Some(segment_buffer_sizes),
48 verbose: true,
49 })
50}
51
52fn collect_segments_to_fetch(
54 layout: &LayoutRef,
55 segment_ids: &mut Vec<SegmentId>,
56) -> VortexResult<()> {
57 if let Some(flat_layout) = layout.as_opt::<FlatVTable>() {
59 if flat_layout.array_tree().is_none() {
60 segment_ids.push(flat_layout.segment_id());
61 }
62 } else {
63 segment_ids.extend(layout.segment_ids());
65 }
66
67 for child in layout.children()? {
69 collect_segments_to_fetch(&child, segment_ids)?;
70 }
71 Ok(())
72}
73
74fn format_flat_layout_buffers(
76 flat_layout: &FlatLayout,
77 segment_buffer_sizes: Option<&HashMap<SegmentId, Vec<usize>>>,
78) -> String {
79 let segment_id = flat_layout.segment_id();
80
81 if let Some(array_tree) = flat_layout.array_tree()
83 && let Ok(parts) = ArrayParts::from_array_tree(array_tree.as_ref().to_vec())
84 {
85 return format_buffer_sizes(&parts.buffer_lengths(), *segment_id);
86 }
87
88 if let Some(sizes_map) = segment_buffer_sizes
90 && let Some(buffer_sizes) = sizes_map.get(&segment_id)
91 {
92 return format_buffer_sizes(buffer_sizes, *segment_id);
93 }
94
95 format!("segment: {}", *segment_id)
97}
98
99fn format_buffer_sizes(buffer_sizes: &[usize], segment_id: u32) -> String {
100 let buffer_sizes_str: Vec<String> = buffer_sizes.iter().map(|s| format!("{}B", s)).collect();
101 let total: usize = buffer_sizes.iter().sum();
102 format!(
103 "segment {}, buffers=[{}], total={}B",
104 segment_id,
105 buffer_sizes_str.join(", "),
106 total
107 )
108}
109
110pub struct DisplayLayoutTree {
112 layout: LayoutRef,
113 segment_buffer_sizes: Option<HashMap<SegmentId, Vec<usize>>>,
114 verbose: bool,
115}
116
117impl DisplayLayoutTree {
118 pub fn new(layout: LayoutRef, verbose: bool) -> Self {
120 Self {
121 layout,
122 segment_buffer_sizes: None,
123 verbose,
124 }
125 }
126
127 fn make_tree(&self, layout: LayoutRef) -> VortexResult<Tree<String>> {
128 let mut node_parts = vec![
130 format!("{}", layout.encoding()),
131 format!("dtype: {}", layout.dtype()),
132 ];
133
134 let nchildren = layout.nchildren();
136 if nchildren > 0 {
137 node_parts.push(format!("children: {}", nchildren));
138 }
139
140 if self.verbose {
142 let metadata = layout.metadata();
143 if !metadata.is_empty() {
144 node_parts.push(format!("metadata: {} bytes", metadata.len()));
145 }
146 node_parts.push(format!("rows: {}", layout.row_count()));
147 }
148
149 if let Some(flat_layout) = layout.as_opt::<FlatVTable>() {
151 node_parts.push(format_flat_layout_buffers(
152 flat_layout,
153 self.segment_buffer_sizes.as_ref(),
154 ));
155 } else {
156 if self.verbose {
158 let segment_ids = layout.segment_ids();
159 if !segment_ids.is_empty() {
160 node_parts.push(format!(
161 "segments: [{}]",
162 segment_ids
163 .iter()
164 .map(|s| format!("{}", **s))
165 .collect::<Vec<_>>()
166 .join(", ")
167 ));
168 }
169 }
170 }
171
172 let node_name = node_parts.join(", ");
173
174 let children = layout.children()?;
176 let child_names: Vec<_> = layout.child_names().collect();
177
178 let child_trees: VortexResult<Vec<Tree<String>>> =
180 if !children.is_empty() && child_names.len() == children.len() {
181 children
183 .into_iter()
184 .zip(child_names.iter())
185 .map(|(child, name)| {
186 let child_tree = self.make_tree(child)?;
187 Ok(Tree::new(format!("{}: {}", name, child_tree.root))
188 .with_leaves(child_tree.leaves))
189 })
190 .collect()
191 } else if !children.is_empty() {
192 children.into_iter().map(|c| self.make_tree(c)).collect()
194 } else {
195 Ok(Vec::new())
197 };
198
199 Ok(Tree::new(node_name).with_leaves(child_trees?))
200 }
201}
202
203impl std::fmt::Display for DisplayLayoutTree {
204 fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
205 match self.make_tree(self.layout.clone()) {
206 Ok(tree) => write!(f, "{}", tree),
207 Err(e) => write!(f, "Error building layout tree: {}", e),
208 }
209 }
210}
211
212#[cfg(test)]
213mod tests {
214 use std::sync::Arc;
215
216 use vortex_array::ArrayContext;
217 use vortex_array::IntoArray;
218 use vortex_array::arrays::BoolArray;
219 use vortex_array::arrays::PrimitiveArray;
220 use vortex_array::builders::ArrayBuilder;
221 use vortex_array::builders::VarBinViewBuilder;
222 use vortex_array::serde::ArrayParts;
223 use vortex_array::validity::Validity;
224 use vortex_buffer::BitBufferMut;
225 use vortex_buffer::buffer;
226 use vortex_dtype::DType;
227 use vortex_dtype::FieldName;
228 use vortex_dtype::Nullability;
229 use vortex_dtype::Nullability::NonNullable;
230 use vortex_dtype::PType;
231 use vortex_dtype::StructFields;
232 use vortex_io::runtime::single::block_on;
233 use vortex_utils::env::EnvVarGuard;
234
235 use crate::IntoLayout;
236 use crate::OwnedLayoutChildren;
237 use crate::layouts::chunked::ChunkedLayout;
238 use crate::layouts::flat::FlatVTable;
239 use crate::layouts::flat::writer::FlatLayoutStrategy;
240 use crate::layouts::struct_::StructLayout;
241 use crate::segments::TestSegments;
242 use crate::sequence::SequenceId;
243 use crate::sequence::SequentialArrayStreamExt;
244 use crate::strategy::LayoutStrategy;
245
246 #[test]
248 fn test_display_tree_inline_array_tree() {
249 let _guard = EnvVarGuard::set("FLAT_LAYOUT_INLINE_ARRAY_NODE", "1");
250 block_on(|handle| async move {
251 let ctx = ArrayContext::empty();
252 let segments = Arc::new(TestSegments::default());
253
254 let (ptr1, eof1) = SequenceId::root().split();
256 let mut validity_builder = BitBufferMut::with_capacity(5);
257 for b in [true, false, true, true, false] {
258 validity_builder.append(b);
259 }
260 let validity = Validity::Array(
261 BoolArray::from_bit_buffer(validity_builder.freeze(), Validity::NonNullable)
262 .into_array(),
263 );
264 let array1 = PrimitiveArray::new(buffer![1i64, 2, 3, 4, 5], validity);
265 let layout1 = FlatLayoutStrategy::default()
266 .write_stream(
267 ctx.clone(),
268 segments.clone(),
269 array1.to_array_stream().sequenced(ptr1),
270 eof1,
271 handle.clone(),
272 )
273 .await
274 .unwrap();
275
276 let (ptr2, eof2) = SequenceId::root().split();
278 let mut builder = VarBinViewBuilder::with_capacity(DType::Utf8(NonNullable), 5);
279 for s in [
280 "hello world this is long",
281 "another long string",
282 "short",
283 "medium str",
284 "x",
285 ] {
286 builder.append_value(s);
287 }
288 let layout2 = FlatLayoutStrategy::default()
289 .write_stream(
290 ctx.clone(),
291 segments.clone(),
292 builder.finish().to_array_stream().sequenced(ptr2),
293 eof2,
294 handle.clone(),
295 )
296 .await
297 .unwrap();
298
299 let struct_layout = StructLayout::new(
301 5,
302 DType::Struct(
303 StructFields::new(
304 vec![FieldName::from("numbers"), FieldName::from("strings")].into(),
305 vec![
306 DType::Primitive(PType::I64, Nullability::Nullable),
307 DType::Utf8(NonNullable),
308 ],
309 ),
310 NonNullable,
311 ),
312 vec![
313 ChunkedLayout::new(
314 5,
315 DType::Primitive(PType::I64, Nullability::Nullable),
316 OwnedLayoutChildren::layout_children(vec![layout1]),
317 )
318 .into_layout(),
319 layout2,
320 ],
321 )
322 .into_layout();
323
324 let output = format!("{}", struct_layout.display_tree_verbose(true));
325
326 let expected = "\
327vortex.struct, dtype: {numbers=i64?, strings=utf8}, children: 2, rows: 5
328├── numbers: vortex.chunked, dtype: i64?, children: 1, rows: 5
329│ └── [0]: vortex.flat, dtype: i64?, metadata: 171 bytes, rows: 5, segment 0, buffers=[40B, 1B], total=41B
330└── strings: vortex.flat, dtype: utf8, metadata: 110 bytes, rows: 5, segment 1, buffers=[43B, 80B], total=123B
331";
332 assert_eq!(output, expected);
333 })
334 }
335
336 #[test]
338 fn test_display_tree_with_segment_source() {
339 let _guard = EnvVarGuard::remove("FLAT_LAYOUT_INLINE_ARRAY_NODE");
341 block_on(|handle| async move {
342 let ctx = ArrayContext::empty();
343 let segments = Arc::new(TestSegments::default());
344
345 let (ptr1, eof1) = SequenceId::root().split();
347 let array1 = PrimitiveArray::new(buffer![1i32, 2, 3, 4, 5], Validity::NonNullable);
348 let layout1 = FlatLayoutStrategy::default()
349 .write_stream(
350 ctx.clone(),
351 segments.clone(),
352 array1.to_array_stream().sequenced(ptr1),
353 eof1,
354 handle.clone(),
355 )
356 .await
357 .unwrap();
358
359 let (ptr2, eof2) = SequenceId::root().split();
361 let array2 = PrimitiveArray::new(buffer![6i32, 7, 8, 9, 10], Validity::NonNullable);
362 let layout2 = FlatLayoutStrategy::default()
363 .write_stream(
364 ctx.clone(),
365 segments.clone(),
366 array2.to_array_stream().sequenced(ptr2),
367 eof2,
368 handle.clone(),
369 )
370 .await
371 .unwrap();
372
373 let chunked_layout = ChunkedLayout::new(
375 10,
376 DType::Primitive(PType::I32, NonNullable),
377 OwnedLayoutChildren::layout_children(vec![layout1, layout2]),
378 )
379 .into_layout();
380
381 let output = chunked_layout
382 .display_tree_with_segments(segments)
383 .await
384 .unwrap();
385
386 let expected = "\
387vortex.chunked, dtype: i32, children: 2, rows: 10
388├── [0]: vortex.flat, dtype: i32, rows: 5, segment 0, buffers=[20B], total=20B
389└── [1]: vortex.flat, dtype: i32, rows: 5, segment 1, buffers=[20B], total=20B
390";
391 assert_eq!(output.to_string(), expected);
392 })
393 }
394
395 #[test]
397 fn test_display_array_tree_with_inline_node() {
398 let _guard = EnvVarGuard::set("FLAT_LAYOUT_INLINE_ARRAY_NODE", "1");
399
400 let ctx = ArrayContext::empty();
401 let segments = Arc::new(TestSegments::default());
402 let (ptr, eof) = SequenceId::root().split();
403
404 let array = PrimitiveArray::new(buffer![1i32, 2, 3, 4, 5], Validity::AllValid);
406 let layout = block_on(|handle| async {
407 FlatLayoutStrategy::default()
408 .write_stream(
409 ctx.clone(),
410 segments.clone(),
411 array.to_array_stream().sequenced(ptr),
412 eof,
413 handle,
414 )
415 .await
416 .unwrap()
417 });
418
419 let flat_layout = layout.as_::<FlatVTable>();
420
421 let array_tree = flat_layout
422 .array_tree()
423 .expect("array_tree should be populated when FLAT_LAYOUT_INLINE_ARRAY_NODE is set");
424
425 let parts = ArrayParts::from_array_tree(array_tree.as_ref().to_vec())
426 .expect("should parse array_tree");
427 assert_eq!(parts.buffer_lengths(), vec![20]); assert_eq!(
430 layout.display_tree().to_string(),
431 "\
432vortex.flat, dtype: i32?, segment 0, buffers=[20B], total=20B
433"
434 );
435 }
436
437 #[test]
439 fn test_display_tree_without_inline_node() {
440 let _guard = EnvVarGuard::set("FLAT_LAYOUT_INLINE_ARRAY_NODE", "1");
441
442 let ctx = ArrayContext::empty();
443 let segments = Arc::new(TestSegments::default());
444 let (ptr, eof) = SequenceId::root().split();
445
446 let array = PrimitiveArray::new(buffer![10i64, 20, 30], Validity::NonNullable);
448 let layout = block_on(|handle| async {
449 FlatLayoutStrategy::default()
450 .write_stream(
451 ctx,
452 segments.clone(),
453 array.to_array_stream().sequenced(ptr),
454 eof,
455 handle,
456 )
457 .await
458 .unwrap()
459 });
460
461 assert_eq!(
463 layout.display_tree().to_string(),
464 "\
465vortex.flat, dtype: i64, segment 0, buffers=[24B], total=24B
466"
467 );
468 }
469}