|
| 1 | +use std::path::Path; |
| 2 | +use std::pin::Pin; |
| 3 | +use std::task::{Context, Poll}; |
| 4 | + |
| 5 | +use arrow_array::StructArray as ArrowStructArray; |
| 6 | +use futures_util::Stream; |
| 7 | +use indicatif::ProgressBar; |
| 8 | +use parquet::arrow::ParquetRecordBatchStreamBuilder; |
| 9 | +use pin_project::pin_project; |
| 10 | +use tokio::fs::File; |
| 11 | +use vortex::arrays::ChunkedArray; |
| 12 | +use vortex::arrow::FromArrowArray; |
| 13 | +use vortex::dtype::DType; |
| 14 | +use vortex::error::{VortexExpect, VortexResult}; |
| 15 | +use vortex::file::VortexWriteOptions; |
| 16 | +use vortex::stream::{ArrayStream, ArrayStreamArrayExt}; |
| 17 | +use vortex::{Array, ArrayRef}; |
| 18 | + |
| 19 | +/// Convert Parquet files to Vortex. |
| 20 | +pub async fn exec_convert(input_path: impl AsRef<Path>) -> VortexResult<()> { |
| 21 | + println!( |
| 22 | + "Converting input Parquet file: {}", |
| 23 | + input_path.as_ref().display() |
| 24 | + ); |
| 25 | + |
| 26 | + let output_path = input_path.as_ref().with_extension("vortex"); |
| 27 | + let file = File::open(input_path).await?; |
| 28 | + let mut reader = ParquetRecordBatchStreamBuilder::new(file).await?.build()?; |
| 29 | + let mut chunks = Vec::new(); |
| 30 | + |
| 31 | + while let Some(mut reader) = reader.next_row_group().await? { |
| 32 | + for batch in reader.by_ref() { |
| 33 | + let batch = ArrowStructArray::from(batch?); |
| 34 | + let next_chunk = ArrayRef::from_arrow(&batch, true); |
| 35 | + chunks.push(next_chunk); |
| 36 | + } |
| 37 | + } |
| 38 | + |
| 39 | + let dtype = chunks.first().vortex_expect("empty chunks").dtype().clone(); |
| 40 | + let chunked_array = ChunkedArray::try_new(chunks, dtype)?; |
| 41 | + |
| 42 | + let writer = VortexWriteOptions::default(); |
| 43 | + |
| 44 | + let pb = ProgressBar::new(chunked_array.nchunks() as u64); |
| 45 | + let stream = ProgressArrayStream { |
| 46 | + inner: chunked_array.to_array_stream(), |
| 47 | + progress: pb, |
| 48 | + }; |
| 49 | + |
| 50 | + println!( |
| 51 | + "Writing {} chunks to new Vortex file {}", |
| 52 | + chunked_array.nchunks(), |
| 53 | + output_path.display() |
| 54 | + ); |
| 55 | + let output_file = File::create(output_path).await?; |
| 56 | + writer.write(output_file, stream).await?; |
| 57 | + |
| 58 | + Ok(()) |
| 59 | +} |
| 60 | + |
| 61 | +#[pin_project] |
| 62 | +struct ProgressArrayStream<S> { |
| 63 | + #[pin] |
| 64 | + inner: S, |
| 65 | + progress: ProgressBar, |
| 66 | +} |
| 67 | + |
| 68 | +impl<S> Stream for ProgressArrayStream<S> |
| 69 | +where |
| 70 | + S: Stream<Item = VortexResult<ArrayRef>>, |
| 71 | +{ |
| 72 | + type Item = VortexResult<ArrayRef>; |
| 73 | + |
| 74 | + fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Option<Self::Item>> { |
| 75 | + let this = self.project(); |
| 76 | + match this.inner.poll_next(cx) { |
| 77 | + Poll::Ready(inner) => { |
| 78 | + this.progress.inc(1); |
| 79 | + Poll::Ready(inner) |
| 80 | + } |
| 81 | + Poll::Pending => Poll::Pending, |
| 82 | + } |
| 83 | + } |
| 84 | +} |
| 85 | + |
| 86 | +impl<S> ArrayStream for ProgressArrayStream<S> |
| 87 | +where |
| 88 | + S: ArrayStream, |
| 89 | +{ |
| 90 | + fn dtype(&self) -> &DType { |
| 91 | + self.inner.dtype() |
| 92 | + } |
| 93 | +} |
0 commit comments