arrow_ipc/reader/stream.rs
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 244 245 246 247 248 249 250 251 252 253 254 255 256 257 258 259 260 261 262 263 264 265 266 267 268 269 270 271 272 273 274 275 276 277 278 279 280 281 282 283 284 285 286 287 288 289 290 291 292 293 294 295 296 297 298 299 300 301 302 303 304 305 306 307 308 309 310 311 312 313 314 315 316 317 318 319 320 321 322 323 324 325 326 327 328 329 330 331 332 333 334 335 336 337 338 339 340 341 342 343 344 345 346 347 348 349 350 351 352 353 354 355 356 357 358 359 360 361 362 363 364 365 366 367 368 369 370 371 372 373 374 375 376 377
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
use std::collections::HashMap;
use std::fmt::Debug;
use std::sync::Arc;
use arrow_array::{ArrayRef, RecordBatch};
use arrow_buffer::{Buffer, MutableBuffer};
use arrow_schema::{ArrowError, SchemaRef};
use crate::convert::MessageBuffer;
use crate::reader::{read_dictionary_impl, read_record_batch_impl};
use crate::{MessageHeader, CONTINUATION_MARKER};
/// A low-level interface for reading [`RecordBatch`] data from a stream of bytes
///
/// See [StreamReader](crate::reader::StreamReader) for a higher-level interface
#[derive(Debug, Default)]
pub struct StreamDecoder {
/// The schema of this decoder, if read
schema: Option<SchemaRef>,
/// Lookup table for dictionaries by ID
dictionaries: HashMap<i64, ArrayRef>,
/// The decoder state
state: DecoderState,
/// A scratch buffer when a read is split across multiple `Buffer`
buf: MutableBuffer,
/// Whether or not array data in input buffers are required to be aligned
require_alignment: bool,
}
#[derive(Debug)]
enum DecoderState {
/// Decoding the message header
Header {
/// Temporary buffer
buf: [u8; 4],
/// Number of bytes read into buf
read: u8,
/// If we have read a continuation token
continuation: bool,
},
/// Decoding the message flatbuffer
Message {
/// The size of the message flatbuffer
size: u32,
},
/// Decoding the message body
Body {
/// The message flatbuffer
message: MessageBuffer,
},
/// Reached the end of the stream
Finished,
}
impl Default for DecoderState {
fn default() -> Self {
Self::Header {
buf: [0; 4],
read: 0,
continuation: false,
}
}
}
impl StreamDecoder {
/// Create a new [`StreamDecoder`]
pub fn new() -> Self {
Self::default()
}
/// Specifies whether or not array data in input buffers is required to be properly aligned.
///
/// If `require_alignment` is true, this decoder will return an error if any array data in the
/// input `buf` is not properly aligned.
/// Under the hood it will use [`arrow_data::ArrayDataBuilder::build`] to construct
/// [`arrow_data::ArrayData`].
///
/// If `require_alignment` is false (the default), this decoder will automatically allocate a
/// new aligned buffer and copy over the data if any array data in the input `buf` is not
/// properly aligned. (Properly aligned array data will remain zero-copy.)
/// Under the hood it will use [`arrow_data::ArrayDataBuilder::build_aligned`] to construct
/// [`arrow_data::ArrayData`].
pub fn with_require_alignment(mut self, require_alignment: bool) -> Self {
self.require_alignment = require_alignment;
self
}
/// Try to read the next [`RecordBatch`] from the provided [`Buffer`]
///
/// [`Buffer::advance`] will be called on `buffer` for any consumed bytes.
///
/// The push-based interface facilitates integration with sources that yield arbitrarily
/// delimited bytes ranges, such as a chunked byte stream received from object storage
///
/// ```
/// # use arrow_array::RecordBatch;
/// # use arrow_buffer::Buffer;
/// # use arrow_ipc::reader::StreamDecoder;
/// # use arrow_schema::ArrowError;
/// #
/// fn print_stream<I>(src: impl Iterator<Item = Buffer>) -> Result<(), ArrowError> {
/// let mut decoder = StreamDecoder::new();
/// for mut x in src {
/// while !x.is_empty() {
/// if let Some(x) = decoder.decode(&mut x)? {
/// println!("{x:?}");
/// }
/// }
/// }
/// decoder.finish().unwrap();
/// Ok(())
/// }
/// ```
pub fn decode(&mut self, buffer: &mut Buffer) -> Result<Option<RecordBatch>, ArrowError> {
while !buffer.is_empty() {
match &mut self.state {
DecoderState::Header {
buf,
read,
continuation,
} => {
let offset_buf = &mut buf[*read as usize..];
let to_read = buffer.len().min(offset_buf.len());
offset_buf[..to_read].copy_from_slice(&buffer[..to_read]);
*read += to_read as u8;
buffer.advance(to_read);
if *read == 4 {
if !*continuation && buf == &CONTINUATION_MARKER {
*continuation = true;
*read = 0;
continue;
}
let size = u32::from_le_bytes(*buf);
if size == 0 {
self.state = DecoderState::Finished;
continue;
}
self.state = DecoderState::Message { size };
}
}
DecoderState::Message { size } => {
let len = *size as usize;
if self.buf.is_empty() && buffer.len() > len {
let message = MessageBuffer::try_new(buffer.slice_with_length(0, len))?;
self.state = DecoderState::Body { message };
buffer.advance(len);
continue;
}
let to_read = buffer.len().min(len - self.buf.len());
self.buf.extend_from_slice(&buffer[..to_read]);
buffer.advance(to_read);
if self.buf.len() == len {
let message = MessageBuffer::try_new(std::mem::take(&mut self.buf).into())?;
self.state = DecoderState::Body { message };
}
}
DecoderState::Body { message } => {
let message = message.as_ref();
let body_length = message.bodyLength() as usize;
let body = if self.buf.is_empty() && buffer.len() >= body_length {
let body = buffer.slice_with_length(0, body_length);
buffer.advance(body_length);
body
} else {
let to_read = buffer.len().min(body_length - self.buf.len());
self.buf.extend_from_slice(&buffer[..to_read]);
buffer.advance(to_read);
if self.buf.len() != body_length {
continue;
}
std::mem::take(&mut self.buf).into()
};
let version = message.version();
match message.header_type() {
MessageHeader::Schema => {
if self.schema.is_some() {
return Err(ArrowError::IpcError(
"Not expecting a schema when messages are read".to_string(),
));
}
let ipc_schema = message.header_as_schema().unwrap();
let schema = crate::convert::fb_to_schema(ipc_schema);
self.state = DecoderState::default();
self.schema = Some(Arc::new(schema));
}
MessageHeader::RecordBatch => {
let batch = message.header_as_record_batch().unwrap();
let schema = self.schema.clone().ok_or_else(|| {
ArrowError::IpcError("Missing schema".to_string())
})?;
let batch = read_record_batch_impl(
&body,
batch,
schema,
&self.dictionaries,
None,
&version,
self.require_alignment,
)?;
self.state = DecoderState::default();
return Ok(Some(batch));
}
MessageHeader::DictionaryBatch => {
let dictionary = message.header_as_dictionary_batch().unwrap();
let schema = self.schema.as_deref().ok_or_else(|| {
ArrowError::IpcError("Missing schema".to_string())
})?;
read_dictionary_impl(
&body,
dictionary,
schema,
&mut self.dictionaries,
&version,
self.require_alignment,
)?;
self.state = DecoderState::default();
}
MessageHeader::NONE => {
self.state = DecoderState::default();
}
t => {
return Err(ArrowError::IpcError(format!(
"Message type unsupported by StreamDecoder: {t:?}"
)))
}
}
}
DecoderState::Finished => {
return Err(ArrowError::IpcError("Unexpected EOS".to_string()))
}
}
}
Ok(None)
}
/// Signal the end of stream
///
/// Returns an error if any partial data remains in the stream
pub fn finish(&mut self) -> Result<(), ArrowError> {
match self.state {
DecoderState::Finished
| DecoderState::Header {
read: 0,
continuation: false,
..
} => Ok(()),
_ => Err(ArrowError::IpcError("Unexpected End of Stream".to_string())),
}
}
}
#[cfg(test)]
mod tests {
use super::*;
use crate::writer::{IpcWriteOptions, StreamWriter};
use arrow_array::{
types::Int32Type, DictionaryArray, Int32Array, Int64Array, RecordBatch, RunArray,
};
use arrow_schema::{DataType, Field, Schema};
// Further tests in arrow-integration-testing/tests/ipc_reader.rs
#[test]
fn test_eos() {
let schema = Arc::new(Schema::new(vec![
Field::new("int32", DataType::Int32, false),
Field::new("int64", DataType::Int64, false),
]));
let input = RecordBatch::try_new(
schema.clone(),
vec![
Arc::new(Int32Array::from(vec![1, 2, 3])) as _,
Arc::new(Int64Array::from(vec![1, 2, 3])) as _,
],
)
.unwrap();
let mut buf = Vec::with_capacity(1024);
let mut s = StreamWriter::try_new(&mut buf, &schema).unwrap();
s.write(&input).unwrap();
s.finish().unwrap();
drop(s);
let buffer = Buffer::from_vec(buf);
let mut b = buffer.slice_with_length(0, buffer.len() - 1);
let mut decoder = StreamDecoder::new();
let output = decoder.decode(&mut b).unwrap().unwrap();
assert_eq!(output, input);
assert_eq!(b.len(), 7); // 8 byte EOS truncated by 1 byte
assert!(decoder.decode(&mut b).unwrap().is_none());
let err = decoder.finish().unwrap_err().to_string();
assert_eq!(err, "Ipc error: Unexpected End of Stream");
}
#[test]
fn test_read_ree_dict_record_batches_from_buffer() {
let schema = Schema::new(vec![Field::new(
"test1",
DataType::RunEndEncoded(
Arc::new(Field::new("run_ends".to_string(), DataType::Int32, false)),
Arc::new(Field::new_dict(
"values".to_string(),
DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8)),
true,
0,
false,
)),
),
true,
)]);
let batch = RecordBatch::try_new(
schema.clone().into(),
vec![Arc::new(
RunArray::try_new(
&Int32Array::from(vec![1, 2, 3]),
&vec![Some("a"), None, Some("a")]
.into_iter()
.collect::<DictionaryArray<Int32Type>>(),
)
.expect("Failed to create RunArray"),
)],
)
.expect("Failed to create RecordBatch");
let mut buffer = vec![];
{
let mut writer = StreamWriter::try_new_with_options(
&mut buffer,
&schema,
IpcWriteOptions::default().with_preserve_dict_id(false),
)
.expect("Failed to create StreamWriter");
writer.write(&batch).expect("Failed to write RecordBatch");
writer.finish().expect("Failed to finish StreamWriter");
}
let mut decoder = StreamDecoder::new();
let buf = &mut Buffer::from(buffer.as_slice());
while let Some(batch) = decoder
.decode(buf)
.map_err(|e| {
ArrowError::ExternalError(format!("Failed to decode record batch: {}", e).into())
})
.expect("Failed to decode record batch")
{
assert_eq!(batch, batch);
}
decoder.finish().expect("Failed to finish decoder");
}
}