1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
use arrow_format::ipc::planus::ReadAsRoot;
use futures::future::BoxFuture;
use futures::AsyncRead;
use futures::AsyncReadExt;
use futures::FutureExt;
use futures::Stream;
use crate::array::*;
use crate::chunk::Chunk;
use crate::error::{Error, Result};
use super::super::CONTINUATION_MARKER;
use super::common::{read_dictionary, read_record_batch};
use super::schema::deserialize_stream_metadata;
use super::Dictionaries;
use super::OutOfSpecKind;
use super::StreamMetadata;
struct ReadState<R> {
pub reader: R,
pub metadata: StreamMetadata,
pub dictionaries: Dictionaries,
pub data_buffer: Vec<u8>,
pub message_buffer: Vec<u8>,
}
enum StreamState<R> {
Waiting(ReadState<R>),
Some((ReadState<R>, Chunk<Box<dyn Array>>)),
}
pub async fn read_stream_metadata_async<R: AsyncRead + Unpin + Send>(
reader: &mut R,
) -> Result<StreamMetadata> {
let mut meta_size: [u8; 4] = [0; 4];
reader.read_exact(&mut meta_size).await?;
let meta_len = {
if meta_size == CONTINUATION_MARKER {
reader.read_exact(&mut meta_size).await?;
}
i32::from_le_bytes(meta_size)
};
let meta_len: usize = meta_len
.try_into()
.map_err(|_| Error::from(OutOfSpecKind::NegativeFooterLength))?;
let mut meta_buffer = vec![];
meta_buffer.try_reserve(meta_len as usize)?;
reader
.take(meta_len as u64)
.read_to_end(&mut meta_buffer)
.await?;
deserialize_stream_metadata(&meta_buffer)
}
async fn maybe_next<R: AsyncRead + Unpin + Send>(
mut state: ReadState<R>,
) -> Result<Option<StreamState<R>>> {
let mut scratch = Default::default();
let mut meta_length: [u8; 4] = [0; 4];
match state.reader.read_exact(&mut meta_length).await {
Ok(()) => (),
Err(e) => {
return if e.kind() == std::io::ErrorKind::UnexpectedEof {
Ok(Some(StreamState::Waiting(state)))
} else {
Err(Error::from(e))
};
}
}
let meta_length = {
if meta_length == CONTINUATION_MARKER {
state.reader.read_exact(&mut meta_length).await?;
}
i32::from_le_bytes(meta_length)
};
let meta_length: usize = meta_length
.try_into()
.map_err(|_| Error::from(OutOfSpecKind::NegativeFooterLength))?;
if meta_length == 0 {
return Ok(None);
}
state.message_buffer.clear();
state.message_buffer.try_reserve(meta_length as usize)?;
(&mut state.reader)
.take(meta_length as u64)
.read_to_end(&mut state.message_buffer)
.await?;
let message = arrow_format::ipc::MessageRef::read_as_root(state.message_buffer.as_ref())
.map_err(|err| Error::from(OutOfSpecKind::InvalidFlatbufferMessage(err)))?;
let header = message
.header()
.map_err(|err| Error::from(OutOfSpecKind::InvalidFlatbufferHeader(err)))?
.ok_or_else(|| Error::from(OutOfSpecKind::MissingMessageHeader))?;
let block_length: usize = message
.body_length()
.map_err(|err| Error::from(OutOfSpecKind::InvalidFlatbufferBodyLength(err)))?
.try_into()
.map_err(|_| Error::from(OutOfSpecKind::UnexpectedNegativeInteger))?;
match header {
arrow_format::ipc::MessageHeaderRef::RecordBatch(batch) => {
state.data_buffer.clear();
state.data_buffer.try_reserve(block_length as usize)?;
(&mut state.reader)
.take(block_length as u64)
.read_to_end(&mut state.data_buffer)
.await?;
read_record_batch(
batch,
&state.metadata.schema.fields,
&state.metadata.ipc_schema,
None,
None,
&state.dictionaries,
state.metadata.version,
&mut std::io::Cursor::new(&state.data_buffer),
0,
state.data_buffer.len() as u64,
&mut scratch,
)
.map(|chunk| Some(StreamState::Some((state, chunk))))
}
arrow_format::ipc::MessageHeaderRef::DictionaryBatch(batch) => {
state.data_buffer.clear();
state.data_buffer.try_reserve(block_length as usize)?;
(&mut state.reader)
.take(block_length as u64)
.read_to_end(&mut state.data_buffer)
.await?;
let file_size = state.data_buffer.len() as u64;
let mut dict_reader = std::io::Cursor::new(&state.data_buffer);
read_dictionary(
batch,
&state.metadata.schema.fields,
&state.metadata.ipc_schema,
&mut state.dictionaries,
&mut dict_reader,
0,
file_size,
&mut scratch,
)?;
Ok(Some(StreamState::Waiting(state)))
}
_ => Err(Error::from(OutOfSpecKind::UnexpectedMessageType)),
}
}
pub struct AsyncStreamReader<'a, R: AsyncRead + Unpin + Send + 'a> {
metadata: StreamMetadata,
future: Option<BoxFuture<'a, Result<Option<StreamState<R>>>>>,
}
impl<'a, R: AsyncRead + Unpin + Send + 'a> AsyncStreamReader<'a, R> {
pub fn new(reader: R, metadata: StreamMetadata) -> Self {
let state = ReadState {
reader,
metadata: metadata.clone(),
dictionaries: Default::default(),
data_buffer: Default::default(),
message_buffer: Default::default(),
};
let future = Some(maybe_next(state).boxed());
Self { metadata, future }
}
pub fn metadata(&self) -> &StreamMetadata {
&self.metadata
}
}
impl<'a, R: AsyncRead + Unpin + Send> Stream for AsyncStreamReader<'a, R> {
type Item = Result<Chunk<Box<dyn Array>>>;
fn poll_next(
self: std::pin::Pin<&mut Self>,
cx: &mut std::task::Context<'_>,
) -> std::task::Poll<Option<Self::Item>> {
use std::pin::Pin;
use std::task::Poll;
let me = Pin::into_inner(self);
match &mut me.future {
Some(fut) => match fut.as_mut().poll(cx) {
Poll::Ready(Ok(None)) => {
me.future = None;
Poll::Ready(None)
}
Poll::Ready(Ok(Some(StreamState::Some((state, batch))))) => {
me.future = Some(Box::pin(maybe_next(state)));
Poll::Ready(Some(Ok(batch)))
}
Poll::Ready(Ok(Some(StreamState::Waiting(_)))) => Poll::Pending,
Poll::Ready(Err(err)) => {
me.future = None;
Poll::Ready(Some(Err(err)))
}
Poll::Pending => Poll::Pending,
},
None => Poll::Ready(None),
}
}
}