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
// 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::io::Read;
use bytes::BytesMut;
use futures::AsyncReadExt;
use hdrs::AsyncFile;
use hdrs::File;
use tokio::io::ReadBuf;
use crate::raw::*;
use crate::*;
pub struct HdfsReader<F> {
f: F,
read: usize,
size: usize,
buf_size: usize,
buf: BytesMut,
}
impl<F> HdfsReader<F> {
pub fn new(f: F, size: usize) -> Self {
Self {
f,
read: 0,
size,
// Use 2 MiB as default value.
buf_size: 2 * 1024 * 1024,
buf: BytesMut::new(),
}
}
}
impl oio::Read for HdfsReader<AsyncFile> {
async fn read(&mut self) -> Result<Buffer> {
if self.read >= self.size {
return Ok(Buffer::new());
}
let size = (self.size - self.read).min(self.buf_size);
self.buf.reserve(size);
let buf = &mut self.buf.spare_capacity_mut()[..size];
let mut read_buf: ReadBuf = ReadBuf::uninit(buf);
// SAFETY: Read at most `limit` bytes into `read_buf`.
unsafe {
read_buf.assume_init(size);
}
let n = self
.f
.read(read_buf.initialize_unfilled())
.await
.map_err(new_std_io_error)?;
read_buf.advance(n);
self.read += n;
// Safety: We make sure that bs contains `n` more bytes.
let filled = read_buf.filled().len();
unsafe { self.buf.set_len(filled) }
let frozen = self.buf.split().freeze();
Ok(Buffer::from(frozen))
}
}
impl oio::BlockingRead for HdfsReader<File> {
fn read(&mut self) -> Result<Buffer> {
if self.read >= self.size {
return Ok(Buffer::new());
}
let size = (self.size - self.read).min(self.buf_size);
self.buf.reserve(size);
let buf = &mut self.buf.spare_capacity_mut()[..size];
let mut read_buf: ReadBuf = ReadBuf::uninit(buf);
// SAFETY: Read at most `limit` bytes into `read_buf`.
unsafe {
read_buf.assume_init(size);
}
let n = self
.f
.read(read_buf.initialize_unfilled())
.map_err(new_std_io_error)?;
read_buf.advance(n);
self.read += n;
// Safety: We make sure that bs contains `n` more bytes.
let filled = read_buf.filled().len();
unsafe { self.buf.set_len(filled) }
let frozen = self.buf.split().freeze();
Ok(Buffer::from(frozen))
}
}