From 8e2886402152d1d8e9e61df01260e68098c4db3b Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Mon, 18 Mar 2024 14:32:42 +0800 Subject: [PATCH 001/111] Save work Signed-off-by: Xuanwo --- core/src/raw/http_util/body.rs | 68 ++++++++++++++++---------------- core/src/raw/http_util/client.rs | 36 ++++++++++++----- core/src/raw/oio/read/api.rs | 55 +++++--------------------- 3 files changed, 70 insertions(+), 89 deletions(-) diff --git a/core/src/raw/http_util/body.rs b/core/src/raw/http_util/body.rs index e32dcb46d449..d6df52c5e55b 100644 --- a/core/src/raw/http_util/body.rs +++ b/core/src/raw/http_util/body.rs @@ -159,37 +159,37 @@ impl IncomingAsyncBody { } } -impl oio::Read for IncomingAsyncBody { - async fn read(&mut self, limit: usize) -> Result { - if self.size == Some(0) { - return Ok(Bytes::new()); - } - - if self.chunk.is_empty() { - self.chunk = match self.inner.next().await.transpose()? { - Some(bs) => bs, - None => { - if let Some(size) = self.size { - Self::check(size, self.consumed)? - } - - return Ok(Bytes::new()); - } - }; - } - - let size = min(limit, self.chunk.len()); - self.consumed += size as u64; - let bs = self.chunk.split_to(size); - Ok(bs) - } - - async fn seek(&mut self, pos: io::SeekFrom) -> Result { - let _ = pos; - - Err(Error::new( - ErrorKind::Unsupported, - "output reader doesn't support seeking", - )) - } -} +// impl oio::Read for IncomingAsyncBody { +// async fn read(&mut self, limit: usize) -> Result { +// if self.size == Some(0) { +// return Ok(Bytes::new()); +// } +// +// if self.chunk.is_empty() { +// self.chunk = match self.inner.next().await.transpose()? { +// Some(bs) => bs, +// None => { +// if let Some(size) = self.size { +// Self::check(size, self.consumed)? +// } +// +// return Ok(Bytes::new()); +// } +// }; +// } +// +// let size = min(limit, self.chunk.len()); +// self.consumed += size as u64; +// let bs = self.chunk.split_to(size); +// Ok(bs) +// } +// +// async fn seek(&mut self, pos: io::SeekFrom) -> Result { +// let _ = pos; +// +// Err(Error::new( +// ErrorKind::Unsupported, +// "output reader doesn't support seeking", +// )) +// } +// } diff --git a/core/src/raw/http_util/client.rs b/core/src/raw/http_util/client.rs index 3f2f50d67571..8a51e79e927a 100644 --- a/core/src/raw/http_util/client.rs +++ b/core/src/raw/http_util/client.rs @@ -15,6 +15,8 @@ // specific language governing permissions and limitations // under the License. +use bytes::Bytes; +use std::cmp::Ordering; use std::fmt::Debug; use std::fmt::Formatter; use std::mem; @@ -78,7 +80,7 @@ impl HttpClient { } /// Send a request in async way. - pub async fn send(&self, req: Request) -> Result> { + pub async fn send(&self, req: Request) -> Result> { // Uri stores all string alike data in `Bytes` which means // the clone here is cheap. let uri = req.uri().clone(); @@ -166,19 +168,35 @@ impl HttpClient { // Swap headers directly instead of copy the entire map. mem::swap(hr.headers_mut().unwrap(), resp.headers_mut()); - let stream = resp.bytes_stream().map_err(move |err| { - // If stream returns a body related error, we can convert - // it to interrupt so we can retry it. - Error::new(ErrorKind::Unexpected, "read data from http stream") - .map(|v| if err.is_body() { v.set_temporary() } else { v }) + let bytes = resp.bytes().await.map_err(|err| { + Error::new(ErrorKind::Unexpected, "read data from http response") .with_context("url", uri.to_string()) .set_source(err) - }); + })?; - let body = IncomingAsyncBody::new(Box::new(oio::into_stream(stream)), content_length); + if let Some(expect) = content_length { + check(expect, bytes.len() as u64)?; + } - let resp = hr.body(body).expect("response must build succeed"); + let resp = hr.body(bytes).expect("response must build succeed"); Ok(resp) } } + +#[inline] +fn check(expect: u64, actual: u64) -> Result<()> { + match actual.cmp(&expect) { + Ordering::Equal => Ok(()), + Ordering::Less => Err(Error::new( + ErrorKind::ContentIncomplete, + &format!("reader got too little data, expect: {expect}, actual: {actual}"), + ) + .set_temporary()), + Ordering::Greater => Err(Error::new( + ErrorKind::ContentTruncated, + &format!("reader got too much data, expect: {expect}, actual: {actual}"), + ) + .set_temporary()), + } +} diff --git a/core/src/raw/oio/read/api.rs b/core/src/raw/oio/read/api.rs index d8ed05c4ece0..4ca6d0457f15 100644 --- a/core/src/raw/oio/read/api.rs +++ b/core/src/raw/oio/read/api.rs @@ -87,63 +87,30 @@ pub type Reader = Box; /// an additional layer of indirection and an extra allocation. Ideally, `ReadDyn` should occur only /// once, at the outermost level of our API. pub trait Read: Unpin + Send + Sync { - /// Fetch more bytes from underlying reader. - /// - /// `limit` is used to hint the data that user want to read at most. Implementer - /// MUST NOT return more than `limit` bytes. However, implementer can decide - /// whether to split or merge the read requests underground. - /// - /// Returning `bytes`'s `length == 0` means: - /// - /// - This reader has reached its “end of file” and will likely no longer be able to produce bytes. - /// - The `limit` specified was `0`. #[cfg(not(target_arch = "wasm32"))] - fn read(&mut self, limit: usize) -> impl Future> + Send; + fn read_at(&self, offset: u64, limit: usize) -> impl Future> + Send; #[cfg(target_arch = "wasm32")] - fn read(&mut self, size: usize) -> impl Future>; - - /// Seek asynchronously. - /// - /// Returns `Unsupported` error if underlying reader doesn't support seek. - #[cfg(not(target_arch = "wasm32"))] - fn seek(&mut self, pos: io::SeekFrom) -> impl Future> + Send; - #[cfg(target_arch = "wasm32")] - fn seek(&mut self, pos: io::SeekFrom) -> impl Future>; + fn read_at(&self, offset: u64, limit: usize) -> impl Future>; } impl Read for () { - async fn read(&mut self, limit: usize) -> Result { - let _ = limit; + async fn read_at(&mut self, offset: u64, limit: usize) -> Result { + let (_, _) = (offset, limit); Err(Error::new( ErrorKind::Unsupported, "output reader doesn't support streaming", )) } - - async fn seek(&mut self, pos: io::SeekFrom) -> Result { - let _ = pos; - - Err(Error::new( - ErrorKind::Unsupported, - "output reader doesn't support seeking", - )) - } } pub trait ReadDyn: Unpin + Send + Sync { - fn read_dyn(&mut self, limit: usize) -> BoxedFuture>; - - fn seek_dyn(&mut self, pos: io::SeekFrom) -> BoxedFuture>; + fn read_at_dyn(&self, offset: u64, limit: usize) -> BoxedFuture>; } impl ReadDyn for T { - fn read_dyn(&mut self, limit: usize) -> BoxedFuture> { - Box::pin(self.read(limit)) - } - - fn seek_dyn(&mut self, pos: io::SeekFrom) -> BoxedFuture> { - Box::pin(self.seek(pos)) + fn read_at_dyn(&self, offset: u64, limit: usize) -> BoxedFuture> { + Box::pin(self.read_at(offset, limit)) } } @@ -152,12 +119,8 @@ impl ReadDyn for T { /// Take care about the `deref_mut()` here. This makes sure that we are calling functions /// upon `&mut T` instead of `&mut Box`. The later could result in infinite recursion. impl Read for Box { - async fn read(&mut self, limit: usize) -> Result { - self.deref_mut().read_dyn(limit).await - } - - async fn seek(&mut self, pos: io::SeekFrom) -> Result { - self.deref_mut().seek_dyn(pos).await + async fn read_at(&mut self, offset: u64, limit: usize) -> Result { + self.deref_mut().read_at_dyn(offset, limit).await } } From ad9d955dc046e554b6db77d9f9c58badb57dcfe1 Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Mon, 18 Mar 2024 15:12:26 +0800 Subject: [PATCH 002/111] Save Signed-off-by: Xuanwo --- core/src/raw/oio/buf/buffer.rs | 90 ++++++++++++++++++++++++++++++++++ core/src/raw/oio/buf/mod.rs | 2 + 2 files changed, 92 insertions(+) create mode 100644 core/src/raw/oio/buf/buffer.rs diff --git a/core/src/raw/oio/buf/buffer.rs b/core/src/raw/oio/buf/buffer.rs new file mode 100644 index 000000000000..0b7c71bf21a4 --- /dev/null +++ b/core/src/raw/oio/buf/buffer.rs @@ -0,0 +1,90 @@ +// 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 bytes::{BufMut, Bytes, BytesMut}; +use std::collections::VecDeque; + +pub struct Buffer(Inner); + +enum Inner { + Contiguous(Bytes), + NonContiguous(VecDeque), +} + +impl bytes::Buf for Buffer { + #[inline] + fn remaining(&self) -> usize { + match &self.0 { + Inner::Contiguous(b) => b.remaining(), + Inner::NonContiguous(v) => v.iter().map(|b| b.remaining()).sum(), + } + } + + #[inline] + fn chunk(&self) -> &[u8] { + match &self.0 { + Inner::Contiguous(b) => b.chunk(), + Inner::NonContiguous(v) => { + if let Some(b) = v.front() { + b.chunk() + } else { + &[] + } + } + } + } + + #[inline] + fn advance(&mut self, cnt: usize) { + match &mut self.0 { + Inner::Contiguous(b) => b.advance(cnt), + Inner::NonContiguous(v) => { + let mut cnt = cnt; + while cnt > 0 { + let b = &mut v[0]; + if b.remaining() > cnt { + b.advance(cnt); + break; + } else { + cnt -= b.remaining(); + v.remove(0); + } + } + } + } + } + + #[inline] + fn copy_to_bytes(&mut self, len: usize) -> Bytes { + match &mut self.0 { + Inner::Contiguous(b) => b.copy_to_bytes(len), + Inner::NonContiguous(v) => { + if len <= v[0].remaining() { + let bs = v[0].copy_to_bytes(len); + if v[0].is_empty() { + v.remove(0); + } + return bs; + } + + let mut bs = BytesMut::with_capacity(len); + bs.put(self.take(len)); + bs.freeze() + } + } + } +} diff --git a/core/src/raw/oio/buf/mod.rs b/core/src/raw/oio/buf/mod.rs index 9c4881ec83ac..1a22305895da 100644 --- a/core/src/raw/oio/buf/mod.rs +++ b/core/src/raw/oio/buf/mod.rs @@ -16,4 +16,6 @@ // under the License. mod adaptive; +mod buffer; + pub use adaptive::AdaptiveBuf; From e2a1a5f4533f913ba28edd9e2c642ba29007fd12 Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Mon, 18 Mar 2024 16:40:16 +0800 Subject: [PATCH 003/111] Build pass Signed-off-by: Xuanwo --- core/src/layers/complete.rs | 203 ++++---- core/src/layers/concurrent_limit.rs | 10 +- core/src/layers/error_context.rs | 14 +- core/src/layers/logging.rs | 65 +-- core/src/layers/retry.rs | 60 +-- core/src/layers/timeout.rs | 9 +- core/src/raw/adapters/kv/backend.rs | 4 +- core/src/raw/adapters/typed_kv/backend.rs | 4 +- core/src/raw/enum_utils.rs | 50 +- core/src/raw/http_util/body.rs | 163 ------- core/src/raw/http_util/client.rs | 12 +- core/src/raw/http_util/mod.rs | 1 - core/src/raw/http_util/multipart.rs | 20 +- core/src/raw/oio/buf/buffer.rs | 27 +- core/src/raw/oio/buf/mod.rs | 5 +- core/src/raw/oio/cursor.rs | 66 +-- core/src/raw/oio/read/api.rs | 37 +- core/src/raw/oio/read/mod.rs | 36 +- core/src/services/fs/backend.rs | 5 +- core/src/services/fs/mod.rs | 1 + core/src/services/fs/reader.rs | 37 ++ core/src/types/operator/operator.rs | 8 +- core/src/types/reader.rs | 544 ++++++++++------------ 23 files changed, 577 insertions(+), 804 deletions(-) create mode 100644 core/src/services/fs/reader.rs diff --git a/core/src/layers/complete.rs b/core/src/layers/complete.rs index d4985a0f5235..6bc8fdfeb963 100644 --- a/core/src/layers/complete.rs +++ b/core/src/layers/complete.rs @@ -24,13 +24,8 @@ use std::sync::Arc; use async_trait::async_trait; use bytes::Bytes; -use crate::raw::oio::BufferReader; -use crate::raw::oio::FileReader; use crate::raw::oio::FlatLister; -use crate::raw::oio::LazyReader; use crate::raw::oio::PrefixLister; -use crate::raw::oio::RangeReader; -use crate::raw::oio::StreamableReader; use crate::raw::TwoWays; use crate::raw::*; use crate::*; @@ -280,91 +275,91 @@ impl CompleteAccessor { }) } - async fn complete_read( - &self, - path: &str, - args: OpRead, - ) -> Result<(RpRead, CompleteReader)> { - let capability = self.meta.native_capability(); - if !capability.read { - return Err(self.new_unsupported_error(Operation::Read)); - } - - let seekable = capability.read_can_seek; - let streamable = capability.read_can_next; - let buffer_cap = args.buffer(); - - let r = match (seekable, streamable) { - (true, true) => { - let r = LazyReader::new(self.inner.clone(), path, args); - InnerCompleteReader::One(r) - } - (true, false) => { - let r = FileReader::new(self.inner.clone(), path, args); - InnerCompleteReader::Two(r) - } - _ => { - let r = RangeReader::new(self.inner.clone(), path, args); - - if streamable { - InnerCompleteReader::Three(r) - } else { - let r = oio::into_streamable_read(r, 256 * 1024); - InnerCompleteReader::Four(r) - } - } - }; - - let r = match buffer_cap { - None => CompleteReader::One(r), - Some(cap) => CompleteReader::Two(BufferReader::new(r, cap)), - }; - - Ok((RpRead::new(), r)) - } - - fn complete_blocking_read( - &self, - path: &str, - args: OpRead, - ) -> Result<(RpRead, CompleteReader)> { - let capability = self.meta.full_capability(); - if !capability.read || !capability.blocking { - return Err(self.new_unsupported_error(Operation::BlockingRead)); - } - - let seekable = capability.read_can_seek; - let streamable = capability.read_can_next; - let buffer_cap = args.buffer(); - - let r = match (seekable, streamable) { - (true, true) => { - let r = LazyReader::new(self.inner.clone(), path, args); - InnerCompleteReader::One(r) - } - (true, false) => { - let r = FileReader::new(self.inner.clone(), path, args); - InnerCompleteReader::Two(r) - } - _ => { - let r = RangeReader::new(self.inner.clone(), path, args); - - if streamable { - InnerCompleteReader::Three(r) - } else { - let r = oio::into_streamable_read(r, 256 * 1024); - InnerCompleteReader::Four(r) - } - } - }; - - let r = match buffer_cap { - None => CompleteReader::One(r), - Some(cap) => CompleteReader::Two(BufferReader::new(r, cap)), - }; - - Ok((RpRead::new(), r)) - } + // async fn complete_read( + // &self, + // path: &str, + // args: OpRead, + // ) -> Result<(RpRead, CompleteReader)> { + // let capability = self.meta.native_capability(); + // if !capability.read { + // return Err(self.new_unsupported_error(Operation::Read)); + // } + // + // let seekable = capability.read_can_seek; + // let streamable = capability.read_can_next; + // let buffer_cap = args.buffer(); + // + // let r = match (seekable, streamable) { + // (true, true) => { + // let r = LazyReader::new(self.inner.clone(), path, args); + // InnerCompleteReader::One(r) + // } + // (true, false) => { + // let r = FileReader::new(self.inner.clone(), path, args); + // InnerCompleteReader::Two(r) + // } + // _ => { + // let r = RangeReader::new(self.inner.clone(), path, args); + // + // if streamable { + // InnerCompleteReader::Three(r) + // } else { + // let r = oio::into_streamable_read(r, 256 * 1024); + // InnerCompleteReader::Four(r) + // } + // } + // }; + // + // let r = match buffer_cap { + // None => CompleteReader::One(r), + // Some(cap) => CompleteReader::Two(BufferReader::new(r, cap)), + // }; + // + // Ok((RpRead::new(), r)) + // } + + // fn complete_blocking_read( + // &self, + // path: &str, + // args: OpRead, + // ) -> Result<(RpRead, CompleteReader)> { + // let capability = self.meta.full_capability(); + // if !capability.read || !capability.blocking { + // return Err(self.new_unsupported_error(Operation::BlockingRead)); + // } + // + // let seekable = capability.read_can_seek; + // let streamable = capability.read_can_next; + // let buffer_cap = args.buffer(); + // + // let r = match (seekable, streamable) { + // (true, true) => { + // let r = LazyReader::new(self.inner.clone(), path, args); + // InnerCompleteReader::One(r) + // } + // (true, false) => { + // let r = FileReader::new(self.inner.clone(), path, args); + // InnerCompleteReader::Two(r) + // } + // _ => { + // let r = RangeReader::new(self.inner.clone(), path, args); + // + // if streamable { + // InnerCompleteReader::Three(r) + // } else { + // let r = oio::into_streamable_read(r, 256 * 1024); + // InnerCompleteReader::Four(r) + // } + // } + // }; + // + // let r = match buffer_cap { + // None => CompleteReader::One(r), + // Some(cap) => CompleteReader::Two(BufferReader::new(r, cap)), + // }; + // + // Ok((RpRead::new(), r)) + // } async fn complete_list( &self, @@ -467,8 +462,8 @@ impl CompleteAccessor { #[cfg_attr(target_arch = "wasm32", async_trait(?Send))] impl LayeredAccessor for CompleteAccessor { type Inner = A; - type Reader = CompleteReader; - type BlockingReader = CompleteReader; + type Reader = A::Reader; + type BlockingReader = A::BlockingReader; type Writer = TwoWays, oio::ExactBufWriter>>; type BlockingWriter = CompleteWriter; @@ -497,7 +492,8 @@ impl LayeredAccessor for CompleteAccessor { } async fn read(&self, path: &str, args: OpRead) -> Result<(RpRead, Self::Reader)> { - self.complete_read(path, args).await + // self.complete_read(path, args).await + self.inner.read(path, args).await } async fn write(&self, path: &str, args: OpWrite) -> Result<(RpWrite, Self::Writer)> { @@ -606,7 +602,8 @@ impl LayeredAccessor for CompleteAccessor { } fn blocking_read(&self, path: &str, args: OpRead) -> Result<(RpRead, Self::BlockingReader)> { - self.complete_blocking_read(path, args) + // self.complete_blocking_read(path, args) + self.inner.blocking_read(path, args) } fn blocking_write(&self, path: &str, args: OpWrite) -> Result<(RpWrite, Self::BlockingWriter)> { @@ -671,15 +668,15 @@ impl LayeredAccessor for CompleteAccessor { } } -pub type CompleteReader = - TwoWays, BufferReader>>; - -type InnerCompleteReader = FourWays< - LazyReader, - FileReader, - RangeReader, - StreamableReader>, ->; +// pub type CompleteReader = +// TwoWays, BufferReader>>; +// +// type InnerCompleteReader = FourWays< +// LazyReader, +// FileReader, +// RangeReader, +// StreamableReader>, +// >; pub type CompleteLister = FourWays, P>, PrefixLister

, PrefixLister, P>>>; diff --git a/core/src/layers/concurrent_limit.rs b/core/src/layers/concurrent_limit.rs index 992042ef0ae4..63de98d9f900 100644 --- a/core/src/layers/concurrent_limit.rs +++ b/core/src/layers/concurrent_limit.rs @@ -16,6 +16,7 @@ // under the License. use std::fmt::Debug; +use std::future::Future; use std::io::SeekFrom; use std::sync::Arc; @@ -25,6 +26,7 @@ use bytes::Bytes; use tokio::sync::OwnedSemaphorePermit; use tokio::sync::Semaphore; +use crate::raw::oio::Buffer; use crate::raw::*; use crate::*; @@ -256,12 +258,8 @@ impl ConcurrentLimitWrapper { } impl oio::Read for ConcurrentLimitWrapper { - async fn seek(&mut self, pos: SeekFrom) -> Result { - self.inner.seek(pos).await - } - - async fn read(&mut self, limit: usize) -> Result { - self.inner.read(limit).await + async fn read_at(&self, offset: u64, limit: usize) -> Result { + self.inner.read_at(offset, limit).await } } diff --git a/core/src/layers/error_context.rs b/core/src/layers/error_context.rs index de37bc690b28..28eee259ceee 100644 --- a/core/src/layers/error_context.rs +++ b/core/src/layers/error_context.rs @@ -348,19 +348,13 @@ pub struct ErrorContextWrapper { } impl oio::Read for ErrorContextWrapper { - async fn read(&mut self, limit: usize) -> Result { - self.inner.read(limit).await.map_err(|err| { + async fn read_at(&self, offset: u64, limit: usize) -> Result { + self.inner.read_at(offset, limit).await.map_err(|err| { err.with_operation(ReadOperation::Read) .with_context("service", self.scheme) .with_context("path", &self.path) - }) - } - - async fn seek(&mut self, pos: SeekFrom) -> Result { - self.inner.seek(pos).await.map_err(|err| { - err.with_operation(ReadOperation::Seek) - .with_context("service", self.scheme) - .with_context("path", &self.path) + .with_context("offset", offset.to_string()) + .with_context("limit", limit.to_string()) }) } } diff --git a/core/src/layers/logging.rs b/core/src/layers/logging.rs index 2f9265dbae24..778d0f68f544 100644 --- a/core/src/layers/logging.rs +++ b/core/src/layers/logging.rs @@ -20,13 +20,14 @@ use std::fmt::Debug; use std::io; use async_trait::async_trait; -use bytes::Bytes; +use bytes::{Buf, Bytes}; use futures::FutureExt; use futures::TryFutureExt; use log::debug; use log::log; use log::trace; use log::Level; +use std::sync::atomic::{AtomicU64, Ordering}; use crate::raw::oio::ReadOperation; use crate::raw::oio::WriteOperation; @@ -958,7 +959,7 @@ pub struct LoggingReader { path: String, op: Operation, - read: u64, + read: AtomicU64, inner: R, } @@ -969,7 +970,7 @@ impl LoggingReader { op, path: path.to_string(), - read: 0, + read: AtomicU64::new(0), inner: reader, } } @@ -983,24 +984,25 @@ impl Drop for LoggingReader { self.ctx.scheme, self.op, self.path, - self.read + self.read.load(Ordering::Relaxed) ); } } impl oio::Read for LoggingReader { - async fn read(&mut self, limit: usize) -> Result { - match self.inner.read(limit).await { + async fn read_at(&self, offset: u64, limit: usize) -> Result { + match self.inner.read_at(offset, limit).await { Ok(bs) => { - self.read += bs.len() as u64; + self.read + .fetch_add(bs.remaining() as u64, Ordering::Relaxed); trace!( target: LOGGING_TARGET, - "service={} operation={} path={} read={} -> next returns {}B", + "service={} operation={} path={} read={} -> read returns {}B", self.ctx.scheme, ReadOperation::Read, self.path, - self.read, - bs.len() + self.read.load(Ordering::Relaxed), + bs.remaining() ); Ok(bs) } @@ -1013,38 +1015,7 @@ impl oio::Read for LoggingReader { self.ctx.scheme, ReadOperation::Read, self.path, - self.read, - self.ctx.error_print(&err), - ) - } - Err(err) - } - } - } - - async fn seek(&mut self, pos: io::SeekFrom) -> Result { - match self.inner.seek(pos).await { - Ok(n) => { - trace!( - target: LOGGING_TARGET, - "service={} operation={} path={} read={} -> seek to {pos:?}, current offset {n}", - self.ctx.scheme, - ReadOperation::Seek, - self.path, - self.read, - ); - Ok(n) - } - Err(err) => { - if let Some(lvl) = self.ctx.error_level(&err) { - log!( - target: LOGGING_TARGET, - lvl, - "service={} operation={} path={} read={} -> seek to {pos:?} failed: {}", - self.ctx.scheme, - ReadOperation::Seek, - self.path, - self.read, + self.read.load(Ordering::Relaxed), self.ctx.error_print(&err), ) } @@ -1058,14 +1029,14 @@ impl oio::BlockingRead for LoggingReader { fn read(&mut self, limit: usize) -> Result { match self.inner.read(limit) { Ok(bs) => { - self.read += bs.len() as u64; + self.read.fetch_add(bs.len() as u64, Ordering::Relaxed); trace!( target: LOGGING_TARGET, "service={} operation={} path={} read={} -> data read {}B", self.ctx.scheme, ReadOperation::BlockingRead, self.path, - self.read, + self.read.load(Ordering::Relaxed), bs.len() ); Ok(bs) @@ -1079,7 +1050,7 @@ impl oio::BlockingRead for LoggingReader { self.ctx.scheme, ReadOperation::BlockingRead, self.path, - self.read, + self.read.load(Ordering::Relaxed), self.ctx.error_print(&err), ); } @@ -1098,7 +1069,7 @@ impl oio::BlockingRead for LoggingReader { self.ctx.scheme, ReadOperation::BlockingSeek, self.path, - self.read, + self.read.load(Ordering::Relaxed), ); Ok(n) } @@ -1111,7 +1082,7 @@ impl oio::BlockingRead for LoggingReader { self.ctx.scheme, ReadOperation::BlockingSeek, self.path, - self.read, + self.read.load(Ordering::Relaxed), self.ctx.error_print(&err), ); } diff --git a/core/src/layers/retry.rs b/core/src/layers/retry.rs index d8bfbbb26d92..9385842e5ec8 100644 --- a/core/src/layers/retry.rs +++ b/core/src/layers/retry.rs @@ -17,6 +17,7 @@ use std::fmt::Debug; use std::fmt::Formatter; +use std::future::Future; use std::io; @@ -33,9 +34,9 @@ use bytes::Bytes; use futures::FutureExt; use log::warn; -use crate::raw::oio::ListOperation; use crate::raw::oio::ReadOperation; use crate::raw::oio::WriteOperation; +use crate::raw::oio::{Buffer, ListOperation}; use crate::raw::*; use crate::*; @@ -665,20 +666,16 @@ impl RetryWrapper { } impl oio::Read for RetryWrapper { - async fn seek(&mut self, pos: io::SeekFrom) -> Result { - use backon::RetryableWithContext; - - let inner = self.inner.take().expect("inner must be valid"); - - let (inner, res) = { - |mut r: R| async move { - let res = r.seek(pos).await; - - (r, res) + async fn read_at(&self, offset: u64, limit: usize) -> Result { + { + || { + self.inner + .as_ref() + .expect("inner must be valid") + .read_at(offset, limit) } } .retry(&self.builder) - .context(inner) .when(|e| e.is_temporary()) .notify(|err, dur| { self.notify.intercept( @@ -690,43 +687,8 @@ impl oio::Read for RetryWrapper { ], ) }) - .map(|(r, res)| (r, res.map_err(|err| err.set_persistent()))) - .await; - - self.inner = Some(inner); - res - } - - async fn read(&mut self, limit: usize) -> Result { - use backon::RetryableWithContext; - - let inner = self.inner.take().expect("inner must be valid"); - - let (inner, res) = { - |mut r: R| async move { - let res = r.read(limit).await; - - (r, res) - } - } - .retry(&self.builder) - .when(|e| e.is_temporary()) - .context(inner) - .notify(|err, dur| { - self.notify.intercept( - err, - dur, - &[ - ("operation", ReadOperation::Read.into_static()), - ("path", &self.path), - ], - ) - }) - .map(|(r, res)| (r, res.map_err(|err| err.set_persistent()))) - .await; - - self.inner = Some(inner); - res + .await + .map_err(|e| e.set_persistent()) } } diff --git a/core/src/layers/timeout.rs b/core/src/layers/timeout.rs index 1679361564d3..57985440e940 100644 --- a/core/src/layers/timeout.rs +++ b/core/src/layers/timeout.rs @@ -294,15 +294,10 @@ impl TimeoutWrapper { } impl oio::Read for TimeoutWrapper { - async fn read(&mut self, limit: usize) -> Result { - let fut = self.inner.read(limit); + async fn read_at(&self, offset: u64, limit: usize) -> Result { + let fut = self.inner.read_at(offset, limit); Self::io_timeout(self.timeout, ReadOperation::Read.into_static(), fut).await } - - async fn seek(&mut self, pos: SeekFrom) -> Result { - let fut = self.inner.seek(pos); - Self::io_timeout(self.timeout, ReadOperation::Seek.into_static(), fut).await - } } impl oio::Write for TimeoutWrapper { diff --git a/core/src/raw/adapters/kv/backend.rs b/core/src/raw/adapters/kv/backend.rs index 08447eeb1e99..cfec881521f1 100644 --- a/core/src/raw/adapters/kv/backend.rs +++ b/core/src/raw/adapters/kv/backend.rs @@ -63,7 +63,7 @@ where #[cfg_attr(not(target_arch = "wasm32"), async_trait)] #[cfg_attr(target_arch = "wasm32", async_trait(?Send))] impl Accessor for Backend { - type Reader = oio::Cursor; + type Reader = Bytes; type BlockingReader = oio::Cursor; type Writer = KvWriter; type BlockingWriter = KvWriter; @@ -106,7 +106,7 @@ impl Accessor for Backend { let bs = self.apply_range(bs, args.range()); - Ok((RpRead::new(), oio::Cursor::from(bs))) + Ok((RpRead::new(), Bytes::from(bs))) } fn blocking_read(&self, path: &str, args: OpRead) -> Result<(RpRead, Self::BlockingReader)> { diff --git a/core/src/raw/adapters/typed_kv/backend.rs b/core/src/raw/adapters/typed_kv/backend.rs index c33acfecdc84..504a33493688 100644 --- a/core/src/raw/adapters/typed_kv/backend.rs +++ b/core/src/raw/adapters/typed_kv/backend.rs @@ -57,7 +57,7 @@ where #[cfg_attr(not(target_arch = "wasm32"), async_trait)] #[cfg_attr(target_arch = "wasm32", async_trait(?Send))] impl Accessor for Backend { - type Reader = oio::Cursor; + type Reader = Bytes; type BlockingReader = oio::Cursor; type Writer = KvWriter; type BlockingWriter = KvWriter; @@ -113,7 +113,7 @@ impl Accessor for Backend { let bs = self.apply_range(bs, args.range()); - Ok((RpRead::new(), oio::Cursor::from(bs))) + Ok((RpRead::new(), bs)) } fn blocking_read(&self, path: &str, args: OpRead) -> Result<(RpRead, Self::BlockingReader)> { diff --git a/core/src/raw/enum_utils.rs b/core/src/raw/enum_utils.rs index be39f0df5bbf..96fe75a8438e 100644 --- a/core/src/raw/enum_utils.rs +++ b/core/src/raw/enum_utils.rs @@ -38,10 +38,12 @@ //! This module is used to provide some enums for the above code. We should remove this module once //! type_alias_impl_trait has been stabilized. +use std::future::Future; use std::io::SeekFrom; use bytes::Bytes; +use crate::raw::oio::Buffer; use crate::raw::*; use crate::*; @@ -56,17 +58,10 @@ pub enum TwoWays { } impl oio::Read for TwoWays { - async fn seek(&mut self, pos: SeekFrom) -> Result { + async fn read_at(&self, offset: u64, limit: usize) -> Result { match self { - Self::One(v) => v.seek(pos).await, - Self::Two(v) => v.seek(pos).await, - } - } - - async fn read(&mut self, limit: usize) -> Result { - match self { - Self::One(v) => v.read(limit).await, - Self::Two(v) => v.read(limit).await, + TwoWays::One(v) => v.read_at(offset, limit).await, + TwoWays::Two(v) => v.read_at(offset, limit).await, } } } @@ -123,19 +118,11 @@ pub enum ThreeWays { } impl oio::Read for ThreeWays { - async fn seek(&mut self, pos: SeekFrom) -> Result { - match self { - Self::One(v) => v.seek(pos).await, - Self::Two(v) => v.seek(pos).await, - Self::Three(v) => v.seek(pos).await, - } - } - - async fn read(&mut self, limit: usize) -> Result { + async fn read_at(&self, offset: u64, limit: usize) -> Result { match self { - Self::One(v) => v.read(limit).await, - Self::Two(v) => v.read(limit).await, - Self::Three(v) => v.read(limit).await, + ThreeWays::One(v) => v.read_at(offset, limit).await, + ThreeWays::Two(v) => v.read_at(offset, limit).await, + ThreeWays::Three(v) => v.read_at(offset, limit).await, } } } @@ -209,21 +196,12 @@ where THREE: oio::Read, FOUR: oio::Read, { - async fn seek(&mut self, pos: SeekFrom) -> Result { - match self { - Self::One(v) => v.seek(pos).await, - Self::Two(v) => v.seek(pos).await, - Self::Three(v) => v.seek(pos).await, - Self::Four(v) => v.seek(pos).await, - } - } - - async fn read(&mut self, limit: usize) -> Result { + async fn read_at(&self, offset: u64, limit: usize) -> Result { match self { - Self::One(v) => v.read(limit).await, - Self::Two(v) => v.read(limit).await, - Self::Three(v) => v.read(limit).await, - Self::Four(v) => v.read(limit).await, + FourWays::One(v) => v.read_at(offset, limit).await, + FourWays::Two(v) => v.read_at(offset, limit).await, + FourWays::Three(v) => v.read_at(offset, limit).await, + FourWays::Four(v) => v.read_at(offset, limit).await, } } } diff --git a/core/src/raw/http_util/body.rs b/core/src/raw/http_util/body.rs index d6df52c5e55b..068334044dae 100644 --- a/core/src/raw/http_util/body.rs +++ b/core/src/raw/http_util/body.rs @@ -15,17 +15,9 @@ // specific language governing permissions and limitations // under the License. -use std::cmp::min; -use std::cmp::Ordering; -use std::io; - -use bytes::Buf; -use bytes::BufMut; use bytes::Bytes; -use futures::StreamExt; use crate::raw::*; -use crate::*; /// Body used in async HTTP requests. #[derive(Default)] @@ -38,158 +30,3 @@ pub enum AsyncBody { /// Body with stream. Stream(oio::Streamer), } - -/// IncomingAsyncBody carries the content returned by remote servers. -/// -/// # Notes -/// -/// Client SHOULD NEVER construct this body. -pub struct IncomingAsyncBody { - /// # TODO - /// - /// hyper returns `impl Stream>` but we can't - /// write the types in stable. So we will box here. - /// - /// After [TAIT](https://rust-lang.github.io/rfcs/2515-type_alias_impl_trait.html) - /// has been stable, we can change `IncomingAsyncBody` into `IncomingAsyncBody`. - inner: oio::Streamer, - size: Option, - consumed: u64, - chunk: Bytes, -} - -impl IncomingAsyncBody { - /// Construct a new incoming async body - pub fn new(s: oio::Streamer, size: Option) -> Self { - Self { - inner: s, - size, - consumed: 0, - chunk: Bytes::new(), - } - } - - /// Create an empty IncomingAsyncBody. - #[allow(dead_code)] - pub(crate) fn empty() -> Self { - Self { - inner: Box::new(()), - size: Some(0), - consumed: 0, - chunk: Bytes::new(), - } - } - - /// Consume the entire body. - pub async fn consume(mut self) -> Result<()> { - use oio::Read; - - loop { - let buf = self.read(4 * 1024 * 1024).await.map_err(|err| { - Error::new(ErrorKind::Unexpected, "fetch bytes from stream") - .with_operation("http_util::IncomingAsyncBody::consume") - .set_source(err) - })?; - if buf.is_empty() { - break; - } - } - - Ok(()) - } - - /// Consume the response to bytes. - /// - /// This code is inspired from hyper's [`to_bytes`](https://docs.rs/hyper/0.14.23/hyper/body/fn.to_bytes.html). - pub async fn bytes(mut self) -> Result { - use oio::Read; - - // If there's only 1 chunk, we can just return Buf::to_bytes() - let first = self.read(4 * 1024 * 1024).await?; - if first.is_empty() { - return Ok(first); - } - - let second = self.read(4 * 1024 * 1024).await?; - if second.is_empty() { - return Ok(first); - } - - // With more than 1 buf, we gotta flatten into a Vec first. - let cap = if let Some(size) = self.size { - // The convert from u64 to usize could fail, but it's unlikely. - // Let's just make it overflow. - size as usize - } else { - // It's highly possible that we have more data to read. - // Add extra 16K buffer to avoid another allocation. - first.remaining() + second.remaining() + 16 * 1024 - }; - let mut vec = Vec::with_capacity(cap); - vec.put(first); - vec.put(second); - - // TODO: we can tune the io size here. - loop { - let buf = self.read(4 * 1024 * 1024).await?; - if buf.is_empty() { - break; - } - vec.put(buf); - } - - Ok(vec.into()) - } - - #[inline] - fn check(expect: u64, actual: u64) -> Result<()> { - match actual.cmp(&expect) { - Ordering::Equal => Ok(()), - Ordering::Less => Err(Error::new( - ErrorKind::ContentIncomplete, - &format!("reader got too little data, expect: {expect}, actual: {actual}"), - ) - .set_temporary()), - Ordering::Greater => Err(Error::new( - ErrorKind::ContentTruncated, - &format!("reader got too much data, expect: {expect}, actual: {actual}"), - ) - .set_temporary()), - } - } -} - -// impl oio::Read for IncomingAsyncBody { -// async fn read(&mut self, limit: usize) -> Result { -// if self.size == Some(0) { -// return Ok(Bytes::new()); -// } -// -// if self.chunk.is_empty() { -// self.chunk = match self.inner.next().await.transpose()? { -// Some(bs) => bs, -// None => { -// if let Some(size) = self.size { -// Self::check(size, self.consumed)? -// } -// -// return Ok(Bytes::new()); -// } -// }; -// } -// -// let size = min(limit, self.chunk.len()); -// self.consumed += size as u64; -// let bs = self.chunk.split_to(size); -// Ok(bs) -// } -// -// async fn seek(&mut self, pos: io::SeekFrom) -> Result { -// let _ = pos; -// -// Err(Error::new( -// ErrorKind::Unsupported, -// "output reader doesn't support seeking", -// )) -// } -// } diff --git a/core/src/raw/http_util/client.rs b/core/src/raw/http_util/client.rs index 8a51e79e927a..a7f1f9291a03 100644 --- a/core/src/raw/http_util/client.rs +++ b/core/src/raw/http_util/client.rs @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -use bytes::Bytes; +use bytes::{Buf, Bytes}; use std::cmp::Ordering; use std::fmt::Debug; use std::fmt::Formatter; @@ -26,7 +26,6 @@ use futures::TryStreamExt; use http::Request; use http::Response; -use super::body::IncomingAsyncBody; use super::parse_content_encoding; use super::parse_content_length; use super::AsyncBody; @@ -80,7 +79,7 @@ impl HttpClient { } /// Send a request in async way. - pub async fn send(&self, req: Request) -> Result> { + pub async fn send(&self, req: Request) -> Result> { // Uri stores all string alike data in `Bytes` which means // the clone here is cheap. let uri = req.uri().clone(); @@ -168,17 +167,18 @@ impl HttpClient { // Swap headers directly instead of copy the entire map. mem::swap(hr.headers_mut().unwrap(), resp.headers_mut()); - let bytes = resp.bytes().await.map_err(|err| { + let bs: Vec = resp.bytes_stream().try_collect().await.map_err(|err| { Error::new(ErrorKind::Unexpected, "read data from http response") .with_context("url", uri.to_string()) .set_source(err) })?; + let buffer = oio::Buffer::from(bs); if let Some(expect) = content_length { - check(expect, bytes.len() as u64)?; + check(expect, buffer.remaining() as u64)?; } - let resp = hr.body(bytes).expect("response must build succeed"); + let resp = hr.body(buffer).expect("response must build succeed"); Ok(resp) } diff --git a/core/src/raw/http_util/mod.rs b/core/src/raw/http_util/mod.rs index 5a5f375e8dba..70ddcc16d1ef 100644 --- a/core/src/raw/http_util/mod.rs +++ b/core/src/raw/http_util/mod.rs @@ -27,7 +27,6 @@ pub use client::HttpClient; mod body; pub use body::AsyncBody; -pub use body::IncomingAsyncBody; mod header; pub use header::build_header_value; diff --git a/core/src/raw/http_util/multipart.rs b/core/src/raw/http_util/multipart.rs index 4d40b01a6128..492d6a8b1b8f 100644 --- a/core/src/raw/http_util/multipart.rs +++ b/core/src/raw/http_util/multipart.rs @@ -41,7 +41,6 @@ use http::Version; use super::new_request_build_error; use super::AsyncBody; -use super::IncomingAsyncBody; use crate::raw::oio; use crate::raw::oio::Stream; use crate::raw::oio::Streamer; @@ -438,7 +437,7 @@ impl MixedPart { } /// Consume a mixed part to build a response. - pub fn into_response(mut self) -> Response { + pub fn into_response(mut self) -> Response { let mut builder = Response::builder(); builder = builder.status(self.status_code.unwrap_or(StatusCode::OK)); @@ -446,15 +445,16 @@ impl MixedPart { // Swap headers directly instead of copy the entire map. mem::swap(builder.headers_mut().unwrap(), &mut self.headers); - let body = if let Some(stream) = self.content { - IncomingAsyncBody::new(stream, Some(self.content_length)) - } else { - IncomingAsyncBody::new(Box::new(oio::into_stream(stream::empty())), Some(0)) - }; + // let body = if let Some(stream) = self.content { + // IncomingAsyncBody::new(stream, Some(self.content_length)) + // } else { + // IncomingAsyncBody::new(Box::new(oio::into_stream(stream::empty())), Some(0)) + // }; - builder - .body(body) - .expect("mixed part must be valid response") + // builder + // .body(body) + // .expect("mixed part must be valid response") + todo!() } /// Insert a part header into part. diff --git a/core/src/raw/oio/buf/buffer.rs b/core/src/raw/oio/buf/buffer.rs index 0b7c71bf21a4..32034a66a516 100644 --- a/core/src/raw/oio/buf/buffer.rs +++ b/core/src/raw/oio/buf/buffer.rs @@ -25,6 +25,31 @@ enum Inner { NonContiguous(VecDeque), } +impl Buffer { + #[inline] + pub fn new() -> Self { + Self(Inner::Contiguous(Bytes::new())) + } +} + +impl From for Buffer { + fn from(bs: Bytes) -> Self { + Self(Inner::Contiguous(bs)) + } +} + +impl From> for Buffer { + fn from(bs: VecDeque) -> Self { + Self(Inner::NonContiguous(bs)) + } +} + +impl From> for Buffer { + fn from(bs: Vec) -> Self { + Self(Inner::NonContiguous(bs.into())) + } +} + impl bytes::Buf for Buffer { #[inline] fn remaining(&self) -> usize { @@ -73,7 +98,7 @@ impl bytes::Buf for Buffer { match &mut self.0 { Inner::Contiguous(b) => b.copy_to_bytes(len), Inner::NonContiguous(v) => { - if len <= v[0].remaining() { + if len > 0 && len <= v[0].remaining() { let bs = v[0].copy_to_bytes(len); if v[0].is_empty() { v.remove(0); diff --git a/core/src/raw/oio/buf/mod.rs b/core/src/raw/oio/buf/mod.rs index 1a22305895da..3009e803e2a1 100644 --- a/core/src/raw/oio/buf/mod.rs +++ b/core/src/raw/oio/buf/mod.rs @@ -16,6 +16,7 @@ // under the License. mod adaptive; -mod buffer; - pub use adaptive::AdaptiveBuf; + +mod buffer; +pub use buffer::Buffer; diff --git a/core/src/raw/oio/cursor.rs b/core/src/raw/oio/cursor.rs index 75a034f1023e..7320d764bc14 100644 --- a/core/src/raw/oio/cursor.rs +++ b/core/src/raw/oio/cursor.rs @@ -70,39 +70,39 @@ impl From> for Cursor { } } -impl oio::Read for Cursor { - async fn read(&mut self, limit: usize) -> Result { - if self.is_empty() { - Ok(Bytes::new()) - } else { - // The clone here is required as we don't want to change it. - let mut bs = self.inner.clone().split_off(self.pos as usize); - let bs = bs.split_to(min(bs.len(), limit)); - self.pos += bs.len() as u64; - Ok(bs) - } - } - - async fn seek(&mut self, pos: SeekFrom) -> Result { - let (base, amt) = match pos { - SeekFrom::Start(n) => (0, n as i64), - SeekFrom::End(n) => (self.inner.len() as i64, n), - SeekFrom::Current(n) => (self.pos as i64, n), - }; - - let n = match base.checked_add(amt) { - Some(n) if n >= 0 => n as u64, - _ => { - return Err(Error::new( - ErrorKind::InvalidInput, - "invalid seek to a negative or overflowing position", - )) - } - }; - self.pos = n; - Ok(n) - } -} +// impl oio::Read for Cursor { +// async fn read(&mut self, limit: usize) -> Result { +// if self.is_empty() { +// Ok(Bytes::new()) +// } else { +// // The clone here is required as we don't want to change it. +// let mut bs = self.inner.clone().split_off(self.pos as usize); +// let bs = bs.split_to(min(bs.len(), limit)); +// self.pos += bs.len() as u64; +// Ok(bs) +// } +// } +// +// async fn seek(&mut self, pos: SeekFrom) -> Result { +// let (base, amt) = match pos { +// SeekFrom::Start(n) => (0, n as i64), +// SeekFrom::End(n) => (self.inner.len() as i64, n), +// SeekFrom::Current(n) => (self.pos as i64, n), +// }; +// +// let n = match base.checked_add(amt) { +// Some(n) if n >= 0 => n as u64, +// _ => { +// return Err(Error::new( +// ErrorKind::InvalidInput, +// "invalid seek to a negative or overflowing position", +// )) +// } +// }; +// self.pos = n; +// Ok(n) +// } +// } impl oio::BlockingRead for Cursor { fn read(&mut self, limit: usize) -> Result { diff --git a/core/src/raw/oio/read/api.rs b/core/src/raw/oio/read/api.rs index 4ca6d0457f15..371347107e88 100644 --- a/core/src/raw/oio/read/api.rs +++ b/core/src/raw/oio/read/api.rs @@ -18,12 +18,13 @@ use std::fmt::Display; use std::fmt::Formatter; use std::io; -use std::ops::DerefMut; +use std::ops::Deref; -use bytes::Bytes; +use bytes::{Buf, Bytes}; use futures::Future; -use crate::raw::BoxedFuture; +use crate::raw::oio::Buffer; +use crate::raw::*; use crate::*; /// PageOperation is the name for APIs of lister. @@ -88,13 +89,17 @@ pub type Reader = Box; /// once, at the outermost level of our API. pub trait Read: Unpin + Send + Sync { #[cfg(not(target_arch = "wasm32"))] - fn read_at(&self, offset: u64, limit: usize) -> impl Future> + Send; + fn read_at( + &self, + offset: u64, + limit: usize, + ) -> impl Future> + Send; #[cfg(target_arch = "wasm32")] - fn read_at(&self, offset: u64, limit: usize) -> impl Future>; + fn read_at(&self, offset: u64, limit: usize) -> impl Future>; } impl Read for () { - async fn read_at(&mut self, offset: u64, limit: usize) -> Result { + async fn read_at(&self, offset: u64, limit: usize) -> Result { let (_, _) = (offset, limit); Err(Error::new( @@ -104,12 +109,24 @@ impl Read for () { } } +impl Read for Bytes { + /// TODO: we can check if the offset is out of range. + async fn read_at(&self, offset: u64, limit: usize) -> Result { + if offset >= self.len() as u64 { + return Ok(Buffer::new()); + } + let offset = offset as usize; + let limit = limit.min(self.len() - offset); + Ok(Buffer::from(self.slice(offset..offset + limit))) + } +} + pub trait ReadDyn: Unpin + Send + Sync { - fn read_at_dyn(&self, offset: u64, limit: usize) -> BoxedFuture>; + fn read_at_dyn(&self, offset: u64, limit: usize) -> BoxedFuture>; } impl ReadDyn for T { - fn read_at_dyn(&self, offset: u64, limit: usize) -> BoxedFuture> { + fn read_at_dyn(&self, offset: u64, limit: usize) -> BoxedFuture> { Box::pin(self.read_at(offset, limit)) } } @@ -119,8 +136,8 @@ impl ReadDyn for T { /// Take care about the `deref_mut()` here. This makes sure that we are calling functions /// upon `&mut T` instead of `&mut Box`. The later could result in infinite recursion. impl Read for Box { - async fn read_at(&mut self, offset: u64, limit: usize) -> Result { - self.deref_mut().read_at_dyn(offset, limit).await + async fn read_at(&self, offset: u64, limit: usize) -> Result { + self.deref().read_at_dyn(offset, limit).await } } diff --git a/core/src/raw/oio/read/mod.rs b/core/src/raw/oio/read/mod.rs index d05b4593c8b1..f2673e251a4c 100644 --- a/core/src/raw/oio/read/mod.rs +++ b/core/src/raw/oio/read/mod.rs @@ -22,31 +22,31 @@ pub use api::Read; pub use api::ReadOperation; pub use api::Reader; -mod into_streamable_read; -pub use into_streamable_read::into_streamable_read; -pub use into_streamable_read::StreamableReader; +// mod into_streamable_read; +// pub use into_streamable_read::into_streamable_read; +// pub use into_streamable_read::StreamableReader; -mod range_read; -pub use range_read::RangeReader; +// mod range_read; +// pub use range_read::RangeReader; -mod file_read; -pub use file_read::FileReader; +// mod file_read; +// pub use file_read::FileReader; -mod into_read_from_stream; -pub use into_read_from_stream::into_read_from_stream; -pub use into_read_from_stream::FromStreamReader; +// mod into_read_from_stream; +// pub use into_read_from_stream::into_read_from_stream; +// pub use into_read_from_stream::FromStreamReader; -mod futures_read; -pub use futures_read::FuturesReader; +// mod futures_read; +// pub use futures_read::FuturesReader; -mod tokio_read; -pub use tokio_read::TokioReader; +// mod tokio_read; +// pub use tokio_read::TokioReader; mod std_read; pub use std_read::StdReader; -mod lazy_read; -pub use lazy_read::LazyReader; +// mod lazy_read; +// pub use lazy_read::LazyReader; -mod buffer_reader; -pub use buffer_reader::BufferReader; +// mod buffer_reader; +// pub use buffer_reader::BufferReader; diff --git a/core/src/services/fs/backend.rs b/core/src/services/fs/backend.rs index 68ebf1b398e0..96dd7b666329 100644 --- a/core/src/services/fs/backend.rs +++ b/core/src/services/fs/backend.rs @@ -27,6 +27,7 @@ use uuid::Uuid; use super::lister::FsLister; use super::writer::FsWriter; use crate::raw::*; +use crate::services::fs::reader::FsReader; use crate::*; /// POSIX file system support. @@ -240,7 +241,7 @@ impl FsBackend { #[async_trait] impl Accessor for FsBackend { - type Reader = oio::TokioReader; + type Reader = FsReader; type Writer = FsWriter; type Lister = Option>; type BlockingReader = oio::StdReader; @@ -327,7 +328,7 @@ impl Accessor for FsBackend { .await .map_err(new_std_io_error)?; - let r = oio::TokioReader::new(f); + let r = FsReader::new(f); Ok((RpRead::new(), r)) } diff --git a/core/src/services/fs/mod.rs b/core/src/services/fs/mod.rs index fd51bffc96ec..795180d41601 100644 --- a/core/src/services/fs/mod.rs +++ b/core/src/services/fs/mod.rs @@ -19,4 +19,5 @@ mod backend; pub use backend::FsBuilder as Fs; mod lister; +mod reader; mod writer; diff --git a/core/src/services/fs/reader.rs b/core/src/services/fs/reader.rs new file mode 100644 index 000000000000..fda7e317269c --- /dev/null +++ b/core/src/services/fs/reader.rs @@ -0,0 +1,37 @@ +// 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 crate::raw::oio; +use crate::*; +use std::future::Future; +use std::path::PathBuf; + +pub struct FsReader { + f: F, +} + +impl FsReader { + pub fn new(f: F) -> Self { + Self { f } + } +} + +impl oio::Read for FsReader { + async fn read_at(&self, offset: u64, limit: usize) -> Result { + todo!() + } +} diff --git a/core/src/types/operator/operator.rs b/core/src/types/operator/operator.rs index aa12f8e74082..0d864c4ced73 100644 --- a/core/src/types/operator/operator.rs +++ b/core/src/types/operator/operator.rs @@ -526,10 +526,10 @@ impl Operator { let (_, r) = inner.read(&path, args.with_range(range)).await?; let mut r = Reader::new(r); - let mut buf = Vec::with_capacity(size_hint as usize); - r.read_to_end(&mut buf).await?; - - Ok(buf) + // let mut buf = Vec::with_capacity(size_hint as usize); + // r.read_to_end(&mut buf).await?; + todo!() + // Ok(buf) }, ) } diff --git a/core/src/types/reader.rs b/core/src/types/reader.rs index 8db4e9f9f4ad..4c5a1248af31 100644 --- a/core/src/types/reader.rs +++ b/core/src/types/reader.rs @@ -17,12 +17,13 @@ use std::io; use std::io::SeekFrom; +use std::ops::{Range, RangeBounds}; use std::pin::Pin; use std::task::ready; use std::task::Context; use std::task::Poll; -use bytes::{BufMut, Bytes, BytesMut}; +use bytes::{Buf, BufMut, Bytes, BytesMut}; use futures::Stream; use tokio::io::ReadBuf; @@ -65,15 +66,13 @@ use crate::*; /// /// [`Reader`] provides [`Reader::into_tokio_read`] to remove extra APIs upon self. pub struct Reader { - state: State, + inner: oio::Reader, } impl Reader { /// Create a new reader from an `oio::Reader`. pub(crate) fn new(r: oio::Reader) -> Self { - Reader { - state: State::Idle(Some(r)), - } + Reader { inner: r } } /// Create a new reader. @@ -86,9 +85,7 @@ impl Reader { pub(crate) async fn create(acc: FusedAccessor, path: &str, op: OpRead) -> Result { let (_, r) = acc.read(path, op).await?; - Ok(Reader { - state: State::Idle(Some(r)), - }) + Ok(Reader { inner: r }) } /// Convert [`Reader`] into an [`futures::AsyncRead`] and [`futures::AsyncSeek`] @@ -98,13 +95,14 @@ impl Reader { /// /// The returning type also implements `Send`, `Sync` and `Unpin`, so users can use it /// as `Box` and calling `poll_read_unpin` on it. - #[inline] - #[cfg(not(target_arch = "wasm32"))] - pub fn into_futures_read( - self, - ) -> impl futures::AsyncRead + futures::AsyncSeek + Send + Sync + Unpin { - self - } + // #[inline] + // #[cfg(not(target_arch = "wasm32"))] + // pub fn into_futures_read( + // self, + // ) -> impl futures::AsyncRead + futures::AsyncSeek + Send + Sync + Unpin { + // // self + // todo!() + // } /// Convert [`Reader`] into an [`tokio::io::AsyncRead`] and [`tokio::io::AsyncSeek`] /// @@ -113,114 +111,76 @@ impl Reader { /// /// The returning type also implements `Send`, `Sync` and `Unpin`, so users can use it /// as `Box` and calling `poll_read_unpin` on it. - #[inline] - #[cfg(not(target_arch = "wasm32"))] - pub fn into_tokio_read( - self, - ) -> impl tokio::io::AsyncRead + tokio::io::AsyncSeek + Send + Sync + Unpin { - self - } - - /// Seek to the position of `pos` of reader. - #[inline] - pub async fn seek(&mut self, pos: SeekFrom) -> Result { - let State::Idle(Some(r)) = &mut self.state else { - return Err(Error::new(ErrorKind::Unexpected, "reader must be valid")); - }; - r.seek_dyn(pos).await - } + // #[inline] + // #[cfg(not(target_arch = "wasm32"))] + // pub fn into_tokio_read( + // self, + // ) -> impl tokio::io::AsyncRead + tokio::io::AsyncSeek + Send + Sync + Unpin { + // // self + // todo!() + // } /// Read at most `size` bytes of data from reader. #[inline] - pub async fn read(&mut self, limit: usize) -> Result { - let State::Idle(Some(r)) = &mut self.state else { - return Err(Error::new(ErrorKind::Unexpected, "reader must be valid")); - }; - r.read_dyn(limit).await + pub async fn read_at(&self, offset: u64, limit: usize) -> Result { + self.inner.read_at_dyn(offset, limit).await } /// Read exact `size` bytes of data from reader. - pub async fn read_exact(&mut self, size: usize) -> Result { - let State::Idle(Some(r)) = &mut self.state else { - return Err(Error::new(ErrorKind::Unexpected, "reader must be valid")); - }; - - // Lucky path. - let bs1 = r.read_dyn(size).await?; - debug_assert!( - bs1.len() <= size, - "read should not return more bytes than expected" - ); - if bs1.len() == size { - return Ok(bs1); - } - if bs1.is_empty() { - return Err( - Error::new(ErrorKind::ContentIncomplete, "reader got too little data") - .with_context("expect", size.to_string()), - ); - } - - let mut bs = BytesMut::with_capacity(size); - bs.put_slice(&bs1); - - let mut remaining = size - bs.len(); + pub async fn read_exact_at(&self, offset: u64, size: usize) -> Result { + let mut buf = BytesMut::with_capacity(size); + let mut remaining = size; + let mut cur = offset; loop { - let tmp = r.read_dyn(remaining).await?; - if tmp.is_empty() { - return Err( - Error::new(ErrorKind::ContentIncomplete, "reader got too little data") - .with_context("expect", size.to_string()) - .with_context("actual", bs.len().to_string()), - ); - } - bs.put_slice(&tmp); - debug_assert!( - tmp.len() <= remaining, - "read should not return more bytes than expected" - ); - - remaining -= tmp.len(); + let bs = self.inner.read_at_dyn(cur, remaining).await?; + let n = bs.remaining(); + buf.put(bs); + cur += n as u64; + remaining -= n; if remaining == 0 { break; } } + Ok(buf.freeze()) + } - Ok(bs.freeze()) + pub async fn read_range(&self, range: Range) -> Result { + todo!() } /// Reads all bytes until EOF in this source, placing them into buf. - pub async fn read_to_end(&mut self, buf: &mut Vec) -> Result { - let start_len = buf.len(); - - loop { - if buf.len() == buf.capacity() { - buf.reserve(32); // buf is full, need more space - } - - let spare = buf.spare_capacity_mut(); - let mut read_buf: ReadBuf = ReadBuf::uninit(spare); - - // SAFETY: These bytes were initialized but not filled in the previous loop - unsafe { - read_buf.assume_init(read_buf.capacity()); - } - - match self.read(read_buf.initialize_unfilled().len()).await { - Ok(bs) if bs.is_empty() => { - return Ok(buf.len() - start_len); - } - Ok(bs) => { - read_buf.initialize_unfilled()[..bs.len()].copy_from_slice(&bs); - // SAFETY: Read API makes sure that returning `n` is correct. - unsafe { - buf.set_len(buf.len() + bs.len()); - } - } - Err(e) => return Err(e), - } - } + pub async fn read_to_end_at(&self, offset: u64, buf: &mut Vec) -> Result { + todo!() + // let start_len = buf.len(); + // + // loop { + // if buf.len() == buf.capacity() { + // buf.reserve(32); // buf is full, need more space + // } + // + // let spare = buf.spare_capacity_mut(); + // let mut read_buf: ReadBuf = ReadBuf::uninit(spare); + // + // // SAFETY: These bytes were initialized but not filled in the previous loop + // unsafe { + // read_buf.assume_init(read_buf.capacity()); + // } + // + // match self.read(read_buf.initialize_unfilled().len()).await { + // Ok(bs) if bs.is_empty() => { + // return Ok(buf.len() - start_len); + // } + // Ok(bs) => { + // read_buf.initialize_unfilled()[..bs.len()].copy_from_slice(&bs); + // // SAFETY: Read API makes sure that returning `n` is correct. + // unsafe { + // buf.set_len(buf.len() + bs.len()); + // } + // } + // Err(e) => return Err(e), + // } + // } } } @@ -235,191 +195,191 @@ enum State { /// Reader will only be used with `&mut self`. unsafe impl Sync for State {} -impl futures::AsyncRead for Reader { - fn poll_read( - mut self: Pin<&mut Self>, - cx: &mut Context<'_>, - buf: &mut [u8], - ) -> Poll> { - use oio::Read; - - match &mut self.state { - State::Idle(r) => { - let mut r = r.take().expect("reader must be valid"); - let size = buf.len(); - let fut = async move { - let res = r.read(size).await; - (r, res) - }; - self.state = State::Reading(Box::pin(fut)); - self.poll_read(cx, buf) - } - State::Reading(fut) => { - let (r, res) = ready!(fut.as_mut().poll(cx)); - self.state = State::Idle(Some(r)); - let bs = res.map_err(format_std_io_error)?; - let n = bs.len(); - buf[..n].copy_from_slice(&bs); - Poll::Ready(Ok(n)) - } - State::Seeking(_) => Poll::Ready(Err(io::Error::new( - io::ErrorKind::Interrupted, - "another io operation is in progress", - ))), - } - } -} - -impl futures::AsyncSeek for Reader { - fn poll_seek( - mut self: Pin<&mut Self>, - cx: &mut Context<'_>, - pos: io::SeekFrom, - ) -> Poll> { - use oio::Read; - - match &mut self.state { - State::Idle(r) => { - let mut r = r.take().expect("reader must be valid"); - let fut = async move { - let res = r.seek(pos).await; - (r, res) - }; - self.state = State::Seeking(Box::pin(fut)); - self.poll_seek(cx, pos) - } - State::Seeking(fut) => { - let (r, res) = ready!(fut.as_mut().poll(cx)); - self.state = State::Idle(Some(r)); - Poll::Ready(res.map_err(format_std_io_error)) - } - State::Reading(_) => Poll::Ready(Err(io::Error::new( - io::ErrorKind::Interrupted, - "another io operation is in progress", - ))), - } - } -} - -impl tokio::io::AsyncRead for Reader { - fn poll_read( - mut self: Pin<&mut Self>, - cx: &mut Context<'_>, - buf: &mut tokio::io::ReadBuf<'_>, - ) -> Poll> { - use oio::Read; - - loop { - match &mut self.state { - State::Idle(r) => { - // Safety: We make sure that we will set filled correctly. - unsafe { buf.assume_init(buf.remaining()) } - let size = buf.initialize_unfilled().len(); - - let mut r = r.take().expect("reader must be valid"); - let fut = async move { - let res = r.read(size).await; - (r, res) - }; - self.state = State::Reading(Box::pin(fut)); - } - State::Reading(fut) => { - let (r, res) = ready!(fut.as_mut().poll(cx)); - self.state = State::Idle(Some(r)); - let bs = res.map_err(format_std_io_error)?; - let n = bs.len(); - buf.initialize_unfilled()[..n].copy_from_slice(&bs); - buf.advance(n); - return Poll::Ready(Ok(())); - } - State::Seeking(_) => { - return Poll::Ready(Err(io::Error::new( - io::ErrorKind::Interrupted, - "another io operation is in progress", - ))) - } - } - } - } -} - -impl tokio::io::AsyncSeek for Reader { - fn start_seek(mut self: Pin<&mut Self>, pos: io::SeekFrom) -> io::Result<()> { - use oio::Read; - - match &mut self.state { - State::Idle(r) => { - let mut r = r.take().expect("reader must be valid"); - let fut = async move { - let res = r.seek(pos).await; - (r, res) - }; - self.state = State::Seeking(Box::pin(fut)); - Ok(()) - } - State::Seeking(_) | State::Reading(_) => Err(io::Error::new( - io::ErrorKind::Interrupted, - "another io operation is in progress", - )), - } - } - - fn poll_complete(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - match &mut self.state { - State::Idle(_) => { - // AsyncSeek recommends calling poll_complete before start_seek. - // We don't have to guarantee that the value returned by - // poll_complete called without start_seek is correct, - // so we'll return 0. - Poll::Ready(Ok(0)) - } - State::Seeking(fut) => { - let (r, res) = ready!(fut.as_mut().poll(cx)); - self.state = State::Idle(Some(r)); - Poll::Ready(res.map_err(format_std_io_error)) - } - State::Reading(_) => Poll::Ready(Err(io::Error::new( - io::ErrorKind::Interrupted, - "another io operation is in progress", - ))), - } - } -} - -impl Stream for Reader { - type Item = io::Result; - - fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - use oio::Read; - - match &mut self.state { - State::Idle(r) => { - let mut r = r.take().expect("reader must be valid"); - let fut = async move { - // TODO: should allow user to tune this value. - let res = r.read(4 * 1024 * 1024).await; - (r, res) - }; - self.state = State::Reading(Box::pin(fut)); - self.poll_next(cx) - } - State::Reading(fut) => { - let (r, res) = ready!(fut.as_mut().poll(cx)); - self.state = State::Idle(Some(r)); - let bs = res.map_err(format_std_io_error)?; - if bs.is_empty() { - Poll::Ready(None) - } else { - Poll::Ready(Some(Ok(bs))) - } - } - State::Seeking(_) => Poll::Ready(Some(Err(io::Error::new( - io::ErrorKind::Interrupted, - "another io operation is in progress", - )))), - } - } -} +// impl futures::AsyncRead for Reader { +// fn poll_read( +// mut self: Pin<&mut Self>, +// cx: &mut Context<'_>, +// buf: &mut [u8], +// ) -> Poll> { +// use oio::Read; +// +// match &mut self.state { +// State::Idle(r) => { +// let mut r = r.take().expect("reader must be valid"); +// let size = buf.len(); +// let fut = async move { +// let res = r.read(size).await; +// (r, res) +// }; +// self.state = State::Reading(Box::pin(fut)); +// self.poll_read(cx, buf) +// } +// State::Reading(fut) => { +// let (r, res) = ready!(fut.as_mut().poll(cx)); +// self.state = State::Idle(Some(r)); +// let bs = res.map_err(format_std_io_error)?; +// let n = bs.len(); +// buf[..n].copy_from_slice(&bs); +// Poll::Ready(Ok(n)) +// } +// State::Seeking(_) => Poll::Ready(Err(io::Error::new( +// io::ErrorKind::Interrupted, +// "another io operation is in progress", +// ))), +// } +// } +// } + +// impl futures::AsyncSeek for Reader { +// fn poll_seek( +// mut self: Pin<&mut Self>, +// cx: &mut Context<'_>, +// pos: io::SeekFrom, +// ) -> Poll> { +// use oio::Read; +// +// match &mut self.state { +// State::Idle(r) => { +// let mut r = r.take().expect("reader must be valid"); +// let fut = async move { +// let res = r.seek(pos).await; +// (r, res) +// }; +// self.state = State::Seeking(Box::pin(fut)); +// self.poll_seek(cx, pos) +// } +// State::Seeking(fut) => { +// let (r, res) = ready!(fut.as_mut().poll(cx)); +// self.state = State::Idle(Some(r)); +// Poll::Ready(res.map_err(format_std_io_error)) +// } +// State::Reading(_) => Poll::Ready(Err(io::Error::new( +// io::ErrorKind::Interrupted, +// "another io operation is in progress", +// ))), +// } +// } +// } +// +// impl tokio::io::AsyncRead for Reader { +// fn poll_read( +// mut self: Pin<&mut Self>, +// cx: &mut Context<'_>, +// buf: &mut tokio::io::ReadBuf<'_>, +// ) -> Poll> { +// use oio::Read; +// +// loop { +// match &mut self.state { +// State::Idle(r) => { +// // Safety: We make sure that we will set filled correctly. +// unsafe { buf.assume_init(buf.remaining()) } +// let size = buf.initialize_unfilled().len(); +// +// let mut r = r.take().expect("reader must be valid"); +// let fut = async move { +// let res = r.read(size).await; +// (r, res) +// }; +// self.state = State::Reading(Box::pin(fut)); +// } +// State::Reading(fut) => { +// let (r, res) = ready!(fut.as_mut().poll(cx)); +// self.state = State::Idle(Some(r)); +// let bs = res.map_err(format_std_io_error)?; +// let n = bs.len(); +// buf.initialize_unfilled()[..n].copy_from_slice(&bs); +// buf.advance(n); +// return Poll::Ready(Ok(())); +// } +// State::Seeking(_) => { +// return Poll::Ready(Err(io::Error::new( +// io::ErrorKind::Interrupted, +// "another io operation is in progress", +// ))) +// } +// } +// } +// } +// } + +// impl tokio::io::AsyncSeek for Reader { +// fn start_seek(mut self: Pin<&mut Self>, pos: io::SeekFrom) -> io::Result<()> { +// use oio::Read; +// +// match &mut self.state { +// State::Idle(r) => { +// let mut r = r.take().expect("reader must be valid"); +// let fut = async move { +// let res = r.seek(pos).await; +// (r, res) +// }; +// self.state = State::Seeking(Box::pin(fut)); +// Ok(()) +// } +// State::Seeking(_) | State::Reading(_) => Err(io::Error::new( +// io::ErrorKind::Interrupted, +// "another io operation is in progress", +// )), +// } +// } +// +// fn poll_complete(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { +// match &mut self.state { +// State::Idle(_) => { +// // AsyncSeek recommends calling poll_complete before start_seek. +// // We don't have to guarantee that the value returned by +// // poll_complete called without start_seek is correct, +// // so we'll return 0. +// Poll::Ready(Ok(0)) +// } +// State::Seeking(fut) => { +// let (r, res) = ready!(fut.as_mut().poll(cx)); +// self.state = State::Idle(Some(r)); +// Poll::Ready(res.map_err(format_std_io_error)) +// } +// State::Reading(_) => Poll::Ready(Err(io::Error::new( +// io::ErrorKind::Interrupted, +// "another io operation is in progress", +// ))), +// } +// } +// } + +// impl Stream for Reader { +// type Item = io::Result; +// +// fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { +// use oio::Read; +// +// match &mut self.state { +// State::Idle(r) => { +// let mut r = r.take().expect("reader must be valid"); +// let fut = async move { +// // TODO: should allow user to tune this value. +// let res = r.read(4 * 1024 * 1024).await; +// (r, res) +// }; +// self.state = State::Reading(Box::pin(fut)); +// self.poll_next(cx) +// } +// State::Reading(fut) => { +// let (r, res) = ready!(fut.as_mut().poll(cx)); +// self.state = State::Idle(Some(r)); +// let bs = res.map_err(format_std_io_error)?; +// if bs.is_empty() { +// Poll::Ready(None) +// } else { +// Poll::Ready(Some(Ok(bs))) +// } +// } +// State::Seeking(_) => Poll::Ready(Some(Err(io::Error::new( +// io::ErrorKind::Interrupted, +// "another io operation is in progress", +// )))), +// } +// } +// } /// BlockingReader is designed to read data from given path in an blocking /// manner. From 07e908d998bdba00c4b4f8be9edf0d6daae51a5b Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Mon, 18 Mar 2024 18:03:16 +0800 Subject: [PATCH 004/111] Build pass Signed-off-by: Xuanwo --- core/src/raw/oio/buf/buffer.rs | 2 +- core/src/services/s3/backend.rs | 41 +++++++----------------- core/src/services/s3/core.rs | 44 ++++++++++++-------------- core/src/services/s3/error.rs | 6 ++-- core/src/services/s3/lister.rs | 2 +- core/src/services/s3/mod.rs | 1 + core/src/services/s3/reader.rs | 55 +++++++++++++++++++++++++++++++++ core/src/services/s3/writer.rs | 24 ++++---------- 8 files changed, 98 insertions(+), 77 deletions(-) create mode 100644 core/src/services/s3/reader.rs diff --git a/core/src/raw/oio/buf/buffer.rs b/core/src/raw/oio/buf/buffer.rs index 32034a66a516..9ec1f65dbc9f 100644 --- a/core/src/raw/oio/buf/buffer.rs +++ b/core/src/raw/oio/buf/buffer.rs @@ -27,7 +27,7 @@ enum Inner { impl Buffer { #[inline] - pub fn new() -> Self { + pub const fn new() -> Self { Self(Inner::Contiguous(Bytes::new())) } } diff --git a/core/src/services/s3/backend.rs b/core/src/services/s3/backend.rs index bb7b3949c383..89c3d8384bbc 100644 --- a/core/src/services/s3/backend.rs +++ b/core/src/services/s3/backend.rs @@ -46,6 +46,7 @@ use super::lister::S3Lister; use super::writer::S3Writer; use super::writer::S3Writers; use crate::raw::*; +use crate::services::s3::reader::S3Reader; use crate::*; /// Allow constructing correct region endpoint if user gives a global endpoint. @@ -984,7 +985,7 @@ pub struct S3Backend { #[cfg_attr(not(target_arch = "wasm32"), async_trait)] #[cfg_attr(target_arch = "wasm32", async_trait(?Send))] impl Accessor for S3Backend { - type Reader = IncomingAsyncBody; + type Reader = S3Reader; type Writer = S3Writers; type Lister = oio::PageLister; type BlockingReader = (); @@ -1065,25 +1066,10 @@ impl Accessor for S3Backend { } async fn read(&self, path: &str, args: OpRead) -> Result<(RpRead, Self::Reader)> { - let resp = self.core.s3_get_object(path, args).await?; - - let status = resp.status(); - - match status { - StatusCode::OK | StatusCode::PARTIAL_CONTENT => { - let size = parse_content_length(resp.headers())?; - let range = parse_content_range(resp.headers())?; - Ok(( - RpRead::new().with_size(size).with_range(range), - resp.into_body(), - )) - } - StatusCode::RANGE_NOT_SATISFIABLE => { - resp.into_body().consume().await?; - Ok((RpRead::new().with_size(Some(0)), IncomingAsyncBody::empty())) - } - _ => Err(parse_error(resp).await?), - } + Ok(( + RpRead::default(), + S3Reader::new(self.core.clone(), path, args), + )) } async fn write(&self, path: &str, args: OpWrite) -> Result<(RpWrite, Self::Writer)> { @@ -1127,13 +1113,7 @@ impl Accessor for S3Backend { let status = resp.status(); match status { - StatusCode::OK => { - // According to the documentation, when using copy_object, a 200 error may occur and we need to detect it. - // https://docs.aws.amazon.com/AmazonS3/latest/API/API_CopyObject.html#API_CopyObject_RequestSyntax - resp.into_body().consume().await?; - - Ok(RpCopy::default()) - } + StatusCode::OK => Ok(RpCopy::default()), _ => Err(parse_error(resp).await?), } } @@ -1144,7 +1124,10 @@ impl Accessor for S3Backend { // We will not send this request out, just for signing. let mut req = match op { PresignOperation::Stat(v) => self.core.s3_head_object_request(path, v)?, - PresignOperation::Read(v) => self.core.s3_get_object_request(path, v)?, + PresignOperation::Read(v) => { + self.core + .s3_get_object_request(path, BytesRange::default(), &v)? + } PresignOperation::Write(_) => self.core.s3_put_object_request( path, None, @@ -1182,7 +1165,7 @@ impl Accessor for S3Backend { let status = resp.status(); if let StatusCode::OK = status { - let bs = resp.into_body().bytes().await?; + let bs = resp.into_body(); let result: DeleteObjectsResult = quick_xml::de::from_reader(bs.reader()).map_err(new_xml_deserialize_error)?; diff --git a/core/src/services/s3/core.rs b/core/src/services/s3/core.rs index 7a271bec8e88..228a037026fd 100644 --- a/core/src/services/s3/core.rs +++ b/core/src/services/s3/core.rs @@ -182,7 +182,7 @@ impl S3Core { } #[inline] - pub async fn send(&self, req: Request) -> Result> { + pub async fn send(&self, req: Request) -> Result> { self.client.send(req).await } @@ -300,7 +300,12 @@ impl S3Core { Ok(req) } - pub fn s3_get_object_request(&self, path: &str, args: OpRead) -> Result> { + pub fn s3_get_object_request( + &self, + path: &str, + range: BytesRange, + args: &OpRead, + ) -> Result> { let p = build_abs_path(&self.root, path); // Construct headers to add to the request @@ -335,7 +340,6 @@ impl S3Core { let mut req = Request::get(&url); - let range = args.range(); if !range.is_full() { req = req.header(http::header::RANGE, range.to_header()); } @@ -361,9 +365,10 @@ impl S3Core { pub async fn s3_get_object( &self, path: &str, - args: OpRead, - ) -> Result> { - let mut req = self.s3_get_object_request(path, args)?; + range: BytesRange, + args: &OpRead, + ) -> Result> { + let mut req = self.s3_get_object_request(path, range, &args)?; self.sign(&mut req).await?; @@ -413,11 +418,7 @@ impl S3Core { Ok(req) } - pub async fn s3_head_object( - &self, - path: &str, - args: OpStat, - ) -> Result> { + pub async fn s3_head_object(&self, path: &str, args: OpStat) -> Result> { let mut req = self.s3_head_object_request(path, args)?; self.sign(&mut req).await?; @@ -425,7 +426,7 @@ impl S3Core { self.send(req).await } - pub async fn s3_delete_object(&self, path: &str) -> Result> { + pub async fn s3_delete_object(&self, path: &str) -> Result> { let p = build_abs_path(&self.root, path); let url = format!("{}/{}", self.endpoint, percent_encode_path(&p)); @@ -439,11 +440,7 @@ impl S3Core { self.send(req).await } - pub async fn s3_copy_object( - &self, - from: &str, - to: &str, - ) -> Result> { + pub async fn s3_copy_object(&self, from: &str, to: &str) -> Result> { let from = build_abs_path(&self.root, from); let to = build_abs_path(&self.root, to); @@ -508,7 +505,7 @@ impl S3Core { delimiter: &str, limit: Option, start_after: Option, - ) -> Result> { + ) -> Result> { let p = build_abs_path(&self.root, path); let mut url = format!("{}?list-type=2", self.endpoint); @@ -553,7 +550,7 @@ impl S3Core { &self, path: &str, args: &OpWrite, - ) -> Result> { + ) -> Result> { let p = build_abs_path(&self.root, path); let url = format!("{}/{}?uploads", self.endpoint, percent_encode_path(&p)); @@ -625,7 +622,7 @@ impl S3Core { path: &str, upload_id: &str, parts: Vec, - ) -> Result> { + ) -> Result> { let p = build_abs_path(&self.root, path); let url = format!( @@ -661,7 +658,7 @@ impl S3Core { &self, path: &str, upload_id: &str, - ) -> Result> { + ) -> Result> { let p = build_abs_path(&self.root, path); let url = format!( @@ -678,10 +675,7 @@ impl S3Core { self.send(req).await } - pub async fn s3_delete_objects( - &self, - paths: Vec, - ) -> Result> { + pub async fn s3_delete_objects(&self, paths: Vec) -> Result> { let url = format!("{}/?delete", self.endpoint); let req = Request::post(&url); diff --git a/core/src/services/s3/error.rs b/core/src/services/s3/error.rs index f683554877f7..388324b9efe5 100644 --- a/core/src/services/s3/error.rs +++ b/core/src/services/s3/error.rs @@ -36,9 +36,9 @@ struct S3Error { } /// Parse error response into Error. -pub async fn parse_error(resp: Response) -> Result { - let (parts, body) = resp.into_parts(); - let bs = body.bytes().await?; +pub async fn parse_error(resp: Response) -> Result { + let (parts, mut body) = resp.into_parts(); + let bs = body.copy_to_bytes(body.remaining()); let (mut kind, mut retryable) = match parts.status.as_u16() { 403 => (ErrorKind::PermissionDenied, false), diff --git a/core/src/services/s3/lister.rs b/core/src/services/s3/lister.rs index 1e1c25b1d702..2a86ae4c2d7d 100644 --- a/core/src/services/s3/lister.rs +++ b/core/src/services/s3/lister.rs @@ -81,7 +81,7 @@ impl oio::PageList for S3Lister { return Err(parse_error(resp).await?); } - let bs = resp.into_body().bytes().await?; + let bs = resp.into_body(); let output: ListObjectsOutput = de::from_reader(bs.reader()).map_err(new_xml_deserialize_error)?; diff --git a/core/src/services/s3/mod.rs b/core/src/services/s3/mod.rs index 4e2f283cb215..01a986b6bb14 100644 --- a/core/src/services/s3/mod.rs +++ b/core/src/services/s3/mod.rs @@ -22,4 +22,5 @@ pub use backend::S3Config; mod core; mod error; mod lister; +mod reader; mod writer; diff --git a/core/src/services/s3/reader.rs b/core/src/services/s3/reader.rs new file mode 100644 index 000000000000..3c74043be2f4 --- /dev/null +++ b/core/src/services/s3/reader.rs @@ -0,0 +1,55 @@ +// 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 super::core::*; +use super::error::*; +use crate::raw::*; +use crate::*; +use http::StatusCode; +use std::sync::Arc; + +pub struct S3Reader { + core: Arc, + + path: String, + op: OpRead, +} + +impl S3Reader { + pub fn new(core: Arc, path: &str, op: OpRead) -> Self { + S3Reader { + core, + path: path.to_string(), + op, + } + } +} + +impl oio::Read for S3Reader { + async fn read_at(&self, offset: u64, limit: usize) -> Result { + let range = BytesRange::new(Some(offset), Some(limit as u64)); + let resp = self.core.s3_get_object(&self.path, range, &self.op).await?; + + let status = resp.status(); + + match status { + StatusCode::OK | StatusCode::PARTIAL_CONTENT => Ok(resp.into_body()), + StatusCode::RANGE_NOT_SATISFIABLE => Ok(oio::Buffer::new()), + _ => Err(parse_error(resp).await?), + } + } +} diff --git a/core/src/services/s3/writer.rs b/core/src/services/s3/writer.rs index 0f6d8194566f..619101447fe4 100644 --- a/core/src/services/s3/writer.rs +++ b/core/src/services/s3/writer.rs @@ -15,6 +15,7 @@ // specific language governing permissions and limitations // under the License. +use bytes::Buf; use std::sync::Arc; use http::StatusCode; @@ -56,10 +57,7 @@ impl oio::MultipartWrite for S3Writer { let status = resp.status(); match status { - StatusCode::CREATED | StatusCode::OK => { - resp.into_body().consume().await?; - Ok(()) - } + StatusCode::CREATED | StatusCode::OK => Ok(()), _ => Err(parse_error(resp).await?), } } @@ -74,11 +72,10 @@ impl oio::MultipartWrite for S3Writer { match status { StatusCode::OK => { - let bs = resp.into_body().bytes().await?; + let bs = resp.into_body(); let result: InitiateMultipartUploadResult = - quick_xml::de::from_reader(bytes::Buf::reader(bs)) - .map_err(new_xml_deserialize_error)?; + quick_xml::de::from_reader(bs.reader()).map_err(new_xml_deserialize_error)?; Ok(result.upload_id) } @@ -117,8 +114,6 @@ impl oio::MultipartWrite for S3Writer { })? .to_string(); - resp.into_body().consume().await?; - Ok(oio::MultipartPart { part_number, etag }) } _ => Err(parse_error(resp).await?), @@ -142,11 +137,7 @@ impl oio::MultipartWrite for S3Writer { let status = resp.status(); match status { - StatusCode::OK => { - resp.into_body().consume().await?; - - Ok(()) - } + StatusCode::OK => Ok(()), _ => Err(parse_error(resp).await?), } } @@ -158,10 +149,7 @@ impl oio::MultipartWrite for S3Writer { .await?; match resp.status() { // s3 returns code 204 if abort succeeds. - StatusCode::NO_CONTENT => { - resp.into_body().consume().await?; - Ok(()) - } + StatusCode::NO_CONTENT => Ok(()), _ => Err(parse_error(resp).await?), } } From d06d49a85d9d03e6230fca96f6d45a26f51e163e Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Mon, 18 Mar 2024 18:05:06 +0800 Subject: [PATCH 005/111] Make happy Signed-off-by: Xuanwo --- core/src/services/s3/backend.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/services/s3/backend.rs b/core/src/services/s3/backend.rs index 89c3d8384bbc..05cf4edeec12 100644 --- a/core/src/services/s3/backend.rs +++ b/core/src/services/s3/backend.rs @@ -43,10 +43,10 @@ use super::core::*; use super::error::parse_error; use super::error::parse_s3_error_code; use super::lister::S3Lister; +use super::reader::S3Reader; use super::writer::S3Writer; use super::writer::S3Writers; use crate::raw::*; -use crate::services::s3::reader::S3Reader; use crate::*; /// Allow constructing correct region endpoint if user gives a global endpoint. From 55a4969cbbf5be63156013ca19f54966bf0e95c4 Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Mon, 18 Mar 2024 18:08:20 +0800 Subject: [PATCH 006/111] Save work Signed-off-by: Xuanwo --- core/src/services/alluxio/mod.rs | 1 + core/src/services/alluxio/reader.rs | 31 +++++++++++++++++++++++++++++ 2 files changed, 32 insertions(+) create mode 100644 core/src/services/alluxio/reader.rs diff --git a/core/src/services/alluxio/mod.rs b/core/src/services/alluxio/mod.rs index 95e0216d34b0..886a7807a3c2 100644 --- a/core/src/services/alluxio/mod.rs +++ b/core/src/services/alluxio/mod.rs @@ -23,3 +23,4 @@ mod core; mod error; mod lister; mod writer; +mod reader; diff --git a/core/src/services/alluxio/reader.rs b/core/src/services/alluxio/reader.rs new file mode 100644 index 000000000000..bc60d58c480d --- /dev/null +++ b/core/src/services/alluxio/reader.rs @@ -0,0 +1,31 @@ +// 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::sync::Arc; + +use bytes::Bytes; + +use crate::raw::*; +use crate::*; + +pub struct AlluxioReader { + core: Arc, + + _op: OpWrite, + path: String, + stream_id: Option, +} From 2a637c3b1843a021a7c9bad16606844412a406d1 Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Mon, 18 Mar 2024 19:36:47 +0800 Subject: [PATCH 007/111] Fix alluxio Signed-off-by: Xuanwo --- core/src/raw/http_util/bytes_range.rs | 21 +++++++++++++ core/src/services/alluxio/backend.rs | 9 +++--- core/src/services/alluxio/core.rs | 27 ++++++++-------- core/src/services/alluxio/error.rs | 6 ++-- core/src/services/alluxio/reader.rs | 36 +++++++++++++++++++--- core/src/services/cloudflare_kv/backend.rs | 4 +-- core/src/services/d1/backend.rs | 2 +- core/src/services/dropbox/core.rs | 2 +- core/src/services/gdrive/backend.rs | 2 +- core/src/services/gdrive/core.rs | 4 +-- core/src/services/github/core.rs | 2 +- core/src/services/icloud/core.rs | 8 ++--- core/src/services/s3/reader.rs | 4 ++- core/src/services/vercel_blob/core.rs | 2 +- core/src/services/yandex_disk/lister.rs | 2 +- 15 files changed, 89 insertions(+), 42 deletions(-) diff --git a/core/src/raw/http_util/bytes_range.rs b/core/src/raw/http_util/bytes_range.rs index 1cf3466b4823..9db8a95fb4ca 100644 --- a/core/src/raw/http_util/bytes_range.rs +++ b/core/src/raw/http_util/bytes_range.rs @@ -123,6 +123,27 @@ impl BytesRange { } } + /// Apply range on offset and limit. + /// + /// Returning `None ` + pub fn apply_on_offset(&self, offset: u64, limit: usize) -> Option { + match (self.offset(), self.size()) { + (Some(base), Some(size)) => { + if offset >= size { + None + } else { + Some(Self( + Some(base + offset), + Some((size - offset).min(limit as u64)), + )) + } + } + (Some(base), None) => Some(Self(Some(base + offset), Some(limit as u64))), + (None, None) => Some(Self(Some(offset), Some(limit as u64))), + (None, Some(size)) => unimplemented!(), + } + } + /// apply_on_bytes will apply range on bytes. pub fn apply_on_bytes(&self, mut bs: Bytes) -> Bytes { match (self.0, self.1) { diff --git a/core/src/services/alluxio/backend.rs b/core/src/services/alluxio/backend.rs index d14d82f0856a..65e55c7beaea 100644 --- a/core/src/services/alluxio/backend.rs +++ b/core/src/services/alluxio/backend.rs @@ -29,6 +29,7 @@ use super::lister::AlluxioLister; use super::writer::AlluxioWriter; use super::writer::AlluxioWriters; use crate::raw::*; +use crate::services::alluxio::reader::AlluxioReader; use crate::*; /// Config for alluxio services support. @@ -181,7 +182,7 @@ pub struct AlluxioBackend { #[async_trait] impl Accessor for AlluxioBackend { - type Reader = IncomingAsyncBody; + type Reader = AlluxioReader; type Writer = AlluxioWriters; type Lister = oio::PageLister; type BlockingReader = (); @@ -225,10 +226,8 @@ impl Accessor for AlluxioBackend { async fn read(&self, path: &str, args: OpRead) -> Result<(RpRead, Self::Reader)> { let stream_id = self.core.open_file(path).await?; - let resp = self.core.read(stream_id, args.range()).await?; - - let size = parse_content_length(resp.headers())?; - Ok((RpRead::new().with_size(size), resp.into_body())) + let r = AlluxioReader::new(self.core.clone(), path, args.clone(), stream_id); + Ok((RpRead::new(), r)) } async fn write(&self, path: &str, args: OpWrite) -> Result<(RpWrite, Self::Writer)> { diff --git a/core/src/services/alluxio/core.rs b/core/src/services/alluxio/core.rs index 111f35718631..53f0f3bc50ea 100644 --- a/core/src/services/alluxio/core.rs +++ b/core/src/services/alluxio/core.rs @@ -15,6 +15,7 @@ // specific language governing permissions and limitations // under the License. +use bytes::Buf; use std::fmt::Debug; use std::fmt::Formatter; @@ -155,9 +156,9 @@ impl AlluxioCore { match status { StatusCode::OK => { - let body = resp.into_body().bytes().await?; + let body = resp.into_body(); let steam_id: u64 = - serde_json::from_slice(&body).map_err(new_json_serialize_error)?; + serde_json::from_reader(body.reader()).map_err(new_json_serialize_error)?; Ok(steam_id) } _ => Err(parse_error(resp).await?), @@ -181,9 +182,9 @@ impl AlluxioCore { match status { StatusCode::OK => { - let body = resp.into_body().bytes().await?; + let body = resp.into_body(); let steam_id: u64 = - serde_json::from_slice(&body).map_err(new_json_serialize_error)?; + serde_json::from_reader(body.reader()).map_err(new_json_serialize_error)?; Ok(steam_id) } _ => Err(parse_error(resp).await?), @@ -261,9 +262,9 @@ impl AlluxioCore { match status { StatusCode::OK => { - let body = resp.into_body().bytes().await?; + let body = resp.into_body(); let file_info: FileInfo = - serde_json::from_slice(&body).map_err(new_json_serialize_error)?; + serde_json::from_reader(body.reader()).map_err(new_json_serialize_error)?; Ok(file_info) } _ => Err(parse_error(resp).await?), @@ -289,20 +290,16 @@ impl AlluxioCore { match status { StatusCode::OK => { - let body = resp.into_body().bytes().await?; + let body = resp.into_body(); let file_infos: Vec = - serde_json::from_slice(&body).map_err(new_json_deserialize_error)?; + serde_json::from_reader(body.reader()).map_err(new_json_deserialize_error)?; Ok(file_infos) } _ => Err(parse_error(resp).await?), } } - pub async fn read( - &self, - stream_id: u64, - range: BytesRange, - ) -> Result> { + pub async fn read(&self, stream_id: u64, range: BytesRange) -> Result> { let mut req = Request::post(format!( "{}/api/v1/streams/{}/read", self.endpoint, stream_id @@ -340,9 +337,9 @@ impl AlluxioCore { match status { StatusCode::OK => { - let body = resp.into_body().bytes().await?; + let body = resp.into_body(); let size: usize = - serde_json::from_slice(&body).map_err(new_json_serialize_error)?; + serde_json::from_reader(body.reader()).map_err(new_json_serialize_error)?; Ok(size) } _ => Err(parse_error(resp).await?), diff --git a/core/src/services/alluxio/error.rs b/core/src/services/alluxio/error.rs index d07fece3103c..be2e9df91877 100644 --- a/core/src/services/alluxio/error.rs +++ b/core/src/services/alluxio/error.rs @@ -31,9 +31,9 @@ struct AlluxioError { message: String, } -pub async fn parse_error(resp: Response) -> Result { - let (parts, body) = resp.into_parts(); - let bs = body.bytes().await?; +pub async fn parse_error(resp: Response) -> Result { + let (parts, mut body) = resp.into_parts(); + let bs = body.copy_to_bytes(body.remaining()); let mut kind = match parts.status.as_u16() { 500 => ErrorKind::Unexpected, diff --git a/core/src/services/alluxio/reader.rs b/core/src/services/alluxio/reader.rs index bc60d58c480d..a354bef2fee1 100644 --- a/core/src/services/alluxio/reader.rs +++ b/core/src/services/alluxio/reader.rs @@ -15,17 +15,45 @@ // specific language governing permissions and limitations // under the License. +use std::future::Future; use std::sync::Arc; -use bytes::Bytes; - +use super::core::*; +use crate::raw::oio::Buffer; use crate::raw::*; +use crate::services::alluxio::error::parse_error; use crate::*; pub struct AlluxioReader { core: Arc, - _op: OpWrite, + op: OpRead, path: String, - stream_id: Option, + stream_id: u64, +} + +impl AlluxioReader { + pub fn new(core: Arc, path: &str, op: OpRead, stream_id: u64) -> Self { + AlluxioReader { + core, + path: path.to_string(), + op, + stream_id, + } + } +} + +impl oio::Read for AlluxioReader { + async fn read_at(&self, offset: u64, limit: usize) -> Result { + let Some(range) = self.op.range().apply_on_offset(offset, limit) else { + return Ok(oio::Buffer::new()); + }; + + let resp = self.core.read(self.stream_id, range).await?; + + if !resp.status().is_success() { + return Err(parse_error(resp).await?); + } + Ok(resp.into_body()) + } } diff --git a/core/src/services/cloudflare_kv/backend.rs b/core/src/services/cloudflare_kv/backend.rs index 3050582c842d..075e7f6ac5e5 100644 --- a/core/src/services/cloudflare_kv/backend.rs +++ b/core/src/services/cloudflare_kv/backend.rs @@ -236,7 +236,7 @@ impl kv::Adapter for Adapter { let status = resp.status(); match status { StatusCode::OK => { - let body = resp.into_body().bytes().await?; + let body = resp.into_body(); Ok(Some(body.into())) } _ => Err(parse_error(resp).await?), @@ -291,7 +291,7 @@ impl kv::Adapter for Adapter { let status = resp.status(); match status { StatusCode::OK => { - let body = resp.into_body().bytes().await?; + let body = resp.into_body(); let response: CfKvScanResponse = serde_json::from_slice(&body).map_err(|e| { Error::new( ErrorKind::Unexpected, diff --git a/core/src/services/d1/backend.rs b/core/src/services/d1/backend.rs index 203ad4d2d931..ce16a6df7949 100644 --- a/core/src/services/d1/backend.rs +++ b/core/src/services/d1/backend.rs @@ -313,7 +313,7 @@ impl kv::Adapter for Adapter { let status = resp.status(); match status { StatusCode::OK | StatusCode::PARTIAL_CONTENT => { - let body = resp.into_body().bytes().await?; + let body = resp.into_body(); let d1_response = D1Response::parse(&body)?; Ok(d1_response.get_result(&self.value_field)) } diff --git a/core/src/services/dropbox/core.rs b/core/src/services/dropbox/core.rs index 67c7cf5e5778..5c2141388073 100644 --- a/core/src/services/dropbox/core.rs +++ b/core/src/services/dropbox/core.rs @@ -101,7 +101,7 @@ impl DropboxCore { .map_err(new_request_build_error)?; let resp = self.client.send(request).await?; - let body = resp.into_body().bytes().await?; + let body = resp.into_body(); let token: DropboxTokenResponse = serde_json::from_slice(&body).map_err(new_json_deserialize_error)?; diff --git a/core/src/services/gdrive/backend.rs b/core/src/services/gdrive/backend.rs index 671fa79e1c08..06a3d1097493 100644 --- a/core/src/services/gdrive/backend.rs +++ b/core/src/services/gdrive/backend.rs @@ -242,7 +242,7 @@ impl Accessor for GdriveBackend { match status { StatusCode::OK => { - let body = resp.into_body().bytes().await?; + let body = resp.into_body(); let meta = serde_json::from_slice::(&body) .map_err(new_json_deserialize_error)?; diff --git a/core/src/services/gdrive/core.rs b/core/src/services/gdrive/core.rs index 0f6211666d6a..52d52505929c 100644 --- a/core/src/services/gdrive/core.rs +++ b/core/src/services/gdrive/core.rs @@ -379,7 +379,7 @@ impl PathQuery for GdrivePathQuery { match status { StatusCode::OK => { - let body = resp.into_body().bytes().await?; + let body = resp.into_body(); let meta: GdriveFileList = serde_json::from_slice(&body).map_err(new_json_deserialize_error)?; @@ -416,7 +416,7 @@ impl PathQuery for GdrivePathQuery { return Err(parse_error(resp).await?); } - let body = resp.into_body().bytes().await?; + let body = resp.into_body(); let file: GdriveFile = serde_json::from_slice(&body).map_err(new_json_deserialize_error)?; Ok(file.id) } diff --git a/core/src/services/github/core.rs b/core/src/services/github/core.rs index 2e90f66e2698..9443b8ec6353 100644 --- a/core/src/services/github/core.rs +++ b/core/src/services/github/core.rs @@ -241,7 +241,7 @@ impl GithubCore { match resp.status() { StatusCode::OK => { - let body = resp.into_body().bytes().await?; + let body = resp.into_body(); let resp: ListResponse = serde_json::from_slice(&body).map_err(new_json_deserialize_error)?; diff --git a/core/src/services/icloud/core.rs b/core/src/services/icloud/core.rs index f98006563172..378c164217e1 100644 --- a/core/src/services/icloud/core.rs +++ b/core/src/services/icloud/core.rs @@ -359,7 +359,7 @@ impl IcloudCore { return Err(parse_error(resp).await?); } - let body = resp.into_body().bytes().await?; + let body = resp.into_body(); let drive_node: Vec = serde_json::from_slice(body.chunk()).map_err(new_json_deserialize_error)?; Ok(drive_node[0].clone()) @@ -389,7 +389,7 @@ impl IcloudCore { return Err(parse_error(resp).await?); } - let body = resp.into_body().bytes().await?; + let body = resp.into_body(); let object: IcloudObject = serde_json::from_slice(body.chunk()).map_err(new_json_deserialize_error)?; @@ -516,7 +516,7 @@ impl PathQuery for IcloudPathQuery { return Err(parse_error(resp).await?); } - let body = resp.into_body().bytes().await?; + let body = resp.into_body(); let root: Vec = serde_json::from_slice(body.chunk()).map_err(new_json_deserialize_error)?; @@ -557,7 +557,7 @@ impl PathQuery for IcloudPathQuery { return Err(parse_error(resp).await?); } - let body = resp.into_body().bytes().await?; + let body = resp.into_body(); let create_folder: IcloudCreateFolder = serde_json::from_slice(body.chunk()).map_err(new_json_deserialize_error)?; Ok(create_folder.destination_drivews_id) diff --git a/core/src/services/s3/reader.rs b/core/src/services/s3/reader.rs index 3c74043be2f4..d80b85cbb8cd 100644 --- a/core/src/services/s3/reader.rs +++ b/core/src/services/s3/reader.rs @@ -41,7 +41,9 @@ impl S3Reader { impl oio::Read for S3Reader { async fn read_at(&self, offset: u64, limit: usize) -> Result { - let range = BytesRange::new(Some(offset), Some(limit as u64)); + let Some(range) = self.op.range().apply_on_offset(offset, limit) else { + return Ok(oio::Buffer::new()); + }; let resp = self.core.s3_get_object(&self.path, range, &self.op).await?; let status = resp.status(); diff --git a/core/src/services/vercel_blob/core.rs b/core/src/services/vercel_blob/core.rs index 9fea54d9f9eb..e1c1b446d680 100644 --- a/core/src/services/vercel_blob/core.rs +++ b/core/src/services/vercel_blob/core.rs @@ -265,7 +265,7 @@ impl VercelBlobCore { return Err(parse_error(resp).await?); } - let body = resp.into_body().bytes().await?; + let body = resp.into_body(); let resp: ListResponse = serde_json::from_slice(&body).map_err(new_json_deserialize_error)?; diff --git a/core/src/services/yandex_disk/lister.rs b/core/src/services/yandex_disk/lister.rs index 9eaec4535778..f51d8da7966e 100644 --- a/core/src/services/yandex_disk/lister.rs +++ b/core/src/services/yandex_disk/lister.rs @@ -62,7 +62,7 @@ impl oio::PageList for YandexDiskLister { match resp.status() { http::StatusCode::OK => { - let body = resp.into_body().bytes().await?; + let body = resp.into_body(); let resp: MetainformationResponse = serde_json::from_slice(&body).map_err(new_json_deserialize_error)?; From 290cf7252cf28c93dee16cffdf2679198dd58c4f Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Mon, 18 Mar 2024 19:40:34 +0800 Subject: [PATCH 008/111] Fix atomicserver Signed-off-by: Xuanwo --- core/src/services/alluxio/reader.rs | 3 +-- core/src/services/atomicserver/backend.rs | 26 ++++++++++------------- 2 files changed, 12 insertions(+), 17 deletions(-) diff --git a/core/src/services/alluxio/reader.rs b/core/src/services/alluxio/reader.rs index a354bef2fee1..ac9a15cd6f80 100644 --- a/core/src/services/alluxio/reader.rs +++ b/core/src/services/alluxio/reader.rs @@ -19,7 +19,6 @@ use std::future::Future; use std::sync::Arc; use super::core::*; -use crate::raw::oio::Buffer; use crate::raw::*; use crate::services::alluxio::error::parse_error; use crate::*; @@ -44,7 +43,7 @@ impl AlluxioReader { } impl oio::Read for AlluxioReader { - async fn read_at(&self, offset: u64, limit: usize) -> Result { + async fn read_at(&self, offset: u64, limit: usize) -> Result { let Some(range) = self.op.range().apply_on_offset(offset, limit) else { return Ok(oio::Buffer::new()); }; diff --git a/core/src/services/atomicserver/backend.rs b/core/src/services/atomicserver/backend.rs index 0742cd9301fd..db2d54ed6fa2 100644 --- a/core/src/services/atomicserver/backend.rs +++ b/core/src/services/atomicserver/backend.rs @@ -23,7 +23,7 @@ use async_trait::async_trait; use atomic_lib::agents::Agent; use atomic_lib::client::get_authentication_headers; use atomic_lib::commit::sign_message; -use bytes::Bytes; +use bytes::{Buf, Bytes}; use http::header::CONTENT_DISPOSITION; use http::header::CONTENT_TYPE; use http::Request; @@ -375,9 +375,9 @@ impl Adapter { .body(AsyncBody::Empty) .map_err(new_request_build_error)?; let resp = self.client.send(req).await?; - let bytes_file = resp.into_body().bytes().await?; + let mut bytes_file = resp.into_body(); - Ok(bytes_file) + Ok(bytes_file.copy_to_bytes(bytes_file.remaining())) } } @@ -389,10 +389,9 @@ impl Adapter { for _i in 0..1000 { let req = self.atomic_get_object_request(path)?; let resp = self.client.send(req).await?; - let bytes = resp.into_body().bytes().await?; + let bytes = resp.into_body(); let query_result: QueryResultStruct = - serde_json::from_str(std::str::from_utf8(&bytes).unwrap()) - .map_err(new_json_deserialize_error)?; + serde_json::from_reader(bytes.reader()).map_err(new_json_deserialize_error)?; if !expect_exist && query_result.results.is_empty() { break; } @@ -424,11 +423,10 @@ impl kv::Adapter for Adapter { async fn get(&self, path: &str) -> Result>> { let req = self.atomic_get_object_request(path)?; let resp = self.client.send(req).await?; - let bytes = resp.into_body().bytes().await?; + let bytes = resp.into_body(); let query_result: QueryResultStruct = - serde_json::from_str(std::str::from_utf8(&bytes).unwrap()) - .map_err(new_json_deserialize_error)?; + serde_json::from_reader(bytes.reader()).map_err(new_json_deserialize_error)?; if query_result.results.is_empty() { return Err(Error::new( @@ -447,11 +445,10 @@ impl kv::Adapter for Adapter { async fn set(&self, path: &str, value: &[u8]) -> Result<()> { let req = self.atomic_get_object_request(path)?; let res = self.client.send(req).await?; - let bytes = res.into_body().bytes().await?; + let bytes = res.into_body(); let query_result: QueryResultStruct = - serde_json::from_str(std::str::from_utf8(&bytes).unwrap()) - .map_err(new_json_deserialize_error)?; + serde_json::from_reader(bytes.reader()).map_err(new_json_deserialize_error)?; for result in query_result.results { let req = self.atomic_delete_object_request(&result.id)?; @@ -470,11 +467,10 @@ impl kv::Adapter for Adapter { async fn delete(&self, path: &str) -> Result<()> { let req = self.atomic_get_object_request(path)?; let res = self.client.send(req).await?; - let bytes = res.into_body().bytes().await?; + let bytes = res.into_body(); let query_result: QueryResultStruct = - serde_json::from_str(std::str::from_utf8(&bytes).unwrap()) - .map_err(new_json_deserialize_error)?; + serde_json::from_reader(bytes.reader()).map_err(new_json_deserialize_error)?; for result in query_result.results { let req = self.atomic_delete_object_request(&result.id)?; From 73584752c98a6afbefa378278b14a87b780173cf Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Tue, 19 Mar 2024 17:57:02 +0800 Subject: [PATCH 009/111] Fix azblob Signed-off-by: Xuanwo --- core/src/raw/http_util/multipart.rs | 4 +- core/src/services/alluxio/error.rs | 2 +- core/src/services/azblob/backend.rs | 44 +++++-------- core/src/services/azblob/core.rs | 46 +++++-------- core/src/services/azblob/error.rs | 6 +- core/src/services/azblob/lister.rs | 2 +- core/src/services/azblob/mod.rs | 1 + core/src/services/azblob/reader.rs | 64 +++++++++++++++++++ core/src/services/azblob/writer.rs | 20 ++---- core/src/services/azdls/backend.rs | 19 ++---- core/src/services/azdls/core.rs | 18 ++---- core/src/services/azdls/error.rs | 4 +- core/src/services/azdls/lister.rs | 3 +- core/src/services/azdls/writer.rs | 18 ++---- core/src/services/azfile/backend.rs | 20 ++---- core/src/services/azfile/core.rs | 29 ++++----- core/src/services/azfile/error.rs | 4 +- core/src/services/azfile/lister.rs | 2 +- core/src/services/azfile/writer.rs | 14 +--- core/src/services/b2/backend.rs | 9 ++- core/src/services/b2/core.rs | 28 ++++---- core/src/services/b2/error.rs | 6 +- core/src/services/b2/lister.rs | 2 +- core/src/services/b2/writer.rs | 20 ++---- core/src/services/chainsafe/backend.rs | 4 +- core/src/services/chainsafe/core.rs | 20 +++--- core/src/services/chainsafe/error.rs | 6 +- core/src/services/chainsafe/lister.rs | 2 +- core/src/services/chainsafe/writer.rs | 5 +- core/src/services/cloudflare_kv/error.rs | 4 +- core/src/services/cos/backend.rs | 10 +-- core/src/services/cos/core.rs | 28 +++----- core/src/services/cos/error.rs | 4 +- core/src/services/cos/lister.rs | 2 +- core/src/services/cos/writer.rs | 20 ++---- core/src/services/d1/error.rs | 4 +- core/src/services/dbfs/backend.rs | 12 +--- core/src/services/dbfs/core.rs | 12 ++-- core/src/services/dbfs/error.rs | 4 +- core/src/services/dbfs/writer.rs | 5 +- core/src/services/dropbox/backend.rs | 11 ++-- core/src/services/dropbox/core.rs | 19 ++---- core/src/services/dropbox/error.rs | 4 +- core/src/services/dropbox/writer.rs | 5 +- core/src/services/gcs/backend.rs | 7 +- core/src/services/gcs/core.rs | 34 +++------- core/src/services/gcs/error.rs | 4 +- core/src/services/gcs/lister.rs | 2 +- core/src/services/gcs/writer.rs | 15 +---- core/src/services/gdrive/backend.rs | 8 +-- core/src/services/gdrive/core.rs | 16 ++--- core/src/services/gdrive/error.rs | 4 +- core/src/services/gdrive/writer.rs | 3 +- core/src/services/ghac/backend.rs | 14 ++-- core/src/services/ghac/error.rs | 4 +- core/src/services/ghac/writer.rs | 2 - core/src/services/github/backend.rs | 2 +- core/src/services/github/core.rs | 8 +-- core/src/services/github/error.rs | 6 +- core/src/services/github/writer.rs | 5 +- core/src/services/http/backend.rs | 9 ++- core/src/services/http/error.rs | 4 +- core/src/services/huggingface/backend.rs | 7 +- core/src/services/huggingface/core.rs | 10 +-- core/src/services/huggingface/error.rs | 4 +- core/src/services/icloud/backend.rs | 5 +- core/src/services/icloud/core.rs | 17 ++--- core/src/services/ipfs/backend.rs | 10 +-- core/src/services/ipfs/error.rs | 4 +- core/src/services/ipmfs/backend.rs | 34 +++------- core/src/services/ipmfs/error.rs | 4 +- core/src/services/ipmfs/lister.rs | 2 +- core/src/services/ipmfs/writer.rs | 5 +- core/src/services/koofr/backend.rs | 16 ++--- core/src/services/koofr/core.rs | 20 +++--- core/src/services/koofr/error.rs | 6 +- core/src/services/koofr/lister.rs | 2 +- core/src/services/koofr/writer.rs | 5 +- core/src/services/libsql/backend.rs | 2 +- core/src/services/libsql/error.rs | 4 +- core/src/services/obs/backend.rs | 10 +-- core/src/services/obs/core.rs | 28 +++----- core/src/services/obs/error.rs | 4 +- core/src/services/obs/lister.rs | 2 +- core/src/services/obs/writer.rs | 20 ++---- core/src/services/onedrive/backend.rs | 20 +++--- core/src/services/onedrive/error.rs | 4 +- core/src/services/onedrive/lister.rs | 2 +- core/src/services/onedrive/writer.rs | 11 +--- core/src/services/oss/backend.rs | 17 ++--- core/src/services/oss/core.rs | 31 ++++----- core/src/services/oss/error.rs | 4 +- core/src/services/oss/lister.rs | 2 +- core/src/services/oss/writer.rs | 20 ++---- core/src/services/pcloud/backend.rs | 10 +-- core/src/services/pcloud/core.rs | 31 ++++----- core/src/services/pcloud/error.rs | 10 +-- core/src/services/pcloud/lister.rs | 2 +- core/src/services/pcloud/writer.rs | 2 +- core/src/services/seafile/backend.rs | 2 +- core/src/services/seafile/core.rs | 16 ++--- core/src/services/seafile/error.rs | 6 +- core/src/services/seafile/lister.rs | 2 +- core/src/services/seafile/writer.rs | 5 +- core/src/services/supabase/backend.rs | 2 +- core/src/services/supabase/core.rs | 13 ++-- core/src/services/supabase/error.rs | 4 +- core/src/services/supabase/writer.rs | 5 +- core/src/services/swift/backend.rs | 10 +-- core/src/services/swift/core.rs | 16 ++--- core/src/services/swift/error.rs | 4 +- core/src/services/swift/writer.rs | 5 +- core/src/services/upyun/backend.rs | 14 +--- core/src/services/upyun/core.rs | 20 +++--- core/src/services/upyun/error.rs | 6 +- core/src/services/upyun/lister.rs | 2 +- core/src/services/upyun/writer.rs | 23 ++----- core/src/services/vercel_artifacts/backend.rs | 8 +-- core/src/services/vercel_artifacts/error.rs | 4 +- core/src/services/vercel_artifacts/writer.rs | 5 +- core/src/services/vercel_blob/backend.rs | 10 +-- core/src/services/vercel_blob/core.rs | 14 ++-- core/src/services/vercel_blob/error.rs | 6 +- core/src/services/vercel_blob/writer.rs | 15 ++--- core/src/services/webdav/backend.rs | 5 +- core/src/services/webdav/core.rs | 24 +++---- core/src/services/webdav/error.rs | 4 +- core/src/services/webdav/writer.rs | 5 +- core/src/services/webhdfs/backend.rs | 39 +++++------ core/src/services/webhdfs/error.rs | 6 +- core/src/services/webhdfs/lister.rs | 4 +- core/src/services/webhdfs/writer.rs | 26 ++------ core/src/services/yandex_disk/backend.rs | 16 ++--- core/src/services/yandex_disk/core.rs | 20 +++--- core/src/services/yandex_disk/error.rs | 6 +- core/src/services/yandex_disk/writer.rs | 5 +- 136 files changed, 582 insertions(+), 895 deletions(-) create mode 100644 core/src/services/azblob/reader.rs diff --git a/core/src/raw/http_util/multipart.rs b/core/src/raw/http_util/multipart.rs index 492d6a8b1b8f..30f2fc8f74d8 100644 --- a/core/src/raw/http_util/multipart.rs +++ b/core/src/raw/http_util/multipart.rs @@ -446,9 +446,9 @@ impl MixedPart { mem::swap(builder.headers_mut().unwrap(), &mut self.headers); // let body = if let Some(stream) = self.content { - // IncomingAsyncBody::new(stream, Some(self.content_length)) + // oio::Buffer::new(stream, Some(self.content_length)) // } else { - // IncomingAsyncBody::new(Box::new(oio::into_stream(stream::empty())), Some(0)) + // oio::Buffer::new(Box::new(oio::into_stream(stream::empty())), Some(0)) // }; // builder diff --git a/core/src/services/alluxio/error.rs b/core/src/services/alluxio/error.rs index be2e9df91877..1de357e80394 100644 --- a/core/src/services/alluxio/error.rs +++ b/core/src/services/alluxio/error.rs @@ -91,7 +91,7 @@ mod tests { for res in err_res { let bs = bytes::Bytes::from(res.0); - let body = IncomingAsyncBody::new( + let body = oio::Buffer::new( Box::new(oio::into_stream(stream::iter(vec![Ok(bs.clone())]))), None, ); diff --git a/core/src/services/azblob/backend.rs b/core/src/services/azblob/backend.rs index 7d6e71e98a74..b8fe029f2541 100644 --- a/core/src/services/azblob/backend.rs +++ b/core/src/services/azblob/backend.rs @@ -23,6 +23,7 @@ use std::sync::Arc; use async_trait::async_trait; use base64::prelude::BASE64_STANDARD; use base64::Engine; +use bytes::Buf; use http::header::CONTENT_TYPE; use http::StatusCode; use log::debug; @@ -38,6 +39,7 @@ use super::lister::AzblobLister; use super::writer::AzblobWriter; use crate::raw::*; use crate::services::azblob::core::AzblobCore; +use crate::services::azblob::reader::AzblobReader; use crate::services::azblob::writer::AzblobWriters; use crate::*; @@ -543,7 +545,7 @@ pub struct AzblobBackend { #[cfg_attr(not(target_arch = "wasm32"), async_trait)] #[cfg_attr(target_arch = "wasm32", async_trait(?Send))] impl Accessor for AzblobBackend { - type Reader = IncomingAsyncBody; + type Reader = AzblobReader; type Writer = AzblobWriters; type Lister = oio::PageLister; type BlockingReader = (); @@ -607,25 +609,10 @@ impl Accessor for AzblobBackend { } async fn read(&self, path: &str, args: OpRead) -> Result<(RpRead, Self::Reader)> { - let resp = self.core.azblob_get_blob(path, &args).await?; - - let status = resp.status(); - - match status { - StatusCode::OK | StatusCode::PARTIAL_CONTENT => { - let size = parse_content_length(resp.headers())?; - let range = parse_content_range(resp.headers())?; - Ok(( - RpRead::new().with_size(size).with_range(range), - resp.into_body(), - )) - } - StatusCode::RANGE_NOT_SATISFIABLE => { - resp.into_body().consume().await?; - Ok((RpRead::new().with_size(Some(0)), IncomingAsyncBody::empty())) - } - _ => Err(parse_error(resp).await?), - } + Ok(( + RpRead::default(), + AzblobReader::new(self.core.clone(), path, args), + )) } async fn write(&self, path: &str, args: OpWrite) -> Result<(RpWrite, Self::Writer)> { @@ -667,10 +654,7 @@ impl Accessor for AzblobBackend { let status = resp.status(); match status { - StatusCode::ACCEPTED => { - resp.into_body().consume().await?; - Ok(RpCopy::default()) - } + StatusCode::ACCEPTED => Ok(RpCopy::default()), _ => Err(parse_error(resp).await?), } } @@ -678,7 +662,10 @@ impl Accessor for AzblobBackend { async fn presign(&self, path: &str, args: OpPresign) -> Result { let mut req = match args.operation() { PresignOperation::Stat(v) => self.core.azblob_head_blob_request(path, v)?, - PresignOperation::Read(v) => self.core.azblob_get_blob_request(path, v)?, + PresignOperation::Read(v) => { + self.core + .azblob_get_blob_request(path, BytesRange::default(), v)? + } PresignOperation::Write(_) => self.core.azblob_put_blob_request( path, None, @@ -740,9 +727,10 @@ impl Accessor for AzblobBackend { ) })?; - let multipart: Multipart = Multipart::new() - .with_boundary(boundary) - .parse(resp.into_body().bytes().await?)?; + let mut bs = resp.into_body(); + let bs = bs.copy_to_bytes(bs.remaining()); + + let multipart: Multipart = Multipart::new().with_boundary(boundary).parse(bs)?; let parts = multipart.into_parts(); if paths.len() != parts.len() { diff --git a/core/src/services/azblob/core.rs b/core/src/services/azblob/core.rs index dcf474b18435..da23012f5d69 100644 --- a/core/src/services/azblob/core.rs +++ b/core/src/services/azblob/core.rs @@ -126,7 +126,7 @@ impl AzblobCore { } #[inline] - pub async fn send(&self, req: Request) -> Result> { + pub async fn send(&self, req: Request) -> Result> { self.client.send(req).await } @@ -163,7 +163,12 @@ impl AzblobCore { } impl AzblobCore { - pub fn azblob_get_blob_request(&self, path: &str, args: &OpRead) -> Result> { + pub fn azblob_get_blob_request( + &self, + path: &str, + range: BytesRange, + args: &OpRead, + ) -> Result> { let p = build_abs_path(&self.root, path); let mut url = format!( @@ -190,18 +195,7 @@ impl AzblobCore { // Set SSE headers. req = self.insert_sse_headers(req); - let range = args.range(); if !range.is_full() { - // azblob doesn't support read with suffix range. - // - // ref: https://learn.microsoft.com/en-us/rest/api/storageservices/specifying-the-range-header-for-blob-service-operations - if range.offset().is_none() && range.size().is_some() { - return Err(Error::new( - ErrorKind::Unsupported, - "azblob doesn't support read with suffix range", - )); - } - req = req.header(http::header::RANGE, range.to_header()); } @@ -223,9 +217,10 @@ impl AzblobCore { pub async fn azblob_get_blob( &self, path: &str, + range: BytesRange, args: &OpRead, - ) -> Result> { - let mut req = self.azblob_get_blob_request(path, args)?; + ) -> Result> { + let mut req = self.azblob_get_blob_request(path, range, args)?; self.sign(&mut req).await?; @@ -423,7 +418,7 @@ impl AzblobCore { size: Option, args: &OpWrite, body: AsyncBody, - ) -> Result> { + ) -> Result> { let mut req = self.azblob_put_block_request(path, block_id, size, args, body)?; self.sign(&mut req).await?; @@ -477,7 +472,7 @@ impl AzblobCore { path: &str, block_ids: Vec, args: &OpWrite, - ) -> Result> { + ) -> Result> { let mut req = self .azblob_complete_put_block_list_request(path, block_ids, args) .await?; @@ -525,7 +520,7 @@ impl AzblobCore { &self, path: &str, args: &OpStat, - ) -> Result> { + ) -> Result> { let mut req = self.azblob_head_blob_request(path, args)?; self.sign(&mut req).await?; @@ -549,18 +544,14 @@ impl AzblobCore { .map_err(new_request_build_error) } - pub async fn azblob_delete_blob(&self, path: &str) -> Result> { + pub async fn azblob_delete_blob(&self, path: &str) -> Result> { let mut req = self.azblob_delete_blob_request(path)?; self.sign(&mut req).await?; self.send(req).await } - pub async fn azblob_copy_blob( - &self, - from: &str, - to: &str, - ) -> Result> { + pub async fn azblob_copy_blob(&self, from: &str, to: &str) -> Result> { let source = build_abs_path(&self.root, from); let target = build_abs_path(&self.root, to); @@ -593,7 +584,7 @@ impl AzblobCore { next_marker: &str, delimiter: &str, limit: Option, - ) -> Result> { + ) -> Result> { let p = build_abs_path(&self.root, path); let mut url = format!( @@ -622,10 +613,7 @@ impl AzblobCore { self.send(req).await } - pub async fn azblob_batch_delete( - &self, - paths: &[String], - ) -> Result> { + pub async fn azblob_batch_delete(&self, paths: &[String]) -> Result> { let url = format!( "{}/{}?restype=container&comp=batch", self.endpoint, self.container diff --git a/core/src/services/azblob/error.rs b/core/src/services/azblob/error.rs index 58b744134b3d..22cbeb93be6c 100644 --- a/core/src/services/azblob/error.rs +++ b/core/src/services/azblob/error.rs @@ -61,9 +61,9 @@ impl Debug for AzblobError { } /// Parse error response into Error. -pub async fn parse_error(resp: Response) -> Result { - let (parts, body) = resp.into_parts(); - let bs = body.bytes().await?; +pub async fn parse_error(resp: Response) -> Result { + let (parts, mut body) = resp.into_parts(); + let bs = body.copy_to_bytes(body.remaining()); let (kind, retryable) = match parts.status { StatusCode::NOT_FOUND => (ErrorKind::NotFound, false), diff --git a/core/src/services/azblob/lister.rs b/core/src/services/azblob/lister.rs index 6bd1a61243ba..17b800be9cbb 100644 --- a/core/src/services/azblob/lister.rs +++ b/core/src/services/azblob/lister.rs @@ -58,7 +58,7 @@ impl oio::PageList for AzblobLister { return Err(parse_error(resp).await?); } - let bs = resp.into_body().bytes().await?; + let bs = resp.into_body(); let output: ListBlobsOutput = de::from_reader(bs.reader()).map_err(new_xml_deserialize_error)?; diff --git a/core/src/services/azblob/mod.rs b/core/src/services/azblob/mod.rs index da314f5b3ea8..bad56f999f97 100644 --- a/core/src/services/azblob/mod.rs +++ b/core/src/services/azblob/mod.rs @@ -22,4 +22,5 @@ pub use backend::AzblobConfig; mod core; mod error; mod lister; +mod reader; mod writer; diff --git a/core/src/services/azblob/reader.rs b/core/src/services/azblob/reader.rs new file mode 100644 index 000000000000..147765830d6f --- /dev/null +++ b/core/src/services/azblob/reader.rs @@ -0,0 +1,64 @@ +// 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::future::Future; +use std::sync::Arc; + +use http::StatusCode; + +use super::core::AzblobCore; +use super::error::parse_error; +use crate::raw::*; +use crate::*; + +pub struct AzblobReader { + core: Arc, + + path: String, + op: OpRead, +} + +impl AzblobReader { + pub fn new(core: Arc, path: &str, op: OpRead) -> Self { + AzblobReader { + core, + path: path.to_string(), + op, + } + } +} + +impl oio::Read for AzblobReader { + async fn read_at(&self, offset: u64, limit: usize) -> Result { + let Some(range) = self.op.range().apply_on_offset(offset, limit) else { + return Ok(oio::Buffer::new()); + }; + + let resp = self + .core + .azblob_get_blob(&self.path, range, &self.op) + .await?; + + let status = resp.status(); + + match status { + StatusCode::OK | StatusCode::PARTIAL_CONTENT => Ok(resp.into_body()), + StatusCode::RANGE_NOT_SATISFIABLE => Ok(oio::Buffer::new()), + _ => Err(parse_error(resp).await?), + } + } +} diff --git a/core/src/services/azblob/writer.rs b/core/src/services/azblob/writer.rs index 84025362d4a4..224dc078ae2e 100644 --- a/core/src/services/azblob/writer.rs +++ b/core/src/services/azblob/writer.rs @@ -99,10 +99,7 @@ impl oio::AppendWrite for AzblobWriter { let status = resp.status(); match status { - StatusCode::CREATED => { - resp.into_body().consume().await?; - Ok(()) - } + StatusCode::CREATED => Ok(()), _ => Err(parse_error(resp).await?), } } @@ -120,10 +117,7 @@ impl oio::BlockWrite for AzblobWriter { let status = resp.status(); match status { - StatusCode::CREATED | StatusCode::OK => { - resp.into_body().consume().await?; - Ok(()) - } + StatusCode::CREATED | StatusCode::OK => Ok(()), _ => Err(parse_error(resp).await?), } } @@ -136,10 +130,7 @@ impl oio::BlockWrite for AzblobWriter { let status = resp.status(); match status { - StatusCode::CREATED | StatusCode::OK => { - resp.into_body().consume().await?; - Ok(()) - } + StatusCode::CREATED | StatusCode::OK => Ok(()), _ => Err(parse_error(resp).await?), } } @@ -152,10 +143,7 @@ impl oio::BlockWrite for AzblobWriter { let status = resp.status(); match status { - StatusCode::CREATED | StatusCode::OK => { - resp.into_body().consume().await?; - Ok(()) - } + StatusCode::CREATED | StatusCode::OK => Ok(()), _ => Err(parse_error(resp).await?), } } diff --git a/core/src/services/azdls/backend.rs b/core/src/services/azdls/backend.rs index 4d6a36ed3b8c..710b6331ae63 100644 --- a/core/src/services/azdls/backend.rs +++ b/core/src/services/azdls/backend.rs @@ -245,7 +245,7 @@ pub struct AzdlsBackend { #[async_trait] impl Accessor for AzdlsBackend { - type Reader = IncomingAsyncBody; + type Reader = oio::Buffer; type Writer = AzdlsWriters; type Lister = oio::PageLister; type BlockingReader = (); @@ -293,10 +293,7 @@ impl Accessor for AzdlsBackend { let status = resp.status(); match status { - StatusCode::CREATED | StatusCode::OK => { - resp.into_body().consume().await?; - Ok(RpCreateDir::default()) - } + StatusCode::CREATED | StatusCode::OK => Ok(RpCreateDir::default()), _ => Err(parse_error(resp).await?), } } @@ -362,8 +359,7 @@ impl Accessor for AzdlsBackend { )) } StatusCode::RANGE_NOT_SATISFIABLE => { - resp.into_body().consume().await?; - Ok((RpRead::new().with_size(Some(0)), IncomingAsyncBody::empty())) + Ok((RpRead::new().with_size(Some(0)), oio::Buffer::empty())) } _ => Err(parse_error(resp).await?), } @@ -400,9 +396,7 @@ impl Accessor for AzdlsBackend { if let Some(resp) = self.core.azdls_ensure_parent_path(to).await? { let status = resp.status(); match status { - StatusCode::CREATED | StatusCode::CONFLICT => { - resp.into_body().consume().await?; - } + StatusCode::CREATED | StatusCode::CONFLICT => {} _ => return Err(parse_error(resp).await?), } } @@ -412,10 +406,7 @@ impl Accessor for AzdlsBackend { let status = resp.status(); match status { - StatusCode::CREATED => { - resp.into_body().consume().await?; - Ok(RpRename::default()) - } + StatusCode::CREATED => Ok(RpRename::default()), _ => Err(parse_error(resp).await?), } } diff --git a/core/src/services/azdls/core.rs b/core/src/services/azdls/core.rs index dce6e5bb80ae..d14544264e42 100644 --- a/core/src/services/azdls/core.rs +++ b/core/src/services/azdls/core.rs @@ -91,17 +91,13 @@ impl AzdlsCore { } #[inline] - pub async fn send(&self, req: Request) -> Result> { + pub async fn send(&self, req: Request) -> Result> { self.client.send(req).await } } impl AzdlsCore { - pub async fn azdls_read( - &self, - path: &str, - range: BytesRange, - ) -> Result> { + pub async fn azdls_read(&self, path: &str, range: BytesRange) -> Result> { let p = build_abs_path(&self.root, path); let url = format!( @@ -175,7 +171,7 @@ impl AzdlsCore { Ok(req) } - pub async fn azdls_rename(&self, from: &str, to: &str) -> Result> { + pub async fn azdls_rename(&self, from: &str, to: &str) -> Result> { let source = build_abs_path(&self.root, from); let target = build_abs_path(&self.root, to); @@ -231,7 +227,7 @@ impl AzdlsCore { Ok(req) } - pub async fn azdls_get_properties(&self, path: &str) -> Result> { + pub async fn azdls_get_properties(&self, path: &str) -> Result> { let p = build_abs_path(&self.root, path) .trim_end_matches('/') .to_string(); @@ -253,7 +249,7 @@ impl AzdlsCore { self.client.send(req).await } - pub async fn azdls_delete(&self, path: &str) -> Result> { + pub async fn azdls_delete(&self, path: &str) -> Result> { let p = build_abs_path(&self.root, path) .trim_end_matches('/') .to_string(); @@ -280,7 +276,7 @@ impl AzdlsCore { path: &str, continuation: &str, limit: Option, - ) -> Result> { + ) -> Result> { let p = build_abs_path(&self.root, path) .trim_end_matches('/') .to_string(); @@ -312,7 +308,7 @@ impl AzdlsCore { pub async fn azdls_ensure_parent_path( &self, path: &str, - ) -> Result>> { + ) -> Result>> { let abs_target_path = path.trim_end_matches('/').to_string(); let abs_target_path = abs_target_path.as_str(); let mut parts: Vec<&str> = abs_target_path diff --git a/core/src/services/azdls/error.rs b/core/src/services/azdls/error.rs index 04b839ce9f72..3e656e78eeb3 100644 --- a/core/src/services/azdls/error.rs +++ b/core/src/services/azdls/error.rs @@ -61,9 +61,9 @@ impl Debug for AzdlsError { } /// Parse error response into Error. -pub async fn parse_error(resp: Response) -> Result { +pub async fn parse_error(resp: Response) -> Result { let (parts, body) = resp.into_parts(); - let bs = body.bytes().await?; + let bs = body.copy_to_bytes(body.remaining()); let (kind, retryable) = match parts.status { StatusCode::NOT_FOUND => (ErrorKind::NotFound, false), diff --git a/core/src/services/azdls/lister.rs b/core/src/services/azdls/lister.rs index 3a74b27a5835..f76e68602129 100644 --- a/core/src/services/azdls/lister.rs +++ b/core/src/services/azdls/lister.rs @@ -47,7 +47,6 @@ impl oio::PageList for AzdlsLister { // azdls will return not found for not-exist path. if resp.status() == http::StatusCode::NOT_FOUND { - resp.into_body().consume().await?; ctx.done = true; return Ok(()); } @@ -68,7 +67,7 @@ impl oio::PageList for AzdlsLister { ctx.done = true; } - let bs = resp.into_body().bytes().await?; + let bs = resp.into_body(); let output: Output = de::from_slice(&bs).map_err(new_json_deserialize_error)?; diff --git a/core/src/services/azdls/writer.rs b/core/src/services/azdls/writer.rs index 6fc3709d04a1..7dd3638b9f03 100644 --- a/core/src/services/azdls/writer.rs +++ b/core/src/services/azdls/writer.rs @@ -53,9 +53,7 @@ impl oio::OneShotWrite for AzdlsWriter { let status = resp.status(); match status { - StatusCode::CREATED | StatusCode::OK => { - resp.into_body().consume().await?; - } + StatusCode::CREATED | StatusCode::OK => {} _ => { return Err(parse_error(resp) .await? @@ -76,10 +74,7 @@ impl oio::OneShotWrite for AzdlsWriter { let status = resp.status(); match status { - StatusCode::OK | StatusCode::ACCEPTED => { - resp.into_body().consume().await?; - Ok(()) - } + StatusCode::OK | StatusCode::ACCEPTED => Ok(()), _ => Err(parse_error(resp) .await? .with_operation("Backend::azdls_update_request")), @@ -113,9 +108,7 @@ impl oio::AppendWrite for AzdlsWriter { let status = resp.status(); match status { - StatusCode::CREATED | StatusCode::OK => { - resp.into_body().consume().await?; - } + StatusCode::CREATED | StatusCode::OK => {} _ => { return Err(parse_error(resp) .await? @@ -134,10 +127,7 @@ impl oio::AppendWrite for AzdlsWriter { let status = resp.status(); match status { - StatusCode::OK | StatusCode::ACCEPTED => { - resp.into_body().consume().await?; - Ok(()) - } + StatusCode::OK | StatusCode::ACCEPTED => Ok(()), _ => Err(parse_error(resp) .await? .with_operation("Backend::azdls_update_request")), diff --git a/core/src/services/azfile/backend.rs b/core/src/services/azfile/backend.rs index d11d7a700b8d..d6f2c1ad5924 100644 --- a/core/src/services/azfile/backend.rs +++ b/core/src/services/azfile/backend.rs @@ -266,7 +266,7 @@ pub struct AzfileBackend { #[async_trait] impl Accessor for AzfileBackend { - type Reader = IncomingAsyncBody; + type Reader = oio::Buffer; type Writer = AzfileWriters; type Lister = oio::PageLister; type BlockingReader = (); @@ -303,10 +303,7 @@ impl Accessor for AzfileBackend { let status = resp.status(); match status { - StatusCode::CREATED => { - resp.into_body().consume().await?; - Ok(RpCreateDir::default()) - } + StatusCode::CREATED => Ok(RpCreateDir::default()), _ => { // we cannot just check status code because 409 Conflict has two meaning: // 1. If a directory by the same name is being deleted when Create Directory is called, the server returns status code 409 (Conflict) @@ -360,8 +357,7 @@ impl Accessor for AzfileBackend { )) } StatusCode::RANGE_NOT_SATISFIABLE => { - resp.into_body().consume().await?; - Ok((RpRead::new().with_size(Some(0)), IncomingAsyncBody::empty())) + Ok((RpRead::new().with_size(Some(0)), oio::Buffer::empty())) } _ => Err(parse_error(resp).await?), } @@ -387,10 +383,7 @@ impl Accessor for AzfileBackend { let status = resp.status(); match status { - StatusCode::ACCEPTED | StatusCode::NOT_FOUND => { - resp.into_body().consume().await?; - Ok(RpDelete::default()) - } + StatusCode::ACCEPTED | StatusCode::NOT_FOUND => Ok(RpDelete::default()), _ => Err(parse_error(resp).await?), } } @@ -406,10 +399,7 @@ impl Accessor for AzfileBackend { let resp = self.core.azfile_rename(from, to).await?; let status = resp.status(); match status { - StatusCode::OK => { - resp.into_body().consume().await?; - Ok(RpRename::default()) - } + StatusCode::OK => Ok(RpRename::default()), _ => Err(parse_error(resp).await?), } } diff --git a/core/src/services/azfile/core.rs b/core/src/services/azfile/core.rs index ced2315ec5f7..1fa0814dc700 100644 --- a/core/src/services/azfile/core.rs +++ b/core/src/services/azfile/core.rs @@ -93,7 +93,7 @@ impl AzfileCore { } #[inline] - pub async fn send(&self, req: Request) -> Result> { + pub async fn send(&self, req: Request) -> Result> { self.client.send(req).await } @@ -101,7 +101,7 @@ impl AzfileCore { &self, path: &str, range: BytesRange, - ) -> Result> { + ) -> Result> { let p = build_abs_path(&self.root, path); let url = format!( @@ -139,7 +139,7 @@ impl AzfileCore { path: &str, size: usize, args: &OpWrite, - ) -> Result> { + ) -> Result> { let p = build_abs_path(&self.root, path) .trim_start_matches('/') .to_string(); @@ -182,7 +182,7 @@ impl AzfileCore { size: u64, position: u64, body: AsyncBody, - ) -> Result> { + ) -> Result> { let p = build_abs_path(&self.root, path) .trim_start_matches('/') .to_string(); @@ -210,10 +210,7 @@ impl AzfileCore { self.send(req).await } - pub async fn azfile_get_file_properties( - &self, - path: &str, - ) -> Result> { + pub async fn azfile_get_file_properties(&self, path: &str) -> Result> { let p = build_abs_path(&self.root, path); let url = format!( "{}/{}/{}", @@ -234,7 +231,7 @@ impl AzfileCore { pub async fn azfile_get_directory_properties( &self, path: &str, - ) -> Result> { + ) -> Result> { let p = build_abs_path(&self.root, path); let url = format!( @@ -253,11 +250,7 @@ impl AzfileCore { self.send(req).await } - pub async fn azfile_rename( - &self, - path: &str, - new_path: &str, - ) -> Result> { + pub async fn azfile_rename(&self, path: &str, new_path: &str) -> Result> { let p = build_abs_path(&self.root, path) .trim_start_matches('/') .to_string(); @@ -309,7 +302,7 @@ impl AzfileCore { self.send(req).await } - pub async fn azfile_create_dir(&self, path: &str) -> Result> { + pub async fn azfile_create_dir(&self, path: &str) -> Result> { let p = build_abs_path(&self.root, path) .trim_start_matches('/') .to_string(); @@ -332,7 +325,7 @@ impl AzfileCore { self.send(req).await } - pub async fn azfile_delete_file(&self, path: &str) -> Result> { + pub async fn azfile_delete_file(&self, path: &str) -> Result> { let p = build_abs_path(&self.root, path) .trim_start_matches('/') .to_string(); @@ -353,7 +346,7 @@ impl AzfileCore { self.send(req).await } - pub async fn azfile_delete_dir(&self, path: &str) -> Result> { + pub async fn azfile_delete_dir(&self, path: &str) -> Result> { let p = build_abs_path(&self.root, path) .trim_start_matches('/') .to_string(); @@ -379,7 +372,7 @@ impl AzfileCore { path: &str, limit: &Option, continuation: &String, - ) -> Result> { + ) -> Result> { let p = build_abs_path(&self.root, path) .trim_start_matches('/') .to_string(); diff --git a/core/src/services/azfile/error.rs b/core/src/services/azfile/error.rs index 026f5b3980f7..e5bd569897dc 100644 --- a/core/src/services/azfile/error.rs +++ b/core/src/services/azfile/error.rs @@ -61,9 +61,9 @@ impl Debug for AzfileError { } /// Parse error response into Error. -pub async fn parse_error(resp: Response) -> Result { +pub async fn parse_error(resp: Response) -> Result { let (parts, body) = resp.into_parts(); - let bs = body.bytes().await?; + let bs = body.copy_to_bytes(body.remaining()); let (kind, retryable) = match parts.status { StatusCode::NOT_FOUND => (ErrorKind::NotFound, false), diff --git a/core/src/services/azfile/lister.rs b/core/src/services/azfile/lister.rs index 9c3e6819c5da..ac65b123cbcd 100644 --- a/core/src/services/azfile/lister.rs +++ b/core/src/services/azfile/lister.rs @@ -55,7 +55,7 @@ impl oio::PageList for AzfileLister { return Err(parse_error(resp).await?); } - let bs = resp.into_body().bytes().await?; + let bs = resp.into_body(); let text = String::from_utf8(bs.to_vec()).expect("response convert to string must success"); diff --git a/core/src/services/azfile/writer.rs b/core/src/services/azfile/writer.rs index e2c3dde86eb6..0fe1ea28a8df 100644 --- a/core/src/services/azfile/writer.rs +++ b/core/src/services/azfile/writer.rs @@ -48,9 +48,7 @@ impl oio::OneShotWrite for AzfileWriter { let status = resp.status(); match status { - StatusCode::OK | StatusCode::CREATED => { - resp.into_body().consume().await?; - } + StatusCode::OK | StatusCode::CREATED => {} _ => { return Err(parse_error(resp) .await? @@ -64,10 +62,7 @@ impl oio::OneShotWrite for AzfileWriter { .await?; let status = resp.status(); match status { - StatusCode::OK | StatusCode::CREATED => { - resp.into_body().consume().await?; - Ok(()) - } + StatusCode::OK | StatusCode::CREATED => Ok(()), _ => Err(parse_error(resp) .await? .with_operation("Backend::azfile_update")), @@ -95,10 +90,7 @@ impl oio::AppendWrite for AzfileWriter { let status = resp.status(); match status { - StatusCode::OK | StatusCode::CREATED => { - resp.into_body().consume().await?; - Ok(()) - } + StatusCode::OK | StatusCode::CREATED => Ok(()), _ => Err(parse_error(resp) .await? .with_operation("Backend::azfile_update")), diff --git a/core/src/services/b2/backend.rs b/core/src/services/b2/backend.rs index 5d1ec9b5313a..167dbab111af 100644 --- a/core/src/services/b2/backend.rs +++ b/core/src/services/b2/backend.rs @@ -267,7 +267,7 @@ pub struct B2Backend { #[async_trait] impl Accessor for B2Backend { - type Reader = IncomingAsyncBody; + type Reader = oio::Buffer; type Writer = B2Writers; type Lister = oio::PageLister; type BlockingReader = (); @@ -339,7 +339,7 @@ impl Accessor for B2Backend { match status { StatusCode::OK => { - let bs = resp.into_body().bytes().await?; + let bs = resp.into_body(); let resp: ListFileNamesResponse = serde_json::from_slice(&bs).map_err(new_json_deserialize_error)?; @@ -368,8 +368,7 @@ impl Accessor for B2Backend { )) } StatusCode::RANGE_NOT_SATISFIABLE => { - resp.into_body().consume().await?; - Ok((RpRead::new().with_size(Some(0)), IncomingAsyncBody::empty())) + Ok((RpRead::new().with_size(Some(0)), oio::Buffer::empty())) } _ => Err(parse_error(resp).await?), } @@ -426,7 +425,7 @@ impl Accessor for B2Backend { let source_file_id = match status { StatusCode::OK => { - let bs = resp.into_body().bytes().await?; + let bs = resp.into_body(); let resp: ListFileNamesResponse = serde_json::from_slice(&bs).map_err(new_json_deserialize_error)?; diff --git a/core/src/services/b2/core.rs b/core/src/services/b2/core.rs index 20e24f11ac60..17c76513a8b2 100644 --- a/core/src/services/b2/core.rs +++ b/core/src/services/b2/core.rs @@ -70,7 +70,7 @@ impl Debug for B2Core { impl B2Core { #[inline] - pub async fn send(&self, req: Request) -> Result> { + pub async fn send(&self, req: Request) -> Result> { self.client.send(req).await } @@ -105,7 +105,7 @@ impl B2Core { match status { StatusCode::OK => { - let resp_body = &resp.into_body().bytes().await?; + let resp_body = &resp.into_body(); let token = serde_json::from_slice::(resp_body) .map_err(new_json_deserialize_error)?; signer.auth_info = AuthInfo { @@ -130,7 +130,7 @@ impl B2Core { &self, path: &str, args: &OpRead, - ) -> Result> { + ) -> Result> { let path = build_abs_path(&self.root, path); let auth_info = self.get_auth_info().await?; @@ -180,7 +180,7 @@ impl B2Core { let status = resp.status(); match status { StatusCode::OK => { - let resp_body = &resp.into_body().bytes().await?; + let resp_body = &resp.into_body(); let resp = serde_json::from_slice::(resp_body) .map_err(new_json_deserialize_error)?; Ok(resp) @@ -229,7 +229,7 @@ impl B2Core { let status = resp.status(); match status { StatusCode::OK => { - let resp_body = &resp.into_body().bytes().await?; + let resp_body = &resp.into_body(); let resp = serde_json::from_slice::(resp_body) .map_err(new_json_deserialize_error)?; Ok(resp) @@ -244,7 +244,7 @@ impl B2Core { size: Option, args: &OpWrite, body: AsyncBody, - ) -> Result> { + ) -> Result> { let resp = self.get_upload_url().await?; let p = build_abs_path(&self.root, path); @@ -281,7 +281,7 @@ impl B2Core { &self, path: &str, args: &OpWrite, - ) -> Result> { + ) -> Result> { let p = build_abs_path(&self.root, path); let auth_info = self.get_auth_info().await?; @@ -335,7 +335,7 @@ impl B2Core { let status = resp.status(); match status { StatusCode::OK => { - let resp_body = &resp.into_body().bytes().await?; + let resp_body = &resp.into_body(); let resp = serde_json::from_slice::(resp_body) .map_err(new_json_deserialize_error)?; Ok(resp) @@ -350,7 +350,7 @@ impl B2Core { part_number: usize, size: u64, body: AsyncBody, - ) -> Result> { + ) -> Result> { let resp = self.get_upload_part_url(file_id).await?; let mut req = Request::post(resp.upload_url); @@ -373,7 +373,7 @@ impl B2Core { &self, file_id: &str, part_sha1_array: Vec, - ) -> Result> { + ) -> Result> { let auth_info = self.get_auth_info().await?; let url = format!("{}/b2api/v2/b2_finish_large_file", auth_info.api_url); @@ -397,7 +397,7 @@ impl B2Core { self.send(req).await } - pub async fn cancel_large_file(&self, file_id: &str) -> Result> { + pub async fn cancel_large_file(&self, file_id: &str) -> Result> { let auth_info = self.get_auth_info().await?; let url = format!("{}/b2api/v2/b2_cancel_large_file", auth_info.api_url); @@ -426,7 +426,7 @@ impl B2Core { delimiter: Option<&str>, limit: Option, start_after: Option, - ) -> Result> { + ) -> Result> { let auth_info = self.get_auth_info().await?; let mut url = format!( @@ -470,7 +470,7 @@ impl B2Core { &self, source_file_id: String, to: &str, - ) -> Result> { + ) -> Result> { let to = build_abs_path(&self.root, to); let auth_info = self.get_auth_info().await?; @@ -497,7 +497,7 @@ impl B2Core { self.send(req).await } - pub async fn hide_file(&self, path: &str) -> Result> { + pub async fn hide_file(&self, path: &str) -> Result> { let path = build_abs_path(&self.root, path); let auth_info = self.get_auth_info().await?; diff --git a/core/src/services/b2/error.rs b/core/src/services/b2/error.rs index 24b8bb1839f7..80bde3761f44 100644 --- a/core/src/services/b2/error.rs +++ b/core/src/services/b2/error.rs @@ -34,9 +34,9 @@ struct B2Error { } /// Parse error response into Error. -pub async fn parse_error(resp: Response) -> Result { +pub async fn parse_error(resp: Response) -> Result { let (parts, body) = resp.into_parts(); - let bs = body.bytes().await?; + let bs = body.copy_to_bytes(body.remaining()); let (mut kind, mut retryable) = match parts.status.as_u16() { 403 => (ErrorKind::PermissionDenied, false), @@ -124,7 +124,7 @@ mod test { for res in err_res { let bs = bytes::Bytes::from(res.0); - let body = IncomingAsyncBody::new( + let body = oio::Buffer::new( Box::new(oio::into_stream(stream::iter(vec![Ok(bs.clone())]))), None, ); diff --git a/core/src/services/b2/lister.rs b/core/src/services/b2/lister.rs index 3d12ec6cf3e4..9b27b121c18b 100644 --- a/core/src/services/b2/lister.rs +++ b/core/src/services/b2/lister.rs @@ -79,7 +79,7 @@ impl oio::PageList for B2Lister { return Err(parse_error(resp).await?); } - let bs = resp.into_body().bytes().await?; + let bs = resp.into_body(); let output: ListFileNamesResponse = serde_json::from_reader(bs.reader()).map_err(new_json_deserialize_error)?; diff --git a/core/src/services/b2/writer.rs b/core/src/services/b2/writer.rs index 78cb8a9fc433..9452fd07d7e7 100644 --- a/core/src/services/b2/writer.rs +++ b/core/src/services/b2/writer.rs @@ -55,10 +55,7 @@ impl oio::MultipartWrite for B2Writer { let status = resp.status(); match status { - StatusCode::OK => { - resp.into_body().consume().await?; - Ok(()) - } + StatusCode::OK => Ok(()), _ => Err(parse_error(resp).await?), } } @@ -70,7 +67,7 @@ impl oio::MultipartWrite for B2Writer { match status { StatusCode::OK => { - let bs = resp.into_body().bytes().await?; + let bs = resp.into_body(); let result: StartLargeFileResponse = serde_json::from_slice(&bs).map_err(new_json_deserialize_error)?; @@ -100,7 +97,7 @@ impl oio::MultipartWrite for B2Writer { match status { StatusCode::OK => { - let bs = resp.into_body().bytes().await?; + let bs = resp.into_body(); let result: UploadPartResponse = serde_json::from_slice(&bs).map_err(new_json_deserialize_error)?; @@ -135,11 +132,7 @@ impl oio::MultipartWrite for B2Writer { let status = resp.status(); match status { - StatusCode::OK => { - resp.into_body().consume().await?; - - Ok(()) - } + StatusCode::OK => Ok(()), _ => Err(parse_error(resp).await?), } } @@ -148,10 +141,7 @@ impl oio::MultipartWrite for B2Writer { let resp = self.core.cancel_large_file(upload_id).await?; match resp.status() { // b2 returns code 200 if abort succeeds. - StatusCode::OK => { - resp.into_body().consume().await?; - Ok(()) - } + StatusCode::OK => Ok(()), _ => Err(parse_error(resp).await?), } } diff --git a/core/src/services/chainsafe/backend.rs b/core/src/services/chainsafe/backend.rs index 3957f4f900d6..d3acb0e70701 100644 --- a/core/src/services/chainsafe/backend.rs +++ b/core/src/services/chainsafe/backend.rs @@ -203,7 +203,7 @@ pub struct ChainsafeBackend { #[async_trait] impl Accessor for ChainsafeBackend { - type Reader = IncomingAsyncBody; + type Reader = oio::Buffer; type Writer = ChainsafeWriters; @@ -258,7 +258,7 @@ impl Accessor for ChainsafeBackend { match status { StatusCode::OK => { - let bs = resp.into_body().bytes().await?; + let bs = resp.into_body(); let output: ObjectInfoResponse = serde_json::from_slice(&bs).map_err(new_json_deserialize_error)?; diff --git a/core/src/services/chainsafe/core.rs b/core/src/services/chainsafe/core.rs index 50e3e370698e..c367dd96e65c 100644 --- a/core/src/services/chainsafe/core.rs +++ b/core/src/services/chainsafe/core.rs @@ -52,13 +52,13 @@ impl Debug for ChainsafeCore { impl ChainsafeCore { #[inline] - pub async fn send(&self, req: Request) -> Result> { + pub async fn send(&self, req: Request) -> Result> { self.client.send(req).await } } impl ChainsafeCore { - pub async fn download_object(&self, path: &str) -> Result> { + pub async fn download_object(&self, path: &str) -> Result> { let path = build_abs_path(&self.root, path); let url = format!( @@ -83,7 +83,7 @@ impl ChainsafeCore { self.send(req).await } - pub async fn object_info(&self, path: &str) -> Result> { + pub async fn object_info(&self, path: &str) -> Result> { let path = build_abs_path(&self.root, path); let url = format!( @@ -109,7 +109,7 @@ impl ChainsafeCore { self.send(req).await } - pub async fn move_object(&self, from: &str, to: &str) -> Result> { + pub async fn move_object(&self, from: &str, to: &str) -> Result> { let from = build_abs_path(&self.root, from); let to = build_abs_path(&self.root, to); @@ -136,7 +136,7 @@ impl ChainsafeCore { self.send(req).await } - pub async fn delete_object(&self, path: &str) -> Result> { + pub async fn delete_object(&self, path: &str) -> Result> { let path = build_abs_path(&self.root, path); let url = format!( @@ -162,11 +162,7 @@ impl ChainsafeCore { self.send(req).await } - pub async fn upload_object( - &self, - path: &str, - bs: Bytes, - ) -> Result> { + pub async fn upload_object(&self, path: &str, bs: Bytes) -> Result> { let path = build_abs_path(&self.root, path); let url = format!( @@ -190,7 +186,7 @@ impl ChainsafeCore { self.send(req).await } - pub async fn list_objects(&self, path: &str) -> Result> { + pub async fn list_objects(&self, path: &str) -> Result> { let path = build_abs_path(&self.root, path); let url = format!( @@ -216,7 +212,7 @@ impl ChainsafeCore { self.send(req).await } - pub async fn create_dir(&self, path: &str) -> Result> { + pub async fn create_dir(&self, path: &str) -> Result> { let path = build_abs_path(&self.root, path); let url = format!( diff --git a/core/src/services/chainsafe/error.rs b/core/src/services/chainsafe/error.rs index f998b625a4d9..aa54209baaa9 100644 --- a/core/src/services/chainsafe/error.rs +++ b/core/src/services/chainsafe/error.rs @@ -38,9 +38,9 @@ struct ChainsafeSubError { } /// Parse error response into Error. -pub async fn parse_error(resp: Response) -> Result { +pub async fn parse_error(resp: Response) -> Result { let (parts, body) = resp.into_parts(); - let bs = body.bytes().await?; + let bs = body.copy_to_bytes(body.remaining()); let (kind, retryable) = match parts.status.as_u16() { 401 | 403 => (ErrorKind::PermissionDenied, false), @@ -95,7 +95,7 @@ mod test { for res in err_res { let bs = bytes::Bytes::from(res.0); - let body = IncomingAsyncBody::new( + let body = oio::Buffer::new( Box::new(oio::into_stream(stream::iter(vec![Ok(bs.clone())]))), None, ); diff --git a/core/src/services/chainsafe/lister.rs b/core/src/services/chainsafe/lister.rs index faf00b6df70b..37a1a2dd4647 100644 --- a/core/src/services/chainsafe/lister.rs +++ b/core/src/services/chainsafe/lister.rs @@ -48,7 +48,7 @@ impl oio::PageList for ChainsafeLister { match resp.status() { StatusCode::OK => { - let bs = resp.into_body().bytes().await?; + let bs = resp.into_body(); let output: Vec = serde_json::from_slice(&bs).map_err(new_json_deserialize_error)?; diff --git a/core/src/services/chainsafe/writer.rs b/core/src/services/chainsafe/writer.rs index 79b58e324f0a..bf9547d5b147 100644 --- a/core/src/services/chainsafe/writer.rs +++ b/core/src/services/chainsafe/writer.rs @@ -50,10 +50,7 @@ impl oio::OneShotWrite for ChainsafeWriter { let status = resp.status(); match status { - StatusCode::OK => { - resp.into_body().consume().await?; - Ok(()) - } + StatusCode::OK => Ok(()), _ => Err(parse_error(resp).await?), } } diff --git a/core/src/services/cloudflare_kv/error.rs b/core/src/services/cloudflare_kv/error.rs index eb323f9ed2ce..0041adbe45cf 100644 --- a/core/src/services/cloudflare_kv/error.rs +++ b/core/src/services/cloudflare_kv/error.rs @@ -28,9 +28,9 @@ use crate::ErrorKind; use crate::Result; /// Parse error response into Error. -pub(crate) async fn parse_error(resp: Response) -> Result { +pub(crate) async fn parse_error(resp: Response) -> Result { let (parts, body) = resp.into_parts(); - let bs = body.bytes().await?; + let bs = body.copy_to_bytes(body.remaining()); let (mut kind, mut retryable) = match parts.status { StatusCode::NOT_FOUND => (ErrorKind::NotFound, false), diff --git a/core/src/services/cos/backend.rs b/core/src/services/cos/backend.rs index 48f0ca444d9d..b83922d1e1a1 100644 --- a/core/src/services/cos/backend.rs +++ b/core/src/services/cos/backend.rs @@ -254,7 +254,7 @@ pub struct CosBackend { #[async_trait] impl Accessor for CosBackend { - type Reader = IncomingAsyncBody; + type Reader = oio::Buffer; type Writer = CosWriters; type Lister = oio::PageLister; type BlockingReader = (); @@ -340,8 +340,7 @@ impl Accessor for CosBackend { )) } StatusCode::RANGE_NOT_SATISFIABLE => { - resp.into_body().consume().await?; - Ok((RpRead::new().with_size(Some(0)), IncomingAsyncBody::empty())) + Ok((RpRead::new().with_size(Some(0)), oio::Buffer::empty())) } _ => Err(parse_error(resp).await?), } @@ -383,10 +382,7 @@ impl Accessor for CosBackend { let status = resp.status(); match status { - StatusCode::OK => { - resp.into_body().consume().await?; - Ok(RpCopy::default()) - } + StatusCode::OK => Ok(RpCopy::default()), _ => Err(parse_error(resp).await?), } } diff --git a/core/src/services/cos/core.rs b/core/src/services/cos/core.rs index 29659cfc1c1c..208fc2721bd0 100644 --- a/core/src/services/cos/core.rs +++ b/core/src/services/cos/core.rs @@ -95,17 +95,13 @@ impl CosCore { } #[inline] - pub async fn send(&self, req: Request) -> Result> { + pub async fn send(&self, req: Request) -> Result> { self.client.send(req).await } } impl CosCore { - pub async fn cos_get_object( - &self, - path: &str, - args: &OpRead, - ) -> Result> { + pub async fn cos_get_object(&self, path: &str, args: &OpRead) -> Result> { let mut req = self.cos_get_object_request(path, args)?; self.sign(&mut req).await?; @@ -175,7 +171,7 @@ impl CosCore { &self, path: &str, args: &OpStat, - ) -> Result> { + ) -> Result> { let mut req = self.cos_head_object_request(path, args)?; self.sign(&mut req).await?; @@ -205,7 +201,7 @@ impl CosCore { Ok(req) } - pub async fn cos_delete_object(&self, path: &str) -> Result> { + pub async fn cos_delete_object(&self, path: &str) -> Result> { let p = build_abs_path(&self.root, path); let url = format!("{}/{}", self.endpoint, percent_encode_path(&p)); @@ -257,11 +253,7 @@ impl CosCore { Ok(req) } - pub async fn cos_copy_object( - &self, - from: &str, - to: &str, - ) -> Result> { + pub async fn cos_copy_object(&self, from: &str, to: &str) -> Result> { let source = build_abs_path(&self.root, from); let target = build_abs_path(&self.root, to); @@ -284,7 +276,7 @@ impl CosCore { next_marker: &str, delimiter: &str, limit: Option, - ) -> Result> { + ) -> Result> { let p = build_abs_path(&self.root, path); let mut queries = vec![]; @@ -320,7 +312,7 @@ impl CosCore { &self, path: &str, args: &OpWrite, - ) -> Result> { + ) -> Result> { let p = build_abs_path(&self.root, path); let url = format!("{}/{}?uploads", self.endpoint, percent_encode_path(&p)); @@ -355,7 +347,7 @@ impl CosCore { part_number: usize, size: u64, body: AsyncBody, - ) -> Result> { + ) -> Result> { let p = build_abs_path(&self.root, path); let url = format!( @@ -381,7 +373,7 @@ impl CosCore { path: &str, upload_id: &str, parts: Vec, - ) -> Result> { + ) -> Result> { let p = build_abs_path(&self.root, path); let url = format!( @@ -414,7 +406,7 @@ impl CosCore { &self, path: &str, upload_id: &str, - ) -> Result> { + ) -> Result> { let p = build_abs_path(&self.root, path); let url = format!( diff --git a/core/src/services/cos/error.rs b/core/src/services/cos/error.rs index e0d98920f66f..2f7c2ee670f6 100644 --- a/core/src/services/cos/error.rs +++ b/core/src/services/cos/error.rs @@ -38,9 +38,9 @@ struct CosError { } /// Parse error response into Error. -pub async fn parse_error(resp: Response) -> Result { +pub async fn parse_error(resp: Response) -> Result { let (parts, body) = resp.into_parts(); - let bs = body.bytes().await?; + let bs = body.copy_to_bytes(body.remaining()); let (kind, retryable) = match parts.status { StatusCode::NOT_FOUND => (ErrorKind::NotFound, false), diff --git a/core/src/services/cos/lister.rs b/core/src/services/cos/lister.rs index 098e243d8ba7..7022aa9def0b 100644 --- a/core/src/services/cos/lister.rs +++ b/core/src/services/cos/lister.rs @@ -57,7 +57,7 @@ impl oio::PageList for CosLister { return Err(parse_error(resp).await?); } - let bs = resp.into_body().bytes().await?; + let bs = resp.into_body(); let output: ListObjectsOutput = de::from_reader(bs.reader()).map_err(new_xml_deserialize_error)?; diff --git a/core/src/services/cos/writer.rs b/core/src/services/cos/writer.rs index 51f0ad15edf5..1b938bb4c51c 100644 --- a/core/src/services/cos/writer.rs +++ b/core/src/services/cos/writer.rs @@ -56,10 +56,7 @@ impl oio::MultipartWrite for CosWriter { let status = resp.status(); match status { - StatusCode::CREATED | StatusCode::OK => { - resp.into_body().consume().await?; - Ok(()) - } + StatusCode::CREATED | StatusCode::OK => Ok(()), _ => Err(parse_error(resp).await?), } } @@ -74,7 +71,7 @@ impl oio::MultipartWrite for CosWriter { match status { StatusCode::OK => { - let bs = resp.into_body().bytes().await?; + let bs = resp.into_body(); let result: InitiateMultipartUploadResult = quick_xml::de::from_reader(bytes::Buf::reader(bs)) @@ -114,8 +111,6 @@ impl oio::MultipartWrite for CosWriter { })? .to_string(); - resp.into_body().consume().await?; - Ok(oio::MultipartPart { part_number, etag }) } _ => Err(parse_error(resp).await?), @@ -139,11 +134,7 @@ impl oio::MultipartWrite for CosWriter { let status = resp.status(); match status { - StatusCode::OK => { - resp.into_body().consume().await?; - - Ok(()) - } + StatusCode::OK => Ok(()), _ => Err(parse_error(resp).await?), } } @@ -156,10 +147,7 @@ impl oio::MultipartWrite for CosWriter { match resp.status() { // cos returns code 204 if abort succeeds. // Reference: https://www.tencentcloud.com/document/product/436/7740 - StatusCode::NO_CONTENT => { - resp.into_body().consume().await?; - Ok(()) - } + StatusCode::NO_CONTENT => Ok(()), _ => Err(parse_error(resp).await?), } } diff --git a/core/src/services/d1/error.rs b/core/src/services/d1/error.rs index 96ac072e004e..98c774381611 100644 --- a/core/src/services/d1/error.rs +++ b/core/src/services/d1/error.rs @@ -28,9 +28,9 @@ use crate::ErrorKind; use crate::Result; /// Parse error response into Error. -pub async fn parse_error(resp: Response) -> Result { +pub async fn parse_error(resp: Response) -> Result { let (parts, body) = resp.into_parts(); - let bs = body.bytes().await?; + let bs = body.copy_to_bytes(body.remaining()); let (mut kind, mut retryable) = match parts.status { StatusCode::NOT_FOUND => (ErrorKind::NotFound, false), diff --git a/core/src/services/dbfs/backend.rs b/core/src/services/dbfs/backend.rs index 1028c8b9dea3..59d54ebffb76 100644 --- a/core/src/services/dbfs/backend.rs +++ b/core/src/services/dbfs/backend.rs @@ -203,10 +203,7 @@ impl Accessor for DbfsBackend { let status = resp.status(); match status { - StatusCode::CREATED | StatusCode::OK => { - resp.into_body().consume().await?; - Ok(RpCreateDir::default()) - } + StatusCode::CREATED | StatusCode::OK => Ok(RpCreateDir::default()), _ => Err(parse_error(resp).await?), } } @@ -224,7 +221,7 @@ impl Accessor for DbfsBackend { match status { StatusCode::OK => { let mut meta = parse_into_metadata(path, resp.headers())?; - let bs = resp.into_body().bytes().await?; + let bs = resp.into_body(); let decoded_response = serde_json::from_slice::(&bs) .map_err(new_json_deserialize_error)?; meta.set_last_modified(parse_datetime_from_from_timestamp_millis( @@ -285,10 +282,7 @@ impl Accessor for DbfsBackend { let status = resp.status(); match status { - StatusCode::OK => { - resp.into_body().consume().await?; - Ok(RpRename::default()) - } + StatusCode::OK => Ok(RpRename::default()), _ => Err(parse_error(resp).await?), } } diff --git a/core/src/services/dbfs/core.rs b/core/src/services/dbfs/core.rs index 80541d31e46c..55e4516f1bfa 100644 --- a/core/src/services/dbfs/core.rs +++ b/core/src/services/dbfs/core.rs @@ -48,7 +48,7 @@ impl Debug for DbfsCore { } impl DbfsCore { - pub async fn dbfs_create_dir(&self, path: &str) -> Result> { + pub async fn dbfs_create_dir(&self, path: &str) -> Result> { let url = format!("{}/api/2.0/dbfs/mkdirs", self.endpoint); let mut req = Request::post(&url); @@ -69,7 +69,7 @@ impl DbfsCore { self.client.send(req).await } - pub async fn dbfs_delete(&self, path: &str) -> Result> { + pub async fn dbfs_delete(&self, path: &str) -> Result> { let url = format!("{}/api/2.0/dbfs/delete", self.endpoint); let mut req = Request::post(&url); @@ -93,7 +93,7 @@ impl DbfsCore { self.client.send(req).await } - pub async fn dbfs_rename(&self, from: &str, to: &str) -> Result> { + pub async fn dbfs_rename(&self, from: &str, to: &str) -> Result> { let source = build_rooted_abs_path(&self.root, from); let target = build_rooted_abs_path(&self.root, to); @@ -115,7 +115,7 @@ impl DbfsCore { self.client.send(req).await } - pub async fn dbfs_list(&self, path: &str) -> Result> { + pub async fn dbfs_list(&self, path: &str) -> Result> { let p = build_rooted_abs_path(&self.root, path) .trim_end_matches('/') .to_string(); @@ -162,7 +162,7 @@ impl DbfsCore { path: &str, offset: u64, length: u64, - ) -> Result> { + ) -> Result> { let p = build_rooted_abs_path(&self.root, path) .trim_end_matches('/') .to_string(); @@ -200,7 +200,7 @@ impl DbfsCore { } } - pub async fn dbfs_get_status(&self, path: &str) -> Result> { + pub async fn dbfs_get_status(&self, path: &str) -> Result> { let p = build_rooted_abs_path(&self.root, path) .trim_end_matches('/') .to_string(); diff --git a/core/src/services/dbfs/error.rs b/core/src/services/dbfs/error.rs index 41069d45fb0d..6b5b8fac0a0b 100644 --- a/core/src/services/dbfs/error.rs +++ b/core/src/services/dbfs/error.rs @@ -44,9 +44,9 @@ impl Debug for DbfsError { } } -pub async fn parse_error(resp: Response) -> Result { +pub async fn parse_error(resp: Response) -> Result { let (parts, body) = resp.into_parts(); - let bs = body.bytes().await?; + let bs = body.copy_to_bytes(body.remaining()); let (kind, retryable) = match parts.status { StatusCode::NOT_FOUND => (ErrorKind::NotFound, false), diff --git a/core/src/services/dbfs/writer.rs b/core/src/services/dbfs/writer.rs index 47442901335c..625dc552e269 100644 --- a/core/src/services/dbfs/writer.rs +++ b/core/src/services/dbfs/writer.rs @@ -57,10 +57,7 @@ impl oio::OneShotWrite for DbfsWriter { let status = resp.status(); match status { - StatusCode::CREATED | StatusCode::OK => { - resp.into_body().consume().await?; - Ok(()) - } + StatusCode::CREATED | StatusCode::OK => Ok(()), _ => Err(parse_error(resp).await?), } } diff --git a/core/src/services/dropbox/backend.rs b/core/src/services/dropbox/backend.rs index 5d05aaf5f2f8..0cbe9be64e9f 100644 --- a/core/src/services/dropbox/backend.rs +++ b/core/src/services/dropbox/backend.rs @@ -35,7 +35,7 @@ pub struct DropboxBackend { #[async_trait] impl Accessor for DropboxBackend { - type Reader = IncomingAsyncBody; + type Reader = oio::Buffer; type Writer = oio::OneShotWriter; type Lister = (); type BlockingReader = (); @@ -70,7 +70,7 @@ impl Accessor for DropboxBackend { // Check if the folder already exists. let resp = self.core.dropbox_get_metadata(path).await?; if StatusCode::OK == resp.status() { - let bytes = resp.into_body().bytes().await?; + let bytes = resp.into_body(); let decoded_response = serde_json::from_slice::(&bytes) .map_err(new_json_deserialize_error)?; if "folder" == decoded_response.tag { @@ -102,7 +102,7 @@ impl Accessor for DropboxBackend { let status = resp.status(); match status { StatusCode::OK => { - let bytes = resp.into_body().bytes().await?; + let bytes = resp.into_body(); let decoded_response = serde_json::from_slice::(&bytes) .map_err(new_json_deserialize_error)?; let entry_mode: EntryMode = match decoded_response.tag.as_str() { @@ -141,8 +141,7 @@ impl Accessor for DropboxBackend { match status { StatusCode::OK | StatusCode::PARTIAL_CONTENT => Ok((RpRead::new(), resp.into_body())), StatusCode::RANGE_NOT_SATISFIABLE => { - resp.into_body().consume().await?; - Ok((RpRead::new().with_size(Some(0)), IncomingAsyncBody::empty())) + Ok((RpRead::new().with_size(Some(0)), oio::Buffer::empty())) } _ => Err(parse_error(resp).await?), } @@ -193,7 +192,7 @@ impl Accessor for DropboxBackend { return Err(parse_error(resp).await?); } - let bs = resp.into_body().bytes().await?; + let bs = resp.into_body(); let decoded_response = serde_json::from_slice::(&bs) .map_err(new_json_deserialize_error)?; diff --git a/core/src/services/dropbox/core.rs b/core/src/services/dropbox/core.rs index 5c2141388073..054f630ef510 100644 --- a/core/src/services/dropbox/core.rs +++ b/core/src/services/dropbox/core.rs @@ -121,11 +121,7 @@ impl DropboxCore { Ok(()) } - pub async fn dropbox_get( - &self, - path: &str, - args: OpRead, - ) -> Result> { + pub async fn dropbox_get(&self, path: &str, args: OpRead) -> Result> { let url: String = "https://content.dropboxapi.com/2/files/download".to_string(); let download_args = DropboxDownloadArgs { path: build_rooted_abs_path(&self.root, path), @@ -156,7 +152,7 @@ impl DropboxCore { size: Option, args: &OpWrite, body: AsyncBody, - ) -> Result> { + ) -> Result> { let url = "https://content.dropboxapi.com/2/files/upload".to_string(); let dropbox_update_args = DropboxUploadArgs { path: build_rooted_abs_path(&self.root, path), @@ -183,7 +179,7 @@ impl DropboxCore { self.client.send(request).await } - pub async fn dropbox_delete(&self, path: &str) -> Result> { + pub async fn dropbox_delete(&self, path: &str) -> Result> { let url = "https://api.dropboxapi.com/2/files/delete_v2".to_string(); let args = DropboxDeleteArgs { path: self.build_path(path), @@ -201,10 +197,7 @@ impl DropboxCore { self.client.send(request).await } - pub async fn dropbox_delete_batch( - &self, - paths: Vec, - ) -> Result> { + pub async fn dropbox_delete_batch(&self, paths: Vec) -> Result> { let url = "https://api.dropboxapi.com/2/files/delete_batch".to_string(); let args = DropboxDeleteBatchArgs { entries: paths @@ -246,7 +239,7 @@ impl DropboxCore { return Err(parse_error(resp).await?); } - let bs = resp.into_body().bytes().await?; + let bs = resp.into_body(); let decoded_response = serde_json::from_slice::(&bs) .map_err(new_json_deserialize_error)?; @@ -300,7 +293,7 @@ impl DropboxCore { } } - pub async fn dropbox_get_metadata(&self, path: &str) -> Result> { + pub async fn dropbox_get_metadata(&self, path: &str) -> Result> { let url = "https://api.dropboxapi.com/2/files/get_metadata".to_string(); let args = DropboxMetadataArgs { path: self.build_path(path), diff --git a/core/src/services/dropbox/error.rs b/core/src/services/dropbox/error.rs index ae731df785a8..fe69da739671 100644 --- a/core/src/services/dropbox/error.rs +++ b/core/src/services/dropbox/error.rs @@ -31,9 +31,9 @@ pub struct DropboxErrorResponse { } /// Parse error response into Error. -pub async fn parse_error(resp: Response) -> Result { +pub async fn parse_error(resp: Response) -> Result { let (parts, body) = resp.into_parts(); - let bs = body.bytes().await?; + let bs = body.copy_to_bytes(body.remaining()); let (mut kind, mut retryable) = match parts.status { StatusCode::NOT_FOUND => (ErrorKind::NotFound, false), diff --git a/core/src/services/dropbox/writer.rs b/core/src/services/dropbox/writer.rs index de1f594cd6b0..e14ea6bc247a 100644 --- a/core/src/services/dropbox/writer.rs +++ b/core/src/services/dropbox/writer.rs @@ -45,10 +45,7 @@ impl oio::OneShotWrite for DropboxWriter { .await?; let status = resp.status(); match status { - StatusCode::OK => { - resp.into_body().consume().await?; - Ok(()) - } + StatusCode::OK => Ok(()), _ => Err(parse_error(resp).await?), } } diff --git a/core/src/services/gcs/backend.rs b/core/src/services/gcs/backend.rs index 899b5efb3361..2c2ac070f928 100644 --- a/core/src/services/gcs/backend.rs +++ b/core/src/services/gcs/backend.rs @@ -332,7 +332,7 @@ pub struct GcsBackend { #[cfg_attr(not(target_arch = "wasm32"), async_trait)] #[cfg_attr(target_arch = "wasm32", async_trait(?Send))] impl Accessor for GcsBackend { - type Reader = IncomingAsyncBody; + type Reader = oio::Buffer; type Writer = GcsWriters; type Lister = oio::PageLister; type BlockingReader = (); @@ -393,7 +393,7 @@ impl Accessor for GcsBackend { return Err(parse_error(resp).await?); } - let slc = resp.into_body().bytes().await?; + let slc = resp.into_body(); let meta: GetObjectJsonResponse = serde_json::from_slice(&slc).map_err(new_json_deserialize_error)?; @@ -424,7 +424,7 @@ impl Accessor for GcsBackend { let size = parse_content_length(resp.headers())?; Ok((RpRead::new().with_size(size), resp.into_body())) } else if resp.status() == StatusCode::RANGE_NOT_SATISFIABLE { - Ok((RpRead::new(), IncomingAsyncBody::empty())) + Ok((RpRead::new(), oio::Buffer::empty())) } else { Err(parse_error(resp).await?) } @@ -465,7 +465,6 @@ impl Accessor for GcsBackend { let resp = self.core.gcs_copy_object(from, to).await?; if resp.status().is_success() { - resp.into_body().consume().await?; Ok(RpCopy::default()) } else { Err(parse_error(resp).await?) diff --git a/core/src/services/gcs/core.rs b/core/src/services/gcs/core.rs index 46cc2c4123be..01d9f8d204fb 100644 --- a/core/src/services/gcs/core.rs +++ b/core/src/services/gcs/core.rs @@ -141,7 +141,7 @@ impl GcsCore { } #[inline] - pub async fn send(&self, req: Request) -> Result> { + pub async fn send(&self, req: Request) -> Result> { self.client.send(req).await } } @@ -205,11 +205,7 @@ impl GcsCore { Ok(req) } - pub async fn gcs_get_object( - &self, - path: &str, - args: &OpRead, - ) -> Result> { + pub async fn gcs_get_object(&self, path: &str, args: &OpRead) -> Result> { let mut req = self.gcs_get_object_request(path, args)?; self.sign(&mut req).await?; @@ -388,7 +384,7 @@ impl GcsCore { &self, path: &str, args: &OpStat, - ) -> Result> { + ) -> Result> { let mut req = self.gcs_head_object_request(path, args)?; self.sign(&mut req).await?; @@ -396,7 +392,7 @@ impl GcsCore { self.send(req).await } - pub async fn gcs_delete_object(&self, path: &str) -> Result> { + pub async fn gcs_delete_object(&self, path: &str) -> Result> { let mut req = self.gcs_delete_object_request(path)?; self.sign(&mut req).await?; @@ -418,10 +414,7 @@ impl GcsCore { .map_err(new_request_build_error) } - pub async fn gcs_delete_objects( - &self, - paths: Vec, - ) -> Result> { + pub async fn gcs_delete_objects(&self, paths: Vec) -> Result> { let uri = format!("{}/batch/storage/v1", self.endpoint); let mut multipart = Multipart::new(); @@ -441,11 +434,7 @@ impl GcsCore { self.send(req).await } - pub async fn gcs_copy_object( - &self, - from: &str, - to: &str, - ) -> Result> { + pub async fn gcs_copy_object(&self, from: &str, to: &str) -> Result> { let source = build_abs_path(&self.root, from); let dest = build_abs_path(&self.root, to); @@ -474,7 +463,7 @@ impl GcsCore { delimiter: &str, limit: Option, start_after: Option, - ) -> Result> { + ) -> Result> { let p = build_abs_path(&self.root, path); let mut url = format!( @@ -515,10 +504,7 @@ impl GcsCore { self.send(req).await } - pub async fn gcs_initiate_resumable_upload( - &self, - path: &str, - ) -> Result> { + pub async fn gcs_initiate_resumable_upload(&self, path: &str) -> Result> { let p = build_abs_path(&self.root, path); let url = format!( "{}/upload/storage/v1/b/{}/o?uploadType=resumable&name={}", @@ -561,7 +547,7 @@ impl GcsCore { written: u64, size: u64, body: AsyncBody, - ) -> Result> { + ) -> Result> { let mut req = Request::post(location) .header(CONTENT_LENGTH, size) .header( @@ -584,7 +570,7 @@ impl GcsCore { pub async fn gcs_abort_resumable_upload( &self, location: &str, - ) -> Result> { + ) -> Result> { let mut req = Request::delete(location) .header(CONTENT_LENGTH, 0) .body(AsyncBody::Empty) diff --git a/core/src/services/gcs/error.rs b/core/src/services/gcs/error.rs index 2c5665ffd4cf..74a6649741dc 100644 --- a/core/src/services/gcs/error.rs +++ b/core/src/services/gcs/error.rs @@ -50,9 +50,9 @@ struct GcsErrorDetail { } /// Parse error response into Error. -pub async fn parse_error(resp: Response) -> Result { +pub async fn parse_error(resp: Response) -> Result { let (parts, body) = resp.into_parts(); - let bs = body.bytes().await?; + let bs = body.copy_to_bytes(body.remaining()); let (kind, retryable) = match parts.status { StatusCode::NOT_FOUND => (ErrorKind::NotFound, false), diff --git a/core/src/services/gcs/lister.rs b/core/src/services/gcs/lister.rs index f18412c74bc4..31fbfb3894a4 100644 --- a/core/src/services/gcs/lister.rs +++ b/core/src/services/gcs/lister.rs @@ -79,7 +79,7 @@ impl oio::PageList for GcsLister { if !resp.status().is_success() { return Err(parse_error(resp).await?); } - let bytes = resp.into_body().bytes().await?; + let bytes = resp.into_body(); let output: ListResponse = serde_json::from_slice(&bytes).map_err(new_json_deserialize_error)?; diff --git a/core/src/services/gcs/writer.rs b/core/src/services/gcs/writer.rs index 0bf13a4498c3..9e3c3a54d6b0 100644 --- a/core/src/services/gcs/writer.rs +++ b/core/src/services/gcs/writer.rs @@ -58,10 +58,7 @@ impl oio::RangeWrite for GcsWriter { let status = resp.status(); match status { - StatusCode::CREATED | StatusCode::OK => { - resp.into_body().consume().await?; - Ok(()) - } + StatusCode::CREATED | StatusCode::OK => Ok(()), _ => Err(parse_error(resp).await?), } } @@ -122,10 +119,7 @@ impl oio::RangeWrite for GcsWriter { let status = resp.status(); match status { - StatusCode::OK => { - resp.into_body().consume().await?; - Ok(()) - } + StatusCode::OK => Ok(()), _ => Err(parse_error(resp).await?), } } @@ -136,10 +130,7 @@ impl oio::RangeWrite for GcsWriter { match resp.status().as_u16() { // gcs returns 499 if the upload aborted successfully // reference: https://cloud.google.com/storage/docs/performing-resumable-uploads#cancel-upload-json - 499 => { - resp.into_body().consume().await?; - Ok(()) - } + 499 => Ok(()), _ => Err(parse_error(resp).await?), } } diff --git a/core/src/services/gdrive/backend.rs b/core/src/services/gdrive/backend.rs index 06a3d1097493..d65b5f4978d1 100644 --- a/core/src/services/gdrive/backend.rs +++ b/core/src/services/gdrive/backend.rs @@ -42,7 +42,7 @@ pub struct GdriveBackend { #[cfg_attr(not(target_arch = "wasm32"), async_trait)] #[cfg_attr(target_arch = "wasm32", async_trait(?Send))] impl Accessor for GdriveBackend { - type Reader = IncomingAsyncBody; + type Reader = oio::Buffer; type Writer = oio::OneShotWriter; type Lister = oio::PageLister; type BlockingReader = (); @@ -86,7 +86,7 @@ impl Accessor for GdriveBackend { return Err(parse_error(resp).await?); } - let bs = resp.into_body().bytes().await?; + let bs = resp.into_body(); let gdrive_file: GdriveFile = serde_json::from_slice(&bs).map_err(new_json_deserialize_error)?; @@ -155,7 +155,7 @@ impl Accessor for GdriveBackend { } self.core.path_cache.remove(&path).await; - resp.into_body().consume().await?; + return Ok(RpDelete::default()); } @@ -190,7 +190,6 @@ impl Accessor for GdriveBackend { } self.core.path_cache.remove(&to_path).await; - resp.into_body().consume().await?; } let url = format!( @@ -230,7 +229,6 @@ impl Accessor for GdriveBackend { } self.core.path_cache.remove(&target).await; - resp.into_body().consume().await?; } let resp = self diff --git a/core/src/services/gdrive/core.rs b/core/src/services/gdrive/core.rs index 52d52505929c..e2aa317acedc 100644 --- a/core/src/services/gdrive/core.rs +++ b/core/src/services/gdrive/core.rs @@ -58,7 +58,7 @@ impl Debug for GdriveCore { } impl GdriveCore { - pub async fn gdrive_stat(&self, path: &str) -> Result> { + pub async fn gdrive_stat(&self, path: &str) -> Result> { let path = build_abs_path(&self.root, path); let file_id = self.path_cache.get(&path).await?.ok_or(Error::new( ErrorKind::NotFound, @@ -78,7 +78,7 @@ impl GdriveCore { self.client.send(req).await } - pub async fn gdrive_get(&self, path: &str) -> Result> { + pub async fn gdrive_get(&self, path: &str) -> Result> { let path = build_abs_path(&self.root, path); let path_id = self.path_cache.get(&path).await?.ok_or(Error::new( ErrorKind::NotFound, @@ -103,7 +103,7 @@ impl GdriveCore { file_id: &str, page_size: i32, next_page_token: &str, - ) -> Result> { + ) -> Result> { let q = format!("'{}' in parents and trashed = false", file_id); let mut url = format!( "https://www.googleapis.com/drive/v3/files?pageSize={}&q={}", @@ -127,7 +127,7 @@ impl GdriveCore { &self, source: &str, target: &str, - ) -> Result> { + ) -> Result> { let source_file_id = self.path_cache.get(source).await?.ok_or(Error::new( ErrorKind::NotFound, &format!("source path not found: {}", source), @@ -161,7 +161,7 @@ impl GdriveCore { self.client.send(req).await } - pub async fn gdrive_trash(&self, file_id: &str) -> Result> { + pub async fn gdrive_trash(&self, file_id: &str) -> Result> { let url = format!("https://www.googleapis.com/drive/v3/files/{}", file_id); let body = serde_json::to_vec(&json!({ @@ -184,7 +184,7 @@ impl GdriveCore { path: &str, size: u64, body: Bytes, - ) -> Result> { + ) -> Result> { let parent = self.path_cache.ensure_dir(get_parent(path)).await?; let url = "https://www.googleapis.com/upload/drive/v3/files?uploadType=multipart"; @@ -234,7 +234,7 @@ impl GdriveCore { file_id: &str, size: u64, body: Bytes, - ) -> Result> { + ) -> Result> { let url = format!( "https://www.googleapis.com/upload/drive/v3/files/{}?uploadType=media", file_id @@ -310,7 +310,7 @@ impl GdriveSigner { match status { StatusCode::OK => { - let resp_body = &resp.into_body().bytes().await?; + let resp_body = &resp.into_body(); let token = serde_json::from_slice::(resp_body) .map_err(new_json_deserialize_error)?; self.access_token = token.access_token.clone(); diff --git a/core/src/services/gdrive/error.rs b/core/src/services/gdrive/error.rs index 8b7ad781aed7..5fdd634a5233 100644 --- a/core/src/services/gdrive/error.rs +++ b/core/src/services/gdrive/error.rs @@ -35,9 +35,9 @@ struct GdriveInnerError { } /// Parse error response into Error. -pub async fn parse_error(resp: Response) -> Result { +pub async fn parse_error(resp: Response) -> Result { let (parts, body) = resp.into_parts(); - let bs = body.bytes().await?; + let bs = body.copy_to_bytes(body.remaining()); let (mut kind, mut retryable) = match parts.status { StatusCode::NOT_FOUND => (ErrorKind::NotFound, false), diff --git a/core/src/services/gdrive/writer.rs b/core/src/services/gdrive/writer.rs index 0ebbd809803b..b9a80b2789d0 100644 --- a/core/src/services/gdrive/writer.rs +++ b/core/src/services/gdrive/writer.rs @@ -64,12 +64,11 @@ impl oio::OneShotWrite for GdriveWriter { StatusCode::OK | StatusCode::CREATED => { // If we don't have the file id before, let's update the cache to avoid re-fetching. if self.file_id.is_none() { - let bs = resp.into_body().bytes().await?; + let bs = resp.into_body(); let file: GdriveFile = serde_json::from_slice(&bs).map_err(new_json_deserialize_error)?; self.core.path_cache.insert(&self.path, &file.id).await; } else { - resp.into_body().consume().await?; } Ok(()) } diff --git a/core/src/services/ghac/backend.rs b/core/src/services/ghac/backend.rs index 6319ae7f43a8..09c29988fd8d 100644 --- a/core/src/services/ghac/backend.rs +++ b/core/src/services/ghac/backend.rs @@ -227,7 +227,7 @@ pub struct GhacBackend { #[async_trait] impl Accessor for GhacBackend { - type Reader = IncomingAsyncBody; + type Reader = oio::Buffer; type Writer = GhacWriter; type Lister = (); type BlockingReader = (); @@ -266,7 +266,7 @@ impl Accessor for GhacBackend { let resp = self.client.send(req).await?; let location = if resp.status() == StatusCode::OK { - let slc = resp.into_body().bytes().await?; + let slc = resp.into_body(); let query_resp: GhacQueryResponse = serde_json::from_slice(&slc).map_err(new_json_deserialize_error)?; query_resp.archive_location @@ -292,7 +292,6 @@ impl Accessor for GhacBackend { .expect("content range must contains size"), ); - resp.into_body().consume().await?; Ok(RpStat::new(meta)) } _ => Err(parse_error(resp).await?), @@ -305,7 +304,7 @@ impl Accessor for GhacBackend { let resp = self.client.send(req).await?; let location = if resp.status() == StatusCode::OK { - let slc = resp.into_body().bytes().await?; + let slc = resp.into_body(); let query_resp: GhacQueryResponse = serde_json::from_slice(&slc).map_err(new_json_deserialize_error)?; query_resp.archive_location @@ -327,8 +326,7 @@ impl Accessor for GhacBackend { )) } StatusCode::RANGE_NOT_SATISFIABLE => { - resp.into_body().consume().await?; - Ok((RpRead::new().with_size(Some(0)), IncomingAsyncBody::empty())) + Ok((RpRead::new().with_size(Some(0)), oio::Buffer::empty())) } _ => Err(parse_error(resp).await?), } @@ -340,7 +338,7 @@ impl Accessor for GhacBackend { let resp = self.client.send(req).await?; let cache_id = if resp.status().is_success() { - let slc = resp.into_body().bytes().await?; + let slc = resp.into_body(); let reserve_resp: GhacReserveResponse = serde_json::from_slice(&slc).map_err(new_json_deserialize_error)?; reserve_resp.cache_id @@ -488,7 +486,7 @@ impl GhacBackend { Ok(req) } - async fn ghac_delete(&self, path: &str) -> Result> { + async fn ghac_delete(&self, path: &str) -> Result> { let p = build_abs_path(&self.root, path); let url = format!( diff --git a/core/src/services/ghac/error.rs b/core/src/services/ghac/error.rs index 598822f1821e..dae4c334edcd 100644 --- a/core/src/services/ghac/error.rs +++ b/core/src/services/ghac/error.rs @@ -24,7 +24,7 @@ use crate::ErrorKind; use crate::Result; /// Parse error response into Error. -pub async fn parse_error(resp: Response) -> Result { +pub async fn parse_error(resp: Response) -> Result { let (parts, body) = resp.into_parts(); let (kind, retryable) = match parts.status { @@ -39,7 +39,7 @@ pub async fn parse_error(resp: Response) -> Result { _ => (ErrorKind::Unexpected, false), }; - let bs = body.bytes().await?; + let bs = body.copy_to_bytes(body.remaining()); let message = String::from_utf8_lossy(&bs); let mut err = Error::new(kind, &message); diff --git a/core/src/services/ghac/writer.rs b/core/src/services/ghac/writer.rs index 3db0ea597975..6e92cd6aa990 100644 --- a/core/src/services/ghac/writer.rs +++ b/core/src/services/ghac/writer.rs @@ -57,7 +57,6 @@ impl oio::Write for GhacWriter { .map(|err| err.with_operation("Backend::ghac_upload"))?); } - resp.into_body().consume().await?; self.size += size as u64; Ok(size) } @@ -71,7 +70,6 @@ impl oio::Write for GhacWriter { let resp = self.backend.client.send(req).await?; if resp.status().is_success() { - resp.into_body().consume().await?; Ok(()) } else { Err(parse_error(resp) diff --git a/core/src/services/github/backend.rs b/core/src/services/github/backend.rs index 997e2ae92f03..796e345e8b3a 100644 --- a/core/src/services/github/backend.rs +++ b/core/src/services/github/backend.rs @@ -221,7 +221,7 @@ pub struct GithubBackend { #[async_trait] impl Accessor for GithubBackend { - type Reader = IncomingAsyncBody; + type Reader = oio::Buffer; type Writer = GithubWriters; diff --git a/core/src/services/github/core.rs b/core/src/services/github/core.rs index 9443b8ec6353..454a4c7d1dd9 100644 --- a/core/src/services/github/core.rs +++ b/core/src/services/github/core.rs @@ -60,7 +60,7 @@ impl Debug for GithubCore { impl GithubCore { #[inline] - pub async fn send(&self, req: Request) -> Result> { + pub async fn send(&self, req: Request) -> Result> { self.client.send(req).await } @@ -100,7 +100,7 @@ impl GithubCore { } } - pub async fn stat(&self, path: &str) -> Result> { + pub async fn stat(&self, path: &str) -> Result> { let path = build_abs_path(&self.root, path); let url = format!( @@ -122,7 +122,7 @@ impl GithubCore { self.send(req).await } - pub async fn get(&self, path: &str) -> Result> { + pub async fn get(&self, path: &str) -> Result> { let path = build_abs_path(&self.root, path); let url = format!( @@ -144,7 +144,7 @@ impl GithubCore { self.send(req).await } - pub async fn upload(&self, path: &str, bs: Bytes) -> Result> { + pub async fn upload(&self, path: &str, bs: Bytes) -> Result> { let sha = self.get_file_sha(path).await?; let path = build_abs_path(&self.root, path); diff --git a/core/src/services/github/error.rs b/core/src/services/github/error.rs index edd52ce70786..4fcedf851013 100644 --- a/core/src/services/github/error.rs +++ b/core/src/services/github/error.rs @@ -38,9 +38,9 @@ struct GithubSubError { } /// Parse error response into Error. -pub async fn parse_error(resp: Response) -> Result { +pub async fn parse_error(resp: Response) -> Result { let (parts, body) = resp.into_parts(); - let bs = body.bytes().await?; + let bs = body.copy_to_bytes(body.remaining()); let (kind, retryable) = match parts.status.as_u16() { 401 | 403 => (ErrorKind::PermissionDenied, false), @@ -93,7 +93,7 @@ mod test { for res in err_res { let bs = bytes::Bytes::from(res.0); - let body = IncomingAsyncBody::new( + let body = oio::Buffer::new( Box::new(oio::into_stream(stream::iter(vec![Ok(bs.clone())]))), None, ); diff --git a/core/src/services/github/writer.rs b/core/src/services/github/writer.rs index 0ff42415b113..5b6d3f6c1fb7 100644 --- a/core/src/services/github/writer.rs +++ b/core/src/services/github/writer.rs @@ -45,10 +45,7 @@ impl oio::OneShotWrite for GithubWriter { let status = resp.status(); match status { - StatusCode::OK | StatusCode::CREATED => { - resp.into_body().consume().await?; - Ok(()) - } + StatusCode::OK | StatusCode::CREATED => Ok(()), _ => Err(parse_error(resp).await?), } } diff --git a/core/src/services/http/backend.rs b/core/src/services/http/backend.rs index daf704735432..92293b9c2b5c 100644 --- a/core/src/services/http/backend.rs +++ b/core/src/services/http/backend.rs @@ -223,7 +223,7 @@ impl Debug for HttpBackend { #[async_trait] impl Accessor for HttpBackend { - type Reader = IncomingAsyncBody; + type Reader = oio::Buffer; type Writer = (); type Lister = (); type BlockingReader = (); @@ -287,8 +287,7 @@ impl Accessor for HttpBackend { )) } StatusCode::RANGE_NOT_SATISFIABLE => { - resp.into_body().consume().await?; - Ok((RpRead::new().with_size(Some(0)), IncomingAsyncBody::empty())) + Ok((RpRead::new().with_size(Some(0)), oio::Buffer::empty())) } _ => Err(parse_error(resp).await?), } @@ -296,7 +295,7 @@ impl Accessor for HttpBackend { } impl HttpBackend { - async fn http_get(&self, path: &str, args: &OpRead) -> Result> { + async fn http_get(&self, path: &str, args: &OpRead) -> Result> { let p = build_rooted_abs_path(&self.root, path); let url = format!("{}{}", self.endpoint, percent_encode_path(&p)); @@ -326,7 +325,7 @@ impl HttpBackend { self.client.send(req).await } - async fn http_head(&self, path: &str, args: &OpStat) -> Result> { + async fn http_head(&self, path: &str, args: &OpStat) -> Result> { let p = build_rooted_abs_path(&self.root, path); let url = format!("{}{}", self.endpoint, percent_encode_path(&p)); diff --git a/core/src/services/http/error.rs b/core/src/services/http/error.rs index c8afa09a963f..96386a946525 100644 --- a/core/src/services/http/error.rs +++ b/core/src/services/http/error.rs @@ -24,9 +24,9 @@ use crate::ErrorKind; use crate::Result; /// Parse error response into Error. -pub async fn parse_error(resp: Response) -> Result { +pub async fn parse_error(resp: Response) -> Result { let (parts, body) = resp.into_parts(); - let bs = body.bytes().await?; + let bs = body.copy_to_bytes(body.remaining()); let (kind, retryable) = match parts.status { StatusCode::NOT_FOUND => (ErrorKind::NotFound, false), diff --git a/core/src/services/huggingface/backend.rs b/core/src/services/huggingface/backend.rs index defbc7b6086e..3f9187efff17 100644 --- a/core/src/services/huggingface/backend.rs +++ b/core/src/services/huggingface/backend.rs @@ -243,7 +243,7 @@ pub struct HuggingfaceBackend { #[async_trait] impl Accessor for HuggingfaceBackend { - type Reader = IncomingAsyncBody; + type Reader = oio::Buffer; type Writer = (); type Lister = oio::PageLister; type BlockingReader = (); @@ -281,7 +281,7 @@ impl Accessor for HuggingfaceBackend { match status { StatusCode::OK => { let mut meta = parse_into_metadata(path, resp.headers())?; - let bs = resp.into_body().bytes().await?; + let bs = resp.into_body(); let decoded_response = serde_json::from_slice::>(&bs) .map_err(new_json_deserialize_error)?; @@ -326,8 +326,7 @@ impl Accessor for HuggingfaceBackend { )) } StatusCode::RANGE_NOT_SATISFIABLE => { - resp.into_body().consume().await?; - Ok((RpRead::new().with_size(Some(0)), IncomingAsyncBody::empty())) + Ok((RpRead::new().with_size(Some(0)), oio::Buffer::empty())) } _ => Err(parse_error(resp).await?), } diff --git a/core/src/services/huggingface/core.rs b/core/src/services/huggingface/core.rs index 768b416006d6..f7aab77acec3 100644 --- a/core/src/services/huggingface/core.rs +++ b/core/src/services/huggingface/core.rs @@ -49,7 +49,7 @@ impl Debug for HuggingfaceCore { } impl HuggingfaceCore { - pub async fn hf_path_info(&self, path: &str) -> Result> { + pub async fn hf_path_info(&self, path: &str) -> Result> { let p = build_abs_path(&self.root, path) .trim_end_matches('/') .to_string(); @@ -83,11 +83,7 @@ impl HuggingfaceCore { self.client.send(req).await } - pub async fn hf_list( - &self, - path: &str, - recursive: bool, - ) -> Result> { + pub async fn hf_list(&self, path: &str, recursive: bool) -> Result> { let p = build_abs_path(&self.root, path) .trim_end_matches('/') .to_string(); @@ -125,7 +121,7 @@ impl HuggingfaceCore { self.client.send(req).await } - pub async fn hf_resolve(&self, path: &str, arg: OpRead) -> Result> { + pub async fn hf_resolve(&self, path: &str, arg: OpRead) -> Result> { let p = build_abs_path(&self.root, path) .trim_end_matches('/') .to_string(); diff --git a/core/src/services/huggingface/error.rs b/core/src/services/huggingface/error.rs index 4e5361ece07d..0057b9b3e33c 100644 --- a/core/src/services/huggingface/error.rs +++ b/core/src/services/huggingface/error.rs @@ -39,9 +39,9 @@ impl Debug for HuggingfaceError { } } -pub async fn parse_error(resp: Response) -> Result { +pub async fn parse_error(resp: Response) -> Result { let (parts, body) = resp.into_parts(); - let bs = body.bytes().await?; + let bs = body.copy_to_bytes(body.remaining()); let (kind, retryable) = match parts.status { StatusCode::NOT_FOUND => (ErrorKind::NotFound, false), diff --git a/core/src/services/icloud/backend.rs b/core/src/services/icloud/backend.rs index 8f6628ae0e0f..c2c7b174ff4f 100644 --- a/core/src/services/icloud/backend.rs +++ b/core/src/services/icloud/backend.rs @@ -267,7 +267,7 @@ pub struct IcloudBackend { #[async_trait] impl Accessor for IcloudBackend { - type Reader = IncomingAsyncBody; + type Reader = oio::Buffer; type BlockingReader = (); type Writer = (); type BlockingWriter = (); @@ -325,8 +325,7 @@ impl Accessor for IcloudBackend { )) } StatusCode::RANGE_NOT_SATISFIABLE => { - resp.into_body().consume().await?; - Ok((RpRead::new().with_size(Some(0)), IncomingAsyncBody::empty())) + Ok((RpRead::new().with_size(Some(0)), oio::Buffer::empty())) } _ => Err(parse_error(resp).await?), } diff --git a/core/src/services/icloud/core.rs b/core/src/services/icloud/core.rs index 378c164217e1..2a2af72d5042 100644 --- a/core/src/services/icloud/core.rs +++ b/core/src/services/icloud/core.rs @@ -192,7 +192,7 @@ impl IcloudSigner { // Updata SessionData cookies.We need obtain `X-APPLE-WEBAUTH-USER` cookie to get file. self.update(&resp)?; - let bs = resp.into_body().bytes().await?; + let bs = resp.into_body(); let auth_info: IcloudWebservicesResponse = serde_json::from_slice(&bs).map_err(new_json_deserialize_error)?; @@ -269,7 +269,7 @@ impl IcloudSigner { } /// Update signer's data after request sent out. - fn update(&mut self, resp: &Response) -> Result<()> { + fn update(&mut self, resp: &Response) -> Result<()> { if let Some(account_country) = parse_header_to_str(resp.headers(), ACCOUNT_COUNTRY_HEADER)? { self.data.account_country = Some(account_country.to_string()); @@ -306,10 +306,7 @@ impl IcloudSigner { /// - Init the signer if it's not initiated. /// - Sign the request. /// - Update the session data if needed. - pub async fn send( - &mut self, - mut req: Request, - ) -> Result> { + pub async fn send(&mut self, mut req: Request) -> Result> { self.sign(&mut req)?; let resp = self.client.send(req).await?; @@ -370,7 +367,7 @@ impl IcloudCore { id: &str, zone: &str, args: OpRead, - ) -> Result> { + ) -> Result> { let mut signer = self.signer.lock().await; let uri = format!( @@ -418,7 +415,7 @@ impl IcloudCore { Ok(resp) } - pub async fn read(&self, path: &str, args: &OpRead) -> Result> { + pub async fn read(&self, path: &str, args: &OpRead) -> Result> { let path = build_rooted_abs_path(&self.root, path); let base = get_basename(&path); @@ -564,9 +561,9 @@ impl PathQuery for IcloudPathQuery { } } -pub async fn parse_error(resp: Response) -> Result { +pub async fn parse_error(resp: Response) -> Result { let (parts, body) = resp.into_parts(); - let bs = body.bytes().await?; + let bs = body.copy_to_bytes(body.remaining()); let mut kind = match parts.status.as_u16() { 421 | 450 | 500 => ErrorKind::NotFound, diff --git a/core/src/services/ipfs/backend.rs b/core/src/services/ipfs/backend.rs index 964a1cef1923..aa83e5d8de26 100644 --- a/core/src/services/ipfs/backend.rs +++ b/core/src/services/ipfs/backend.rs @@ -161,7 +161,7 @@ impl Debug for IpfsBackend { #[async_trait] impl Accessor for IpfsBackend { - type Reader = IncomingAsyncBody; + type Reader = oio::Buffer; type Writer = (); type Lister = oio::PageLister; type BlockingReader = (); @@ -356,7 +356,7 @@ impl Accessor for IpfsBackend { } impl IpfsBackend { - async fn ipfs_get(&self, path: &str, range: BytesRange) -> Result> { + async fn ipfs_get(&self, path: &str, range: BytesRange) -> Result> { let p = build_rooted_abs_path(&self.root, path); let url = format!("{}{}", self.endpoint, percent_encode_path(&p)); @@ -374,7 +374,7 @@ impl IpfsBackend { self.client.send(req).await } - async fn ipfs_head(&self, path: &str) -> Result> { + async fn ipfs_head(&self, path: &str) -> Result> { let p = build_rooted_abs_path(&self.root, path); let url = format!("{}{}", self.endpoint, percent_encode_path(&p)); @@ -388,7 +388,7 @@ impl IpfsBackend { self.client.send(req).await } - async fn ipfs_list(&self, path: &str) -> Result> { + async fn ipfs_list(&self, path: &str) -> Result> { let p = build_rooted_abs_path(&self.root, path); let url = format!("{}{}", self.endpoint, percent_encode_path(&p)); @@ -431,7 +431,7 @@ impl oio::PageList for DirStream { return Err(parse_error(resp).await?); } - let bs = resp.into_body().bytes().await?; + let bs = resp.into_body(); let pb_node = PBNode::decode(bs).map_err(|e| { Error::new(ErrorKind::Unexpected, "deserialize protobuf from response").set_source(e) })?; diff --git a/core/src/services/ipfs/error.rs b/core/src/services/ipfs/error.rs index 1470b209899e..d5211cd2f3e0 100644 --- a/core/src/services/ipfs/error.rs +++ b/core/src/services/ipfs/error.rs @@ -24,9 +24,9 @@ use crate::ErrorKind; use crate::Result; /// Parse error response into Error. -pub async fn parse_error(resp: Response) -> Result { +pub async fn parse_error(resp: Response) -> Result { let (parts, body) = resp.into_parts(); - let bs = body.bytes().await?; + let bs = body.copy_to_bytes(body.remaining()); let (kind, retryable) = match parts.status { StatusCode::NOT_FOUND => (ErrorKind::NotFound, false), diff --git a/core/src/services/ipmfs/backend.rs b/core/src/services/ipmfs/backend.rs index 4d5f0c6d64c2..250791475739 100644 --- a/core/src/services/ipmfs/backend.rs +++ b/core/src/services/ipmfs/backend.rs @@ -63,7 +63,7 @@ impl IpmfsBackend { #[async_trait] impl Accessor for IpmfsBackend { - type Reader = IncomingAsyncBody; + type Reader = oio::Buffer; type Writer = oio::OneShotWriter; type Lister = oio::PageLister; type BlockingReader = (); @@ -98,10 +98,7 @@ impl Accessor for IpmfsBackend { let status = resp.status(); match status { - StatusCode::CREATED | StatusCode::OK => { - resp.into_body().consume().await?; - Ok(RpCreateDir::default()) - } + StatusCode::CREATED | StatusCode::OK => Ok(RpCreateDir::default()), _ => Err(parse_error(resp).await?), } } @@ -118,7 +115,7 @@ impl Accessor for IpmfsBackend { match status { StatusCode::OK => { - let bs = resp.into_body().bytes().await?; + let bs = resp.into_body(); let res: IpfsStatResponse = serde_json::from_slice(&bs).map_err(new_json_deserialize_error)?; @@ -162,10 +159,7 @@ impl Accessor for IpmfsBackend { let status = resp.status(); match status { - StatusCode::OK => { - resp.into_body().consume().await?; - Ok(RpDelete::default()) - } + StatusCode::OK => Ok(RpDelete::default()), _ => Err(parse_error(resp).await?), } } @@ -177,7 +171,7 @@ impl Accessor for IpmfsBackend { } impl IpmfsBackend { - async fn ipmfs_stat(&self, path: &str) -> Result> { + async fn ipmfs_stat(&self, path: &str) -> Result> { let p = build_rooted_abs_path(&self.root, path); let url = format!( @@ -194,11 +188,7 @@ impl IpmfsBackend { self.client.send(req).await } - async fn ipmfs_read( - &self, - path: &str, - range: BytesRange, - ) -> Result> { + async fn ipmfs_read(&self, path: &str, range: BytesRange) -> Result> { let p = build_rooted_abs_path(&self.root, path); let mut url = format!( @@ -222,7 +212,7 @@ impl IpmfsBackend { self.client.send(req).await } - async fn ipmfs_rm(&self, path: &str) -> Result> { + async fn ipmfs_rm(&self, path: &str) -> Result> { let p = build_rooted_abs_path(&self.root, path); let url = format!( @@ -239,7 +229,7 @@ impl IpmfsBackend { self.client.send(req).await } - pub(crate) async fn ipmfs_ls(&self, path: &str) -> Result> { + pub(crate) async fn ipmfs_ls(&self, path: &str) -> Result> { let p = build_rooted_abs_path(&self.root, path); let url = format!( @@ -256,7 +246,7 @@ impl IpmfsBackend { self.client.send(req).await } - async fn ipmfs_mkdir(&self, path: &str) -> Result> { + async fn ipmfs_mkdir(&self, path: &str) -> Result> { let p = build_rooted_abs_path(&self.root, path); let url = format!( @@ -274,11 +264,7 @@ impl IpmfsBackend { } /// Support write from reader. - pub async fn ipmfs_write( - &self, - path: &str, - body: Bytes, - ) -> Result> { + pub async fn ipmfs_write(&self, path: &str, body: Bytes) -> Result> { let p = build_rooted_abs_path(&self.root, path); let url = format!( diff --git a/core/src/services/ipmfs/error.rs b/core/src/services/ipmfs/error.rs index df6e7f4752d7..118ae17adeb1 100644 --- a/core/src/services/ipmfs/error.rs +++ b/core/src/services/ipmfs/error.rs @@ -45,9 +45,9 @@ struct IpfsError { /// > (if no error, check the daemon logs). /// /// ref: https://docs.ipfs.tech/reference/kubo/rpc/#http-status-codes -pub async fn parse_error(resp: Response) -> Result { +pub async fn parse_error(resp: Response) -> Result { let (parts, body) = resp.into_parts(); - let bs = body.bytes().await?; + let bs = body.copy_to_bytes(body.remaining()); let ipfs_error = de::from_slice::(&bs).ok(); diff --git a/core/src/services/ipmfs/lister.rs b/core/src/services/ipmfs/lister.rs index 81bff72dca52..7767b2518468 100644 --- a/core/src/services/ipmfs/lister.rs +++ b/core/src/services/ipmfs/lister.rs @@ -51,7 +51,7 @@ impl oio::PageList for IpmfsLister { return Err(parse_error(resp).await?); } - let bs = resp.into_body().bytes().await?; + let bs = resp.into_body(); let entries_body: IpfsLsResponse = serde_json::from_slice(&bs).map_err(new_json_deserialize_error)?; diff --git a/core/src/services/ipmfs/writer.rs b/core/src/services/ipmfs/writer.rs index 53f79b840c4f..c8608d533569 100644 --- a/core/src/services/ipmfs/writer.rs +++ b/core/src/services/ipmfs/writer.rs @@ -43,10 +43,7 @@ impl oio::OneShotWrite for IpmfsWriter { let status = resp.status(); match status { - StatusCode::CREATED | StatusCode::OK => { - resp.into_body().consume().await?; - Ok(()) - } + StatusCode::CREATED | StatusCode::OK => Ok(()), _ => Err(parse_error(resp).await?), } } diff --git a/core/src/services/koofr/backend.rs b/core/src/services/koofr/backend.rs index a7f8fc35901c..bf2ec885742f 100644 --- a/core/src/services/koofr/backend.rs +++ b/core/src/services/koofr/backend.rs @@ -235,7 +235,7 @@ pub struct KoofrBackend { #[async_trait] impl Accessor for KoofrBackend { - type Reader = IncomingAsyncBody; + type Reader = oio::Buffer; type Writer = KoofrWriters; type Lister = oio::PageLister; type BlockingReader = (); @@ -286,7 +286,7 @@ impl Accessor for KoofrBackend { match status { StatusCode::OK => { - let bs = resp.into_body().bytes().await?; + let bs = resp.into_body(); let file: File = serde_json::from_slice(&bs).map_err(new_json_deserialize_error)?; @@ -372,11 +372,7 @@ impl Accessor for KoofrBackend { let status = resp.status(); match status { - StatusCode::OK => { - resp.into_body().consume().await?; - - Ok(RpCopy::default()) - } + StatusCode::OK => Ok(RpCopy::default()), _ => Err(parse_error(resp).await?), } } @@ -401,11 +397,7 @@ impl Accessor for KoofrBackend { let status = resp.status(); match status { - StatusCode::OK => { - resp.into_body().consume().await?; - - Ok(RpRename::default()) - } + StatusCode::OK => Ok(RpRename::default()), _ => Err(parse_error(resp).await?), } } diff --git a/core/src/services/koofr/core.rs b/core/src/services/koofr/core.rs index 441c3ffb0142..2e2f7d46282c 100644 --- a/core/src/services/koofr/core.rs +++ b/core/src/services/koofr/core.rs @@ -67,7 +67,7 @@ impl Debug for KoofrCore { impl KoofrCore { #[inline] - pub async fn send(&self, req: Request) -> Result> { + pub async fn send(&self, req: Request) -> Result> { self.client.send(req).await } @@ -90,7 +90,7 @@ impl KoofrCore { return Err(parse_error(resp).await?); } - let bs = resp.into_body().bytes().await?; + let bs = resp.into_body(); let resp: MountsResponse = serde_json::from_slice(&bs).map_err(new_json_deserialize_error)?; @@ -137,7 +137,7 @@ impl KoofrCore { return Err(parse_error(resp).await?); } - let bs = resp.into_body().bytes().await?; + let bs = resp.into_body(); let resp: TokenResponse = serde_json::from_slice(&bs).map_err(new_json_deserialize_error)?; @@ -220,7 +220,7 @@ impl KoofrCore { } } - pub async fn info(&self, path: &str) -> Result> { + pub async fn info(&self, path: &str) -> Result> { let mount_id = self.get_mount_id().await?; let url = format!( @@ -241,7 +241,7 @@ impl KoofrCore { self.send(req).await } - pub async fn get(&self, path: &str) -> Result> { + pub async fn get(&self, path: &str) -> Result> { let path = build_rooted_abs_path(&self.root, path); let mount_id = self.get_mount_id().await?; @@ -264,7 +264,7 @@ impl KoofrCore { self.send(req).await } - pub async fn put(&self, path: &str, bs: Bytes) -> Result> { + pub async fn put(&self, path: &str, bs: Bytes) -> Result> { let path = build_rooted_abs_path(&self.root, path); let filename = get_basename(&path); @@ -300,7 +300,7 @@ impl KoofrCore { self.send(req).await } - pub async fn remove(&self, path: &str) -> Result> { + pub async fn remove(&self, path: &str) -> Result> { let path = build_rooted_abs_path(&self.root, path); let mount_id = self.get_mount_id().await?; @@ -323,7 +323,7 @@ impl KoofrCore { self.send(req).await } - pub async fn copy(&self, from: &str, to: &str) -> Result> { + pub async fn copy(&self, from: &str, to: &str) -> Result> { let from = build_rooted_abs_path(&self.root, from); let to = build_rooted_abs_path(&self.root, to); @@ -355,7 +355,7 @@ impl KoofrCore { self.send(req).await } - pub async fn move_object(&self, from: &str, to: &str) -> Result> { + pub async fn move_object(&self, from: &str, to: &str) -> Result> { let from = build_rooted_abs_path(&self.root, from); let to = build_rooted_abs_path(&self.root, to); @@ -387,7 +387,7 @@ impl KoofrCore { self.send(req).await } - pub async fn list(&self, path: &str) -> Result> { + pub async fn list(&self, path: &str) -> Result> { let path = build_rooted_abs_path(&self.root, path); let mount_id = self.get_mount_id().await?; diff --git a/core/src/services/koofr/error.rs b/core/src/services/koofr/error.rs index c3104c781fd8..559377549b60 100644 --- a/core/src/services/koofr/error.rs +++ b/core/src/services/koofr/error.rs @@ -23,9 +23,9 @@ use crate::ErrorKind; use crate::Result; /// Parse error response into Error. -pub async fn parse_error(resp: Response) -> Result { +pub async fn parse_error(resp: Response) -> Result { let (parts, body) = resp.into_parts(); - let bs = body.bytes().await?; + let bs = body.copy_to_bytes(body.remaining()); let (kind, retryable) = match parts.status.as_u16() { 403 => (ErrorKind::PermissionDenied, false), @@ -64,7 +64,7 @@ mod test { for res in err_res { let bs = bytes::Bytes::from(res.0); - let body = IncomingAsyncBody::new( + let body = oio::Buffer::new( Box::new(oio::into_stream(stream::iter(vec![Ok(bs.clone())]))), None, ); diff --git a/core/src/services/koofr/lister.rs b/core/src/services/koofr/lister.rs index 8d56e65f5be8..1815e9785e4a 100644 --- a/core/src/services/koofr/lister.rs +++ b/core/src/services/koofr/lister.rs @@ -57,7 +57,7 @@ impl oio::PageList for KoofrLister { } } - let bs = resp.into_body().bytes().await?; + let bs = resp.into_body(); let response: ListResponse = serde_json::from_slice(&bs).map_err(new_json_deserialize_error)?; diff --git a/core/src/services/koofr/writer.rs b/core/src/services/koofr/writer.rs index 97c2dbdbc9b8..ff8039e8afd2 100644 --- a/core/src/services/koofr/writer.rs +++ b/core/src/services/koofr/writer.rs @@ -47,10 +47,7 @@ impl oio::OneShotWrite for KoofrWriter { let status = resp.status(); match status { - StatusCode::OK | StatusCode::CREATED => { - resp.into_body().consume().await?; - Ok(()) - } + StatusCode::OK | StatusCode::CREATED => Ok(()), _ => Err(parse_error(resp).await?), } } diff --git a/core/src/services/libsql/backend.rs b/core/src/services/libsql/backend.rs index c000a9ddd2ec..069784e7546f 100644 --- a/core/src/services/libsql/backend.rs +++ b/core/src/services/libsql/backend.rs @@ -309,7 +309,7 @@ impl Adapter { return Err(parse_error(resp).await?); } - let bs = resp.into_body().bytes().await?; + let bs = resp.into_body(); let resp: ServerMsg = serde_json::from_slice(&bs).map_err(|e| { Error::new(ErrorKind::Unexpected, "deserialize json from response").set_source(e) diff --git a/core/src/services/libsql/error.rs b/core/src/services/libsql/error.rs index a2a80a02ec10..f55b4d26b0a8 100644 --- a/core/src/services/libsql/error.rs +++ b/core/src/services/libsql/error.rs @@ -24,9 +24,9 @@ use crate::ErrorKind; use crate::Result; /// Parse error response into Error. -pub async fn parse_error(resp: Response) -> Result { +pub async fn parse_error(resp: Response) -> Result { let (parts, body) = resp.into_parts(); - let bs = body.bytes().await?; + let bs = body.copy_to_bytes(body.remaining()); let (kind, retryable) = match parts.status { StatusCode::NOT_FOUND => (ErrorKind::NotFound, false), diff --git a/core/src/services/obs/backend.rs b/core/src/services/obs/backend.rs index 91bfa8893de3..fcac8b94aea6 100644 --- a/core/src/services/obs/backend.rs +++ b/core/src/services/obs/backend.rs @@ -248,7 +248,7 @@ pub struct ObsBackend { #[async_trait] impl Accessor for ObsBackend { - type Reader = IncomingAsyncBody; + type Reader = oio::Buffer; type Writer = ObsWriters; type Lister = oio::PageLister; type BlockingReader = (); @@ -337,8 +337,7 @@ impl Accessor for ObsBackend { )) } StatusCode::RANGE_NOT_SATISFIABLE => { - resp.into_body().consume().await?; - Ok((RpRead::new().with_size(Some(0)), IncomingAsyncBody::empty())) + Ok((RpRead::new().with_size(Some(0)), oio::Buffer::empty())) } _ => Err(parse_error(resp).await?), } @@ -380,10 +379,7 @@ impl Accessor for ObsBackend { let status = resp.status(); match status { - StatusCode::OK => { - resp.into_body().consume().await?; - Ok(RpCopy::default()) - } + StatusCode::OK => Ok(RpCopy::default()), _ => Err(parse_error(resp).await?), } } diff --git a/core/src/services/obs/core.rs b/core/src/services/obs/core.rs index 344f104561de..71633d603d06 100644 --- a/core/src/services/obs/core.rs +++ b/core/src/services/obs/core.rs @@ -95,17 +95,13 @@ impl ObsCore { } #[inline] - pub async fn send(&self, req: Request) -> Result> { + pub async fn send(&self, req: Request) -> Result> { self.client.send(req).await } } impl ObsCore { - pub async fn obs_get_object( - &self, - path: &str, - args: &OpRead, - ) -> Result> { + pub async fn obs_get_object(&self, path: &str, args: &OpRead) -> Result> { let mut req = self.obs_get_object_request(path, args)?; self.sign(&mut req).await?; @@ -173,7 +169,7 @@ impl ObsCore { &self, path: &str, args: &OpStat, - ) -> Result> { + ) -> Result> { let mut req = self.obs_head_object_request(path, args)?; self.sign(&mut req).await?; @@ -206,7 +202,7 @@ impl ObsCore { Ok(req) } - pub async fn obs_delete_object(&self, path: &str) -> Result> { + pub async fn obs_delete_object(&self, path: &str) -> Result> { let p = build_abs_path(&self.root, path); let url = format!("{}/{}", self.endpoint, percent_encode_path(&p)); @@ -258,11 +254,7 @@ impl ObsCore { Ok(req) } - pub async fn obs_copy_object( - &self, - from: &str, - to: &str, - ) -> Result> { + pub async fn obs_copy_object(&self, from: &str, to: &str) -> Result> { let source = build_abs_path(&self.root, from); let target = build_abs_path(&self.root, to); @@ -285,7 +277,7 @@ impl ObsCore { next_marker: &str, delimiter: &str, limit: Option, - ) -> Result> { + ) -> Result> { let p = build_abs_path(&self.root, path); let mut queries = vec![]; @@ -320,7 +312,7 @@ impl ObsCore { &self, path: &str, content_type: Option<&str>, - ) -> Result> { + ) -> Result> { let p = build_abs_path(&self.root, path); let url = format!("{}/{}?uploads", self.endpoint, percent_encode_path(&p)); @@ -344,7 +336,7 @@ impl ObsCore { part_number: usize, size: Option, body: AsyncBody, - ) -> Result> { + ) -> Result> { let p = build_abs_path(&self.root, path); let url = format!( @@ -374,7 +366,7 @@ impl ObsCore { path: &str, upload_id: &str, parts: Vec, - ) -> Result> { + ) -> Result> { let p = build_abs_path(&self.root, path); let url = format!( "{}/{}?uploadId={}", @@ -407,7 +399,7 @@ impl ObsCore { &self, path: &str, upload_id: &str, - ) -> Result> { + ) -> Result> { let p = build_abs_path(&self.root, path); let url = format!( diff --git a/core/src/services/obs/error.rs b/core/src/services/obs/error.rs index 0c2081312bc9..bb33631fa542 100644 --- a/core/src/services/obs/error.rs +++ b/core/src/services/obs/error.rs @@ -38,9 +38,9 @@ struct ObsError { } /// Parse error response into Error. -pub async fn parse_error(resp: Response) -> Result { +pub async fn parse_error(resp: Response) -> Result { let (parts, body) = resp.into_parts(); - let bs = body.bytes().await?; + let bs = body.copy_to_bytes(body.remaining()); let (kind, retryable) = match parts.status { StatusCode::NOT_FOUND => (ErrorKind::NotFound, false), diff --git a/core/src/services/obs/lister.rs b/core/src/services/obs/lister.rs index e38ebe75197f..819283a1b86b 100644 --- a/core/src/services/obs/lister.rs +++ b/core/src/services/obs/lister.rs @@ -58,7 +58,7 @@ impl oio::PageList for ObsLister { return Err(parse_error(resp).await?); } - let bs = resp.into_body().bytes().await?; + let bs = resp.into_body(); let output: ListObjectsOutput = de::from_reader(bs.reader()).map_err(new_xml_deserialize_error)?; diff --git a/core/src/services/obs/writer.rs b/core/src/services/obs/writer.rs index 7df2e7cbcd04..e738bd04f270 100644 --- a/core/src/services/obs/writer.rs +++ b/core/src/services/obs/writer.rs @@ -57,10 +57,7 @@ impl oio::MultipartWrite for ObsWriter { let status = resp.status(); match status { - StatusCode::CREATED | StatusCode::OK => { - resp.into_body().consume().await?; - Ok(()) - } + StatusCode::CREATED | StatusCode::OK => Ok(()), _ => Err(parse_error(resp).await?), } } @@ -75,7 +72,7 @@ impl oio::MultipartWrite for ObsWriter { match status { StatusCode::OK => { - let bs = resp.into_body().bytes().await?; + let bs = resp.into_body(); let result: InitiateMultipartUploadResult = quick_xml::de::from_reader(bytes::Buf::reader(bs)) @@ -115,8 +112,6 @@ impl oio::MultipartWrite for ObsWriter { })? .to_string(); - resp.into_body().consume().await?; - Ok(MultipartPart { part_number, etag }) } _ => Err(parse_error(resp).await?), @@ -140,11 +135,7 @@ impl oio::MultipartWrite for ObsWriter { let status = resp.status(); match status { - StatusCode::OK => { - resp.into_body().consume().await?; - - Ok(()) - } + StatusCode::OK => Ok(()), _ => Err(parse_error(resp).await?), } } @@ -157,10 +148,7 @@ impl oio::MultipartWrite for ObsWriter { match resp.status() { // Obs returns code 204 No Content if abort succeeds. // Reference: https://support.huaweicloud.com/intl/en-us/api-obs/obs_04_0103.html - StatusCode::NO_CONTENT => { - resp.into_body().consume().await?; - Ok(()) - } + StatusCode::NO_CONTENT => Ok(()), _ => Err(parse_error(resp).await?), } } diff --git a/core/src/services/onedrive/backend.rs b/core/src/services/onedrive/backend.rs index b63288c38c14..c3e41ebc99bf 100644 --- a/core/src/services/onedrive/backend.rs +++ b/core/src/services/onedrive/backend.rs @@ -62,7 +62,7 @@ impl Debug for OnedriveBackend { #[async_trait] impl Accessor for OnedriveBackend { - type Reader = IncomingAsyncBody; + type Reader = oio::Buffer; type Writer = oio::OneShotWriter; type Lister = oio::PageLister; type BlockingReader = (); @@ -120,7 +120,7 @@ impl Accessor for OnedriveBackend { let status = resp.status(); if status.is_success() { - let bytes = resp.into_body().bytes().await?; + let bytes = resp.into_body(); let decoded_response = serde_json::from_slice::(&bytes) .map_err(new_json_deserialize_error)?; @@ -199,7 +199,7 @@ impl Accessor for OnedriveBackend { impl OnedriveBackend { pub(crate) const BASE_URL: &'static str = "https://graph.microsoft.com/v1.0/me"; - async fn onedrive_get_stat(&self, path: &str) -> Result> { + async fn onedrive_get_stat(&self, path: &str) -> Result> { let path = build_rooted_abs_path(&self.root, path); let url: String = format!( "https://graph.microsoft.com/v1.0/me/drive/root:{}{}", @@ -222,7 +222,7 @@ impl OnedriveBackend { pub(crate) async fn onedrive_get_next_list_page( &self, url: &str, - ) -> Result> { + ) -> Result> { let mut req = Request::get(url); let auth_header_content = format!("Bearer {}", self.access_token); @@ -235,7 +235,7 @@ impl OnedriveBackend { self.client.send(req).await } - async fn onedrive_get_content(&self, path: &str) -> Result> { + async fn onedrive_get_content(&self, path: &str) -> Result> { let path = build_rooted_abs_path(&self.root, path); let url: String = format!( "https://graph.microsoft.com/v1.0/me/drive/root:{}{}", @@ -261,7 +261,7 @@ impl OnedriveBackend { size: Option, args: &OpWrite, body: AsyncBody, - ) -> Result> { + ) -> Result> { let url = format!( "https://graph.microsoft.com/v1.0/me/drive/root:{}:/content", percent_encode_path(path) @@ -293,7 +293,7 @@ impl OnedriveBackend { chunk_end: usize, total_len: usize, body: AsyncBody, - ) -> Result> { + ) -> Result> { let mut req = Request::put(url); let auth_header_content = format!("Bearer {}", self.access_token); @@ -318,7 +318,7 @@ impl OnedriveBackend { &self, url: &str, body: OneDriveUploadSessionCreationRequestBody, - ) -> Result> { + ) -> Result> { let mut req = Request::post(url); let auth_header_content = format!("Bearer {}", self.access_token); @@ -337,7 +337,7 @@ impl OnedriveBackend { &self, url: &str, body: CreateDirPayload, - ) -> Result> { + ) -> Result> { let mut req = Request::post(url); let auth_header_content = format!("Bearer {}", self.access_token); @@ -351,7 +351,7 @@ impl OnedriveBackend { self.client.send(req).await } - pub(crate) async fn onedrive_delete(&self, path: &str) -> Result> { + pub(crate) async fn onedrive_delete(&self, path: &str) -> Result> { let path = build_abs_path(&self.root, path); let url = format!( "https://graph.microsoft.com/v1.0/me/drive/root:/{}", diff --git a/core/src/services/onedrive/error.rs b/core/src/services/onedrive/error.rs index 60de6ae0d0a9..1c8b8218c6c2 100644 --- a/core/src/services/onedrive/error.rs +++ b/core/src/services/onedrive/error.rs @@ -24,9 +24,9 @@ use crate::ErrorKind; use crate::Result; /// Parse error response into Error. -pub async fn parse_error(resp: Response) -> Result { +pub async fn parse_error(resp: Response) -> Result { let (parts, body) = resp.into_parts(); - let bs = body.bytes().await?; + let bs = body.copy_to_bytes(body.remaining()); let (kind, retryable) = match parts.status { StatusCode::NOT_FOUND => (ErrorKind::NotFound, false), diff --git a/core/src/services/onedrive/lister.rs b/core/src/services/onedrive/lister.rs index 32fdc426f7b9..64ca0b5cfc3c 100644 --- a/core/src/services/onedrive/lister.rs +++ b/core/src/services/onedrive/lister.rs @@ -76,7 +76,7 @@ impl oio::PageList for OnedriveLister { return Err(error); } - let bytes = resp.into_body().bytes().await?; + let bytes = resp.into_body(); let decoded_response = serde_json::from_slice::(&bytes) .map_err(new_json_deserialize_error)?; diff --git a/core/src/services/onedrive/writer.rs b/core/src/services/onedrive/writer.rs index 60988f20008b..374a037b8be9 100644 --- a/core/src/services/onedrive/writer.rs +++ b/core/src/services/onedrive/writer.rs @@ -68,10 +68,7 @@ impl OneDriveWriter { match status { // Typical response code: 201 Created // Reference: https://learn.microsoft.com/en-us/onedrive/developer/rest-api/api/driveitem_put_content?view=odsp-graph-online#response - StatusCode::CREATED | StatusCode::OK => { - resp.into_body().consume().await?; - Ok(()) - } + StatusCode::CREATED | StatusCode::OK => Ok(()), _ => Err(parse_error(resp).await?), } } @@ -113,9 +110,7 @@ impl OneDriveWriter { match status { // Typical response code: 202 Accepted // Reference: https://learn.microsoft.com/en-us/onedrive/developer/rest-api/api/driveitem_put_content?view=odsp-graph-online#response - StatusCode::ACCEPTED | StatusCode::CREATED | StatusCode::OK => { - resp.into_body().consume().await?; - } + StatusCode::ACCEPTED | StatusCode::CREATED | StatusCode::OK => {} _ => return Err(parse_error(resp).await?), } @@ -149,7 +144,7 @@ impl OneDriveWriter { match status { // Reference: https://learn.microsoft.com/en-us/onedrive/developer/rest-api/api/driveitem_createuploadsession?view=odsp-graph-online#response StatusCode::OK => { - let bs = resp.into_body().bytes().await?; + let bs = resp.into_body(); let result: OneDriveUploadSessionCreationResponseBody = serde_json::from_slice(&bs).map_err(new_json_deserialize_error)?; Ok(result) diff --git a/core/src/services/oss/backend.rs b/core/src/services/oss/backend.rs index 2b5583d34550..a2965184e883 100644 --- a/core/src/services/oss/backend.rs +++ b/core/src/services/oss/backend.rs @@ -376,7 +376,7 @@ pub struct OssBackend { #[async_trait] impl Accessor for OssBackend { - type Reader = IncomingAsyncBody; + type Reader = oio::Buffer; type Writer = OssWriters; type Lister = oio::PageLister; type BlockingReader = (); @@ -479,8 +479,7 @@ impl Accessor for OssBackend { )) } StatusCode::RANGE_NOT_SATISFIABLE => { - resp.into_body().consume().await?; - Ok((RpRead::new().with_size(Some(0)), IncomingAsyncBody::empty())) + Ok((RpRead::new().with_size(Some(0)), oio::Buffer::empty())) } _ => Err(parse_error(resp).await?), } @@ -502,10 +501,7 @@ impl Accessor for OssBackend { let resp = self.core.oss_delete_object(path).await?; let status = resp.status(); match status { - StatusCode::NO_CONTENT | StatusCode::NOT_FOUND => { - resp.into_body().consume().await?; - Ok(RpDelete::default()) - } + StatusCode::NO_CONTENT | StatusCode::NOT_FOUND => Ok(RpDelete::default()), _ => Err(parse_error(resp).await?), } } @@ -526,10 +522,7 @@ impl Accessor for OssBackend { let status = resp.status(); match status { - StatusCode::OK => { - resp.into_body().consume().await?; - Ok(RpCopy::default()) - } + StatusCode::OK => Ok(RpCopy::default()), _ => Err(parse_error(resp).await?), } } @@ -595,7 +588,7 @@ impl Accessor for OssBackend { let status = resp.status(); if let StatusCode::OK = status { - let bs = resp.into_body().bytes().await?; + let bs = resp.into_body(); let result: DeleteObjectsResult = quick_xml::de::from_reader(bs.reader()).map_err(new_xml_deserialize_error)?; diff --git a/core/src/services/oss/core.rs b/core/src/services/oss/core.rs index 66d4654b6e64..d70009cc2b60 100644 --- a/core/src/services/oss/core.rs +++ b/core/src/services/oss/core.rs @@ -126,7 +126,7 @@ impl OssCore { } #[inline] - pub async fn send(&self, req: Request) -> Result> { + pub async fn send(&self, req: Request) -> Result> { self.client.send(req).await } @@ -373,7 +373,7 @@ impl OssCore { if_match: Option<&str>, if_none_match: Option<&str>, override_content_disposition: Option<&str>, - ) -> Result> { + ) -> Result> { let mut req = self.oss_get_object_request( path, range, @@ -391,7 +391,7 @@ impl OssCore { path: &str, if_match: Option<&str>, if_none_match: Option<&str>, - ) -> Result> { + ) -> Result> { let mut req = self.oss_head_object_request(path, false, if_match, if_none_match)?; self.sign(&mut req).await?; @@ -404,18 +404,14 @@ impl OssCore { size: Option, args: &OpWrite, body: AsyncBody, - ) -> Result> { + ) -> Result> { let mut req = self.oss_put_object_request(path, size, args, body, false)?; self.sign(&mut req).await?; self.send(req).await } - pub async fn oss_copy_object( - &self, - from: &str, - to: &str, - ) -> Result> { + pub async fn oss_copy_object(&self, from: &str, to: &str) -> Result> { let source = build_abs_path(&self.root, from); let target = build_abs_path(&self.root, to); @@ -447,23 +443,20 @@ impl OssCore { delimiter: &str, limit: Option, start_after: Option, - ) -> Result> { + ) -> Result> { let mut req = self.oss_list_object_request(path, token, delimiter, limit, start_after)?; self.sign(&mut req).await?; self.send(req).await } - pub async fn oss_delete_object(&self, path: &str) -> Result> { + pub async fn oss_delete_object(&self, path: &str) -> Result> { let mut req = self.oss_delete_object_request(path)?; self.sign(&mut req).await?; self.send(req).await } - pub async fn oss_delete_objects( - &self, - paths: Vec, - ) -> Result> { + pub async fn oss_delete_objects(&self, paths: Vec) -> Result> { let url = format!("{}/?delete", self.endpoint); let req = Request::post(&url); @@ -509,7 +502,7 @@ impl OssCore { content_disposition: Option<&str>, cache_control: Option<&str>, is_presign: bool, - ) -> Result> { + ) -> Result> { let path = build_abs_path(&self.root, path); let endpoint = self.get_endpoint(is_presign); let url = format!("{}/{}?uploads", endpoint, percent_encode_path(&path)); @@ -540,7 +533,7 @@ impl OssCore { is_presign: bool, size: u64, body: AsyncBody, - ) -> Result> { + ) -> Result> { let p = build_abs_path(&self.root, path); let endpoint = self.get_endpoint(is_presign); @@ -565,7 +558,7 @@ impl OssCore { upload_id: &str, is_presign: bool, parts: Vec, - ) -> Result> { + ) -> Result> { let p = build_abs_path(&self.root, path); let endpoint = self.get_endpoint(is_presign); let url = format!( @@ -600,7 +593,7 @@ impl OssCore { &self, path: &str, upload_id: &str, - ) -> Result> { + ) -> Result> { let p = build_abs_path(&self.root, path); let url = format!( diff --git a/core/src/services/oss/error.rs b/core/src/services/oss/error.rs index ecead41c4f76..6aeef66cf491 100644 --- a/core/src/services/oss/error.rs +++ b/core/src/services/oss/error.rs @@ -37,9 +37,9 @@ struct OssError { } /// Parse error response into Error. -pub async fn parse_error(resp: Response) -> Result { +pub async fn parse_error(resp: Response) -> Result { let (parts, body) = resp.into_parts(); - let bs = body.bytes().await?; + let bs = body.copy_to_bytes(body.remaining()); let (kind, retryable) = match parts.status { StatusCode::NOT_FOUND => (ErrorKind::NotFound, false), diff --git a/core/src/services/oss/lister.rs b/core/src/services/oss/lister.rs index 28919cfa80e9..0022597a7eed 100644 --- a/core/src/services/oss/lister.rs +++ b/core/src/services/oss/lister.rs @@ -76,7 +76,7 @@ impl oio::PageList for OssLister { return Err(parse_error(resp).await?); } - let bs = resp.into_body().bytes().await?; + let bs = resp.into_body(); let output: ListObjectsOutput = de::from_reader(bs.reader()) .map_err(|e| Error::new(ErrorKind::Unexpected, "deserialize xml").set_source(e))?; diff --git a/core/src/services/oss/writer.rs b/core/src/services/oss/writer.rs index 066efe6a7c13..f55c7d7af8ee 100644 --- a/core/src/services/oss/writer.rs +++ b/core/src/services/oss/writer.rs @@ -56,10 +56,7 @@ impl oio::MultipartWrite for OssWriter { let status = resp.status(); match status { - StatusCode::CREATED | StatusCode::OK => { - resp.into_body().consume().await?; - Ok(()) - } + StatusCode::CREATED | StatusCode::OK => Ok(()), _ => Err(parse_error(resp).await?), } } @@ -80,7 +77,7 @@ impl oio::MultipartWrite for OssWriter { match status { StatusCode::OK => { - let bs = resp.into_body().bytes().await?; + let bs = resp.into_body(); let result: InitiateMultipartUploadResult = quick_xml::de::from_reader(bytes::Buf::reader(bs)) @@ -120,8 +117,6 @@ impl oio::MultipartWrite for OssWriter { })? .to_string(); - resp.into_body().consume().await?; - Ok(oio::MultipartPart { part_number, etag }) } _ => Err(parse_error(resp).await?), @@ -145,11 +140,7 @@ impl oio::MultipartWrite for OssWriter { let status = resp.status(); match status { - StatusCode::OK => { - resp.into_body().consume().await?; - - Ok(()) - } + StatusCode::OK => Ok(()), _ => Err(parse_error(resp).await?), } } @@ -161,10 +152,7 @@ impl oio::MultipartWrite for OssWriter { .await?; match resp.status() { // OSS returns code 204 if abort succeeds. - StatusCode::NO_CONTENT => { - resp.into_body().consume().await?; - Ok(()) - } + StatusCode::NO_CONTENT => Ok(()), _ => Err(parse_error(resp).await?), } } diff --git a/core/src/services/pcloud/backend.rs b/core/src/services/pcloud/backend.rs index 8283ae884610..b31f08a00089 100644 --- a/core/src/services/pcloud/backend.rs +++ b/core/src/services/pcloud/backend.rs @@ -228,7 +228,7 @@ pub struct PcloudBackend { #[async_trait] impl Accessor for PcloudBackend { - type Reader = IncomingAsyncBody; + type Reader = oio::Buffer; type Writer = PcloudWriters; type Lister = oio::PageLister; type BlockingReader = (); @@ -273,7 +273,7 @@ impl Accessor for PcloudBackend { match status { StatusCode::OK => { - let bs = resp.into_body().bytes().await?; + let bs = resp.into_body(); let resp: StatResponse = serde_json::from_slice(&bs).map_err(new_json_deserialize_error)?; let result = resp.result; @@ -334,7 +334,7 @@ impl Accessor for PcloudBackend { match status { StatusCode::OK => { - let bs = resp.into_body().bytes().await?; + let bs = resp.into_body(); let resp: PcloudError = serde_json::from_slice(&bs).map_err(new_json_deserialize_error)?; let result = resp.result; @@ -368,7 +368,7 @@ impl Accessor for PcloudBackend { match status { StatusCode::OK => { - let bs = resp.into_body().bytes().await?; + let bs = resp.into_body(); let resp: PcloudError = serde_json::from_slice(&bs).map_err(new_json_deserialize_error)?; let result = resp.result; @@ -398,7 +398,7 @@ impl Accessor for PcloudBackend { match status { StatusCode::OK => { - let bs = resp.into_body().bytes().await?; + let bs = resp.into_body(); let resp: PcloudError = serde_json::from_slice(&bs).map_err(new_json_deserialize_error)?; let result = resp.result; diff --git a/core/src/services/pcloud/core.rs b/core/src/services/pcloud/core.rs index 21d3dfedd780..7defb2b88610 100644 --- a/core/src/services/pcloud/core.rs +++ b/core/src/services/pcloud/core.rs @@ -55,7 +55,7 @@ impl Debug for PcloudCore { impl PcloudCore { #[inline] - pub async fn send(&self, req: Request) -> Result> { + pub async fn send(&self, req: Request) -> Result> { self.client.send(req).await } } @@ -84,7 +84,7 @@ impl PcloudCore { let status = resp.status(); match status { StatusCode::OK => { - let bs = resp.into_body().bytes().await?; + let bs = resp.into_body(); let resp: GetFileLinkResponse = serde_json::from_slice(&bs).map_err(new_json_deserialize_error)?; let result = resp.result; @@ -108,7 +108,7 @@ impl PcloudCore { } } - pub async fn download(&self, url: &str) -> Result> { + pub async fn download(&self, url: &str) -> Result> { let req = Request::get(url); // set body @@ -132,7 +132,7 @@ impl PcloudCore { match status { StatusCode::OK => { - let bs = resp.into_body().bytes().await?; + let bs = resp.into_body(); let resp: PcloudError = serde_json::from_slice(&bs).map_err(new_json_deserialize_error)?; let result = resp.result; @@ -153,10 +153,7 @@ impl PcloudCore { Ok(()) } - pub async fn create_folder_if_not_exists( - &self, - path: &str, - ) -> Result> { + pub async fn create_folder_if_not_exists(&self, path: &str) -> Result> { let url = format!( "{}/createfolderifnotexists?path=/{}&username={}&password={}", self.endpoint, @@ -175,7 +172,7 @@ impl PcloudCore { self.send(req).await } - pub async fn rename_file(&self, from: &str, to: &str) -> Result> { + pub async fn rename_file(&self, from: &str, to: &str) -> Result> { let from = build_abs_path(&self.root, from); let to = build_abs_path(&self.root, to); @@ -198,7 +195,7 @@ impl PcloudCore { self.send(req).await } - pub async fn rename_folder(&self, from: &str, to: &str) -> Result> { + pub async fn rename_folder(&self, from: &str, to: &str) -> Result> { let from = build_abs_path(&self.root, from); let to = build_abs_path(&self.root, to); let url = format!( @@ -220,7 +217,7 @@ impl PcloudCore { self.send(req).await } - pub async fn delete_folder(&self, path: &str) -> Result> { + pub async fn delete_folder(&self, path: &str) -> Result> { let path = build_abs_path(&self.root, path); let url = format!( @@ -241,7 +238,7 @@ impl PcloudCore { self.send(req).await } - pub async fn delete_file(&self, path: &str) -> Result> { + pub async fn delete_file(&self, path: &str) -> Result> { let path = build_abs_path(&self.root, path); let url = format!( @@ -262,7 +259,7 @@ impl PcloudCore { self.send(req).await } - pub async fn copy_file(&self, from: &str, to: &str) -> Result> { + pub async fn copy_file(&self, from: &str, to: &str) -> Result> { let from = build_abs_path(&self.root, from); let to = build_abs_path(&self.root, to); @@ -285,7 +282,7 @@ impl PcloudCore { self.send(req).await } - pub async fn copy_folder(&self, from: &str, to: &str) -> Result> { + pub async fn copy_folder(&self, from: &str, to: &str) -> Result> { let from = build_abs_path(&self.root, from); let to = build_abs_path(&self.root, to); @@ -308,7 +305,7 @@ impl PcloudCore { self.send(req).await } - pub async fn stat(&self, path: &str) -> Result> { + pub async fn stat(&self, path: &str) -> Result> { let path = build_abs_path(&self.root, path); let path = path.trim_end_matches('/'); @@ -331,7 +328,7 @@ impl PcloudCore { self.send(req).await } - pub async fn upload_file(&self, path: &str, bs: Bytes) -> Result> { + pub async fn upload_file(&self, path: &str, bs: Bytes) -> Result> { let path = build_abs_path(&self.root, path); let (name, path) = (get_basename(&path), get_parent(&path).trim_end_matches('/')); @@ -355,7 +352,7 @@ impl PcloudCore { self.send(req).await } - pub async fn list_folder(&self, path: &str) -> Result> { + pub async fn list_folder(&self, path: &str) -> Result> { let path = build_abs_path(&self.root, path); let path = normalize_root(&path); diff --git a/core/src/services/pcloud/error.rs b/core/src/services/pcloud/error.rs index e59bcb672cc8..877709b75658 100644 --- a/core/src/services/pcloud/error.rs +++ b/core/src/services/pcloud/error.rs @@ -43,9 +43,9 @@ impl Debug for PcloudError { } /// Parse error response into Error. -pub async fn parse_error(resp: Response) -> Result { +pub async fn parse_error(resp: Response) -> Result { let (parts, body) = resp.into_parts(); - let bs = body.bytes().await?; + let bs = body.copy_to_bytes(body.remaining()); let message = String::from_utf8_lossy(&bs).into_owned(); let mut err = Error::new(ErrorKind::Unexpected, &message); @@ -70,9 +70,9 @@ mod test { Invalid link - + This link was generated for another IP address. Try previous step again. - + "#, ErrorKind::Unexpected, StatusCode::GONE, @@ -80,7 +80,7 @@ mod test { for res in err_res { let bs = bytes::Bytes::from(res.0); - let body = IncomingAsyncBody::new( + let body = oio::Buffer::new( Box::new(oio::into_stream(stream::iter(vec![Ok(bs.clone())]))), None, ); diff --git a/core/src/services/pcloud/lister.rs b/core/src/services/pcloud/lister.rs index eb4a0c670bf7..1a2f810d7221 100644 --- a/core/src/services/pcloud/lister.rs +++ b/core/src/services/pcloud/lister.rs @@ -48,7 +48,7 @@ impl oio::PageList for PcloudLister { match status { StatusCode::OK => { - let bs = resp.into_body().bytes().await?; + let bs = resp.into_body(); let resp: ListFolderResponse = serde_json::from_slice(&bs).map_err(new_json_deserialize_error)?; diff --git a/core/src/services/pcloud/writer.rs b/core/src/services/pcloud/writer.rs index a19ba154c394..08750e50fd5f 100644 --- a/core/src/services/pcloud/writer.rs +++ b/core/src/services/pcloud/writer.rs @@ -49,7 +49,7 @@ impl oio::OneShotWrite for PcloudWriter { match status { StatusCode::OK => { - let bs = resp.into_body().bytes().await?; + let bs = resp.into_body(); let resp: PcloudError = serde_json::from_slice(&bs).map_err(new_json_deserialize_error)?; let result = resp.result; diff --git a/core/src/services/seafile/backend.rs b/core/src/services/seafile/backend.rs index 5bf5879c14ee..eb0294c55b68 100644 --- a/core/src/services/seafile/backend.rs +++ b/core/src/services/seafile/backend.rs @@ -256,7 +256,7 @@ pub struct SeafileBackend { #[async_trait] impl Accessor for SeafileBackend { - type Reader = IncomingAsyncBody; + type Reader = oio::Buffer; type Writer = SeafileWriters; type Lister = oio::PageLister; type BlockingReader = (); diff --git a/core/src/services/seafile/core.rs b/core/src/services/seafile/core.rs index c8af7e17014e..3c3f3737f86c 100644 --- a/core/src/services/seafile/core.rs +++ b/core/src/services/seafile/core.rs @@ -64,7 +64,7 @@ impl Debug for SeafileCore { impl SeafileCore { #[inline] - pub async fn send(&self, req: Request) -> Result> { + pub async fn send(&self, req: Request) -> Result> { self.client.send(req).await } @@ -96,7 +96,7 @@ impl SeafileCore { match status { StatusCode::OK => { - let resp_body = &resp.into_body().bytes().await?; + let resp_body = &resp.into_body(); let auth_response = serde_json::from_slice::(resp_body) .map_err(new_json_deserialize_error)?; signer.auth_info = AuthInfo { @@ -125,7 +125,7 @@ impl SeafileCore { match status { StatusCode::OK => { - let resp_body = &resp.into_body().bytes().await?; + let resp_body = &resp.into_body(); let list_library_response = serde_json::from_slice::>(resp_body) .map_err(new_json_deserialize_error)?; @@ -174,7 +174,7 @@ impl SeafileCore { match status { StatusCode::OK => { - let resp_body = &resp.into_body().bytes().await?; + let resp_body = &resp.into_body(); let upload_url = serde_json::from_slice::(resp_body) .map_err(new_json_deserialize_error)?; Ok(upload_url) @@ -205,7 +205,7 @@ impl SeafileCore { match status { StatusCode::OK => { - let resp_body = &resp.into_body().bytes().await?; + let resp_body = &resp.into_body(); let download_url = serde_json::from_slice::(resp_body) .map_err(new_json_deserialize_error)?; @@ -216,7 +216,7 @@ impl SeafileCore { } /// download file - pub async fn download_file(&self, path: &str) -> Result> { + pub async fn download_file(&self, path: &str) -> Result> { let download_url = self.get_download_url(path).await?; let req = Request::get(download_url); @@ -256,7 +256,7 @@ impl SeafileCore { match status { StatusCode::OK => { - let resp_body = &resp.into_body().bytes().await?; + let resp_body = &resp.into_body(); let file_detail = serde_json::from_slice::(resp_body) .map_err(new_json_deserialize_error)?; Ok(file_detail) @@ -287,7 +287,7 @@ impl SeafileCore { match status { StatusCode::OK => { - let resp_body = &resp.into_body().bytes().await?; + let resp_body = &resp.into_body(); let dir_detail = serde_json::from_slice::(resp_body) .map_err(new_json_deserialize_error)?; Ok(dir_detail) diff --git a/core/src/services/seafile/error.rs b/core/src/services/seafile/error.rs index 05a0fed99919..dba17133d78a 100644 --- a/core/src/services/seafile/error.rs +++ b/core/src/services/seafile/error.rs @@ -32,9 +32,9 @@ struct SeafileError { } /// Parse error response into Error. -pub async fn parse_error(resp: Response) -> Result { +pub async fn parse_error(resp: Response) -> Result { let (parts, body) = resp.into_parts(); - let bs = body.bytes().await?; + let bs = body.copy_to_bytes(body.remaining()); let (kind, _retryable) = match parts.status.as_u16() { 400 => (ErrorKind::InvalidInput, false), @@ -79,7 +79,7 @@ mod test { for res in err_res { let bs = bytes::Bytes::from(res.0); - let body = IncomingAsyncBody::new( + let body = oio::Buffer::new( Box::new(oio::into_stream(stream::iter(vec![Ok(bs.clone())]))), None, ); diff --git a/core/src/services/seafile/lister.rs b/core/src/services/seafile/lister.rs index ff687981ffe9..990973af1f46 100644 --- a/core/src/services/seafile/lister.rs +++ b/core/src/services/seafile/lister.rs @@ -69,7 +69,7 @@ impl oio::PageList for SeafileLister { match status { StatusCode::OK => { - let resp_body = &resp.into_body().bytes().await?; + let resp_body = &resp.into_body(); let infos = serde_json::from_slice::>(resp_body) .map_err(new_json_deserialize_error)?; diff --git a/core/src/services/seafile/writer.rs b/core/src/services/seafile/writer.rs index dccba7283072..ad7cc98b2e6a 100644 --- a/core/src/services/seafile/writer.rs +++ b/core/src/services/seafile/writer.rs @@ -80,10 +80,7 @@ impl oio::OneShotWrite for SeafileWriter { let status = resp.status(); match status { - StatusCode::OK => { - resp.into_body().consume().await?; - Ok(()) - } + StatusCode::OK => Ok(()), _ => Err(parse_error(resp).await?), } } diff --git a/core/src/services/supabase/backend.rs b/core/src/services/supabase/backend.rs index 99f04723ad90..08b9245b69d5 100644 --- a/core/src/services/supabase/backend.rs +++ b/core/src/services/supabase/backend.rs @@ -156,7 +156,7 @@ pub struct SupabaseBackend { #[async_trait] impl Accessor for SupabaseBackend { - type Reader = IncomingAsyncBody; + type Reader = oio::Buffer; type Writer = oio::OneShotWriter; // todo: implement Lister to support list and scan type Lister = (); diff --git a/core/src/services/supabase/core.rs b/core/src/services/supabase/core.rs index ccdb2e1abad0..628af9e601b4 100644 --- a/core/src/services/supabase/core.rs +++ b/core/src/services/supabase/core.rs @@ -221,7 +221,7 @@ impl SupabaseCore { // core utils impl SupabaseCore { - pub async fn send(&self, req: Request) -> Result> { + pub async fn send(&self, req: Request) -> Result> { self.http_client.send(req).await } @@ -229,7 +229,7 @@ impl SupabaseCore { &self, path: &str, range: BytesRange, - ) -> Result> { + ) -> Result> { let mut req = if self.key.is_some() { self.supabase_get_object_auth_request(path, range)? } else { @@ -239,7 +239,7 @@ impl SupabaseCore { self.send(req).await } - pub async fn supabase_head_object(&self, path: &str) -> Result> { + pub async fn supabase_head_object(&self, path: &str) -> Result> { let mut req = if self.key.is_some() { self.supabase_head_object_auth_request(path)? } else { @@ -249,10 +249,7 @@ impl SupabaseCore { self.send(req).await } - pub async fn supabase_get_object_info( - &self, - path: &str, - ) -> Result> { + pub async fn supabase_get_object_info(&self, path: &str) -> Result> { let mut req = if self.key.is_some() { self.supabase_get_object_info_auth_request(path)? } else { @@ -262,7 +259,7 @@ impl SupabaseCore { self.send(req).await } - pub async fn supabase_delete_object(&self, path: &str) -> Result> { + pub async fn supabase_delete_object(&self, path: &str) -> Result> { let mut req = self.supabase_delete_object_request(path)?; self.sign(&mut req)?; self.send(req).await diff --git a/core/src/services/supabase/error.rs b/core/src/services/supabase/error.rs index 58288959b001..239090c212bc 100644 --- a/core/src/services/supabase/error.rs +++ b/core/src/services/supabase/error.rs @@ -35,9 +35,9 @@ struct SupabaseError { } /// Parse the supabase error type to the OpenDAL error type -pub async fn parse_error(resp: Response) -> Result { +pub async fn parse_error(resp: Response) -> Result { let (parts, body) = resp.into_parts(); - let bs = body.bytes().await?; + let bs = body.copy_to_bytes(body.remaining()); // Check HTTP status code first/ let (mut kind, mut retryable) = match parts.status.as_u16() { diff --git a/core/src/services/supabase/writer.rs b/core/src/services/supabase/writer.rs index e5a753942eb5..7268c150af78 100644 --- a/core/src/services/supabase/writer.rs +++ b/core/src/services/supabase/writer.rs @@ -57,10 +57,7 @@ impl oio::OneShotWrite for SupabaseWriter { let resp = self.core.send(req).await?; match resp.status() { - StatusCode::OK => { - resp.into_body().consume().await?; - Ok(()) - } + StatusCode::OK => Ok(()), _ => Err(parse_error(resp).await?), } } diff --git a/core/src/services/swift/backend.rs b/core/src/services/swift/backend.rs index 393e3038729b..e8ff32606329 100644 --- a/core/src/services/swift/backend.rs +++ b/core/src/services/swift/backend.rs @@ -190,7 +190,7 @@ pub struct SwiftBackend { #[async_trait] impl Accessor for SwiftBackend { - type Reader = IncomingAsyncBody; + type Reader = oio::Buffer; type Writer = oio::OneShotWriter; type Lister = oio::PageLister; type BlockingReader = (); @@ -247,8 +247,7 @@ impl Accessor for SwiftBackend { )) } StatusCode::RANGE_NOT_SATISFIABLE => { - resp.into_body().consume().await?; - Ok((RpRead::new().with_size(Some(0)), IncomingAsyncBody::empty())) + Ok((RpRead::new().with_size(Some(0)), oio::Buffer::empty())) } _ => Err(parse_error(resp).await?), } @@ -293,10 +292,7 @@ impl Accessor for SwiftBackend { let status = resp.status(); match status { - StatusCode::CREATED | StatusCode::OK => { - resp.into_body().consume().await?; - Ok(RpCopy::default()) - } + StatusCode::CREATED | StatusCode::OK => Ok(RpCopy::default()), _ => Err(parse_error(resp).await?), } } diff --git a/core/src/services/swift/core.rs b/core/src/services/swift/core.rs index 1eb0f3d2a94e..bfc8b35ecc95 100644 --- a/core/src/services/swift/core.rs +++ b/core/src/services/swift/core.rs @@ -44,7 +44,7 @@ impl Debug for SwiftCore { } impl SwiftCore { - pub async fn swift_delete(&self, path: &str) -> Result> { + pub async fn swift_delete(&self, path: &str) -> Result> { let p = build_abs_path(&self.root, path); let url = format!( @@ -71,7 +71,7 @@ impl SwiftCore { delimiter: &str, limit: Option, marker: &str, - ) -> Result> { + ) -> Result> { let p = build_abs_path(&self.root, path); // The delimiter is used to disable recursive listing. @@ -107,7 +107,7 @@ impl SwiftCore { path: &str, length: u64, body: AsyncBody, - ) -> Result> { + ) -> Result> { let p = build_abs_path(&self.root, path); let url = format!( @@ -127,7 +127,7 @@ impl SwiftCore { self.client.send(req).await } - pub async fn swift_read(&self, path: &str, arg: OpRead) -> Result> { + pub async fn swift_read(&self, path: &str, arg: OpRead) -> Result> { let range = arg.range(); let p = build_abs_path(&self.root, path) @@ -156,11 +156,7 @@ impl SwiftCore { self.client.send(req).await } - pub async fn swift_copy( - &self, - src_p: &str, - dst_p: &str, - ) -> Result> { + pub async fn swift_copy(&self, src_p: &str, dst_p: &str) -> Result> { // NOTE: current implementation is limited to same container and root let src_p = format!( @@ -197,7 +193,7 @@ impl SwiftCore { self.client.send(req).await } - pub async fn swift_get_metadata(&self, path: &str) -> Result> { + pub async fn swift_get_metadata(&self, path: &str) -> Result> { let p = build_abs_path(&self.root, path); let url = format!( diff --git a/core/src/services/swift/error.rs b/core/src/services/swift/error.rs index 836787ec0029..c73a43c01ba3 100644 --- a/core/src/services/swift/error.rs +++ b/core/src/services/swift/error.rs @@ -34,9 +34,9 @@ struct ErrorResponse { p: String, } -pub async fn parse_error(resp: Response) -> Result { +pub async fn parse_error(resp: Response) -> Result { let (parts, body) = resp.into_parts(); - let bs = body.bytes().await?; + let bs = body.copy_to_bytes(body.remaining()); let (kind, retryable) = match parts.status { StatusCode::NOT_FOUND => (ErrorKind::NotFound, false), diff --git a/core/src/services/swift/writer.rs b/core/src/services/swift/writer.rs index 6004580c7e5b..c63a4fbaa787 100644 --- a/core/src/services/swift/writer.rs +++ b/core/src/services/swift/writer.rs @@ -47,10 +47,7 @@ impl oio::OneShotWrite for SwiftWriter { let status = resp.status(); match status { - StatusCode::CREATED | StatusCode::OK => { - resp.into_body().consume().await?; - Ok(()) - } + StatusCode::CREATED | StatusCode::OK => Ok(()), _ => Err(parse_error(resp).await?), } } diff --git a/core/src/services/upyun/backend.rs b/core/src/services/upyun/backend.rs index c198bb32b50c..2b5fa84c1104 100644 --- a/core/src/services/upyun/backend.rs +++ b/core/src/services/upyun/backend.rs @@ -233,7 +233,7 @@ pub struct UpyunBackend { #[async_trait] impl Accessor for UpyunBackend { - type Reader = IncomingAsyncBody; + type Reader = oio::Buffer; type Writer = UpyunWriters; type Lister = oio::PageLister; type BlockingReader = (); @@ -348,11 +348,7 @@ impl Accessor for UpyunBackend { let status = resp.status(); match status { - StatusCode::OK => { - resp.into_body().consume().await?; - - Ok(RpCopy::default()) - } + StatusCode::OK => Ok(RpCopy::default()), _ => Err(parse_error(resp).await?), } } @@ -363,11 +359,7 @@ impl Accessor for UpyunBackend { let status = resp.status(); match status { - StatusCode::OK => { - resp.into_body().consume().await?; - - Ok(RpRename::default()) - } + StatusCode::OK => Ok(RpRename::default()), _ => Err(parse_error(resp).await?), } } diff --git a/core/src/services/upyun/core.rs b/core/src/services/upyun/core.rs index 49b430768f71..ea0b648d9817 100644 --- a/core/src/services/upyun/core.rs +++ b/core/src/services/upyun/core.rs @@ -82,7 +82,7 @@ impl Debug for UpyunCore { impl UpyunCore { #[inline] - pub async fn send(&self, req: Request) -> Result> { + pub async fn send(&self, req: Request) -> Result> { self.client.send(req).await } @@ -104,7 +104,7 @@ impl UpyunCore { } impl UpyunCore { - pub async fn download_file(&self, path: &str) -> Result> { + pub async fn download_file(&self, path: &str) -> Result> { let path = build_abs_path(&self.root, path); let url = format!( @@ -124,7 +124,7 @@ impl UpyunCore { self.send(req).await } - pub async fn info(&self, path: &str) -> Result> { + pub async fn info(&self, path: &str) -> Result> { let path = build_abs_path(&self.root, path); let url = format!( @@ -185,7 +185,7 @@ impl UpyunCore { Ok(req) } - pub async fn delete(&self, path: &str) -> Result> { + pub async fn delete(&self, path: &str) -> Result> { let path = build_abs_path(&self.root, path); let url = format!( @@ -205,7 +205,7 @@ impl UpyunCore { self.send(req).await } - pub async fn copy(&self, from: &str, to: &str) -> Result> { + pub async fn copy(&self, from: &str, to: &str) -> Result> { let from = format!("/{}/{}", self.bucket, build_abs_path(&self.root, from)); let to = build_abs_path(&self.root, to); @@ -233,7 +233,7 @@ impl UpyunCore { self.send(req).await } - pub async fn move_object(&self, from: &str, to: &str) -> Result> { + pub async fn move_object(&self, from: &str, to: &str) -> Result> { let from = format!("/{}/{}", self.bucket, build_abs_path(&self.root, from)); let to = build_abs_path(&self.root, to); @@ -261,7 +261,7 @@ impl UpyunCore { self.send(req).await } - pub async fn create_dir(&self, path: &str) -> Result> { + pub async fn create_dir(&self, path: &str) -> Result> { let path = build_abs_path(&self.root, path); let path = path[..path.len() - 1].to_string(); @@ -290,7 +290,7 @@ impl UpyunCore { &self, path: &str, args: &OpWrite, - ) -> Result> { + ) -> Result> { let path = build_abs_path(&self.root, path); let url = format!( @@ -364,7 +364,7 @@ impl UpyunCore { &self, path: &str, upload_id: &str, - ) -> Result> { + ) -> Result> { let p = build_abs_path(&self.root, path); let url = format!( @@ -393,7 +393,7 @@ impl UpyunCore { path: &str, iter: &str, limit: Option, - ) -> Result> { + ) -> Result> { let path = build_abs_path(&self.root, path); let url = format!( diff --git a/core/src/services/upyun/error.rs b/core/src/services/upyun/error.rs index 51f4934b11b2..63614e8a9659 100644 --- a/core/src/services/upyun/error.rs +++ b/core/src/services/upyun/error.rs @@ -35,9 +35,9 @@ struct UpyunError { } /// Parse error response into Error. -pub async fn parse_error(resp: Response) -> Result { +pub async fn parse_error(resp: Response) -> Result { let (parts, body) = resp.into_parts(); - let bs = body.bytes().await?; + let bs = body.copy_to_bytes(body.remaining()); let (kind, retryable) = match parts.status.as_u16() { 403 => (ErrorKind::PermissionDenied, false), @@ -89,7 +89,7 @@ mod test { for res in err_res { let bs = bytes::Bytes::from(res.0); - let body = IncomingAsyncBody::new( + let body = oio::Buffer::new( Box::new(oio::into_stream(stream::iter(vec![Ok(bs.clone())]))), None, ); diff --git a/core/src/services/upyun/lister.rs b/core/src/services/upyun/lister.rs index 547facbf7508..e7a6e4ea16ad 100644 --- a/core/src/services/upyun/lister.rs +++ b/core/src/services/upyun/lister.rs @@ -66,7 +66,7 @@ impl oio::PageList for UpyunLister { } } - let bs = resp.into_body().bytes().await?; + let bs = resp.into_body(); let response = serde_json::from_slice::(&bs) .map_err(new_json_deserialize_error)?; diff --git a/core/src/services/upyun/writer.rs b/core/src/services/upyun/writer.rs index 833fff34f3ba..fc05fbf818ea 100644 --- a/core/src/services/upyun/writer.rs +++ b/core/src/services/upyun/writer.rs @@ -51,10 +51,7 @@ impl oio::MultipartWrite for UpyunWriter { let status = resp.status(); match status { - StatusCode::OK => { - resp.into_body().consume().await?; - Ok(()) - } + StatusCode::OK => Ok(()), _ => Err(parse_error(resp).await?), } } @@ -98,14 +95,10 @@ impl oio::MultipartWrite for UpyunWriter { let status = resp.status(); match status { - StatusCode::NO_CONTENT | StatusCode::CREATED => { - resp.into_body().consume().await?; - - Ok(oio::MultipartPart { - part_number, - etag: "".to_string(), - }) - } + StatusCode::NO_CONTENT | StatusCode::CREATED => Ok(oio::MultipartPart { + part_number, + etag: "".to_string(), + }), _ => Err(parse_error(resp).await?), } } @@ -119,11 +112,7 @@ impl oio::MultipartWrite for UpyunWriter { let status = resp.status(); match status { - StatusCode::NO_CONTENT => { - resp.into_body().consume().await?; - - Ok(()) - } + StatusCode::NO_CONTENT => Ok(()), _ => Err(parse_error(resp).await?), } } diff --git a/core/src/services/vercel_artifacts/backend.rs b/core/src/services/vercel_artifacts/backend.rs index 27ef788e19bc..177f234901b3 100644 --- a/core/src/services/vercel_artifacts/backend.rs +++ b/core/src/services/vercel_artifacts/backend.rs @@ -45,7 +45,7 @@ impl Debug for VercelArtifactsBackend { #[async_trait] impl Accessor for VercelArtifactsBackend { - type Reader = IncomingAsyncBody; + type Reader = oio::Buffer; type Writer = oio::OneShotWriter; type Lister = (); type BlockingReader = (); @@ -113,7 +113,7 @@ impl VercelArtifactsBackend { &self, hash: &str, args: OpRead, - ) -> Result> { + ) -> Result> { let url: String = format!( "https://api.vercel.com/v8/artifacts/{}", percent_encode_path(hash) @@ -140,7 +140,7 @@ impl VercelArtifactsBackend { hash: &str, size: u64, body: AsyncBody, - ) -> Result> { + ) -> Result> { let url = format!( "https://api.vercel.com/v8/artifacts/{}", percent_encode_path(hash) @@ -158,7 +158,7 @@ impl VercelArtifactsBackend { self.client.send(req).await } - pub async fn vercel_artifacts_stat(&self, hash: &str) -> Result> { + pub async fn vercel_artifacts_stat(&self, hash: &str) -> Result> { let url = format!( "https://api.vercel.com/v8/artifacts/{}", percent_encode_path(hash) diff --git a/core/src/services/vercel_artifacts/error.rs b/core/src/services/vercel_artifacts/error.rs index 60de6ae0d0a9..1c8b8218c6c2 100644 --- a/core/src/services/vercel_artifacts/error.rs +++ b/core/src/services/vercel_artifacts/error.rs @@ -24,9 +24,9 @@ use crate::ErrorKind; use crate::Result; /// Parse error response into Error. -pub async fn parse_error(resp: Response) -> Result { +pub async fn parse_error(resp: Response) -> Result { let (parts, body) = resp.into_parts(); - let bs = body.bytes().await?; + let bs = body.copy_to_bytes(body.remaining()); let (kind, retryable) = match parts.status { StatusCode::NOT_FOUND => (ErrorKind::NotFound, false), diff --git a/core/src/services/vercel_artifacts/writer.rs b/core/src/services/vercel_artifacts/writer.rs index e48b8fe5bb78..b5d804dab14e 100644 --- a/core/src/services/vercel_artifacts/writer.rs +++ b/core/src/services/vercel_artifacts/writer.rs @@ -50,10 +50,7 @@ impl oio::OneShotWrite for VercelArtifactsWriter { let status = resp.status(); match status { - StatusCode::OK | StatusCode::ACCEPTED => { - resp.into_body().consume().await?; - Ok(()) - } + StatusCode::OK | StatusCode::ACCEPTED => Ok(()), _ => Err(parse_error(resp).await?), } } diff --git a/core/src/services/vercel_blob/backend.rs b/core/src/services/vercel_blob/backend.rs index 4f63fc0362a7..9bec6341a3b6 100644 --- a/core/src/services/vercel_blob/backend.rs +++ b/core/src/services/vercel_blob/backend.rs @@ -178,7 +178,7 @@ pub struct VercelBlobBackend { #[async_trait] impl Accessor for VercelBlobBackend { - type Reader = IncomingAsyncBody; + type Reader = oio::Buffer; type Writer = VercelBlobWriters; type Lister = oio::PageLister; type BlockingReader = (); @@ -220,7 +220,7 @@ impl Accessor for VercelBlobBackend { match status { StatusCode::OK => { - let bs = resp.into_body().bytes().await?; + let bs = resp.into_body(); let resp: Blob = serde_json::from_slice(&bs).map_err(new_json_deserialize_error)?; @@ -267,11 +267,7 @@ impl Accessor for VercelBlobBackend { let status = resp.status(); match status { - StatusCode::OK => { - resp.into_body().consume().await?; - - Ok(RpCopy::default()) - } + StatusCode::OK => Ok(RpCopy::default()), _ => Err(parse_error(resp).await?), } } diff --git a/core/src/services/vercel_blob/core.rs b/core/src/services/vercel_blob/core.rs index e1c1b446d680..0b52bc9655aa 100644 --- a/core/src/services/vercel_blob/core.rs +++ b/core/src/services/vercel_blob/core.rs @@ -73,7 +73,7 @@ impl Debug for VercelBlobCore { impl VercelBlobCore { #[inline] - pub async fn send(&self, req: Request) -> Result> { + pub async fn send(&self, req: Request) -> Result> { self.client.send(req).await } @@ -83,7 +83,7 @@ impl VercelBlobCore { } impl VercelBlobCore { - pub async fn download(&self, path: &str, args: OpRead) -> Result> { + pub async fn download(&self, path: &str, args: OpRead) -> Result> { let p = build_abs_path(&self.root, path); // Vercel blob use an unguessable random id url to download the file // So we use list to get the url of the file and then use it to download the file @@ -179,7 +179,7 @@ impl VercelBlobCore { } } - pub async fn head(&self, path: &str) -> Result> { + pub async fn head(&self, path: &str) -> Result> { let p = build_abs_path(&self.root, path); let resp = self.list(&p, Some(1)).await?; @@ -205,7 +205,7 @@ impl VercelBlobCore { self.send(req).await } - pub async fn copy(&self, from: &str, to: &str) -> Result> { + pub async fn copy(&self, from: &str, to: &str) -> Result> { let from = build_abs_path(&self.root, from); let resp = self.list(&from, Some(1)).await?; @@ -277,7 +277,7 @@ impl VercelBlobCore { &self, path: &str, args: &OpWrite, - ) -> Result> { + ) -> Result> { let p = build_abs_path(&self.root, path); let url = format!( @@ -311,7 +311,7 @@ impl VercelBlobCore { part_number: usize, size: u64, body: AsyncBody, - ) -> Result> { + ) -> Result> { let p = build_abs_path(&self.root, path); let url = format!( @@ -340,7 +340,7 @@ impl VercelBlobCore { path: &str, upload_id: &str, parts: Vec, - ) -> Result> { + ) -> Result> { let p = build_abs_path(&self.root, path); let url = format!( diff --git a/core/src/services/vercel_blob/error.rs b/core/src/services/vercel_blob/error.rs index 2f8f5ef600f0..b781183d5434 100644 --- a/core/src/services/vercel_blob/error.rs +++ b/core/src/services/vercel_blob/error.rs @@ -40,9 +40,9 @@ struct VercelBlobErrorDetail { } /// Parse error response into Error. -pub async fn parse_error(resp: Response) -> Result { +pub async fn parse_error(resp: Response) -> Result { let (parts, body) = resp.into_parts(); - let bs = body.bytes().await?; + let bs = body.copy_to_bytes(body.remaining()); let (kind, retryable) = match parts.status.as_u16() { 403 => (ErrorKind::PermissionDenied, false), @@ -88,7 +88,7 @@ mod test { for res in err_res { let bs = bytes::Bytes::from(res.0); - let body = IncomingAsyncBody::new( + let body = oio::Buffer::new( Box::new(oio::into_stream(stream::iter(vec![Ok(bs.clone())]))), None, ); diff --git a/core/src/services/vercel_blob/writer.rs b/core/src/services/vercel_blob/writer.rs index c3bcbf87cc63..905d50a424bb 100644 --- a/core/src/services/vercel_blob/writer.rs +++ b/core/src/services/vercel_blob/writer.rs @@ -53,10 +53,7 @@ impl oio::MultipartWrite for VercelBlobWriter { let status = resp.status(); match status { - StatusCode::OK => { - resp.into_body().consume().await?; - Ok(()) - } + StatusCode::OK => Ok(()), _ => Err(parse_error(resp).await?), } } @@ -71,7 +68,7 @@ impl oio::MultipartWrite for VercelBlobWriter { match status { StatusCode::OK => { - let bs = resp.into_body().bytes().await?; + let bs = resp.into_body(); let resp = serde_json::from_slice::(&bs) .map_err(new_json_deserialize_error)?; @@ -100,7 +97,7 @@ impl oio::MultipartWrite for VercelBlobWriter { match status { StatusCode::OK => { - let bs = resp.into_body().bytes().await?; + let bs = resp.into_body(); let resp = serde_json::from_slice::(&bs) .map_err(new_json_deserialize_error)?; @@ -131,11 +128,7 @@ impl oio::MultipartWrite for VercelBlobWriter { let status = resp.status(); match status { - StatusCode::OK => { - resp.into_body().consume().await?; - - Ok(()) - } + StatusCode::OK => Ok(()), _ => Err(parse_error(resp).await?), } } diff --git a/core/src/services/webdav/backend.rs b/core/src/services/webdav/backend.rs index ed358b943f5c..447c6c958317 100644 --- a/core/src/services/webdav/backend.rs +++ b/core/src/services/webdav/backend.rs @@ -237,7 +237,7 @@ impl Debug for WebdavBackend { #[async_trait] impl Accessor for WebdavBackend { - type Reader = IncomingAsyncBody; + type Reader = oio::Buffer; type Writer = oio::OneShotWriter; type Lister = oio::PageLister; type BlockingReader = (); @@ -297,8 +297,7 @@ impl Accessor for WebdavBackend { )) } StatusCode::RANGE_NOT_SATISFIABLE => { - resp.into_body().consume().await?; - Ok((RpRead::new().with_size(Some(0)), IncomingAsyncBody::empty())) + Ok((RpRead::new().with_size(Some(0)), oio::Buffer::empty())) } _ => Err(parse_error(resp).await?), } diff --git a/core/src/services/webdav/core.rs b/core/src/services/webdav/core.rs index 9f24e5f0609f..5557525a6258 100644 --- a/core/src/services/webdav/core.rs +++ b/core/src/services/webdav/core.rs @@ -116,7 +116,7 @@ impl WebdavCore { return Err(parse_error(resp).await?); } - let bs = resp.into_body().bytes().await?; + let bs = resp.into_body(); let result: Multistatus = deserialize_multistatus(&bs)?; let propfind_resp = result.response.first().ok_or_else(|| { @@ -130,11 +130,7 @@ impl WebdavCore { Ok(metadata) } - pub async fn webdav_get( - &self, - path: &str, - args: OpRead, - ) -> Result> { + pub async fn webdav_get(&self, path: &str, args: OpRead) -> Result> { let path = build_rooted_abs_path(&self.root, path); let url: String = format!("{}{}", self.endpoint, percent_encode_path(&path)); @@ -162,7 +158,7 @@ impl WebdavCore { size: Option, args: &OpWrite, body: AsyncBody, - ) -> Result> { + ) -> Result> { let path = build_rooted_abs_path(&self.root, path); let url = format!("{}{}", self.endpoint, percent_encode_path(&path)); @@ -189,7 +185,7 @@ impl WebdavCore { self.client.send(req).await } - pub async fn webdav_delete(&self, path: &str) -> Result> { + pub async fn webdav_delete(&self, path: &str) -> Result> { let path = build_rooted_abs_path(&self.root, path); let url = format!("{}{}", self.endpoint, percent_encode_path(&path)); @@ -206,7 +202,7 @@ impl WebdavCore { self.client.send(req).await } - pub async fn webdav_copy(&self, from: &str, to: &str) -> Result> { + pub async fn webdav_copy(&self, from: &str, to: &str) -> Result> { // Check if source file exists. let _ = self.webdav_stat(from).await?; // Make sure target's dir is exist. @@ -234,7 +230,7 @@ impl WebdavCore { self.client.send(req).await } - pub async fn webdav_move(&self, from: &str, to: &str) -> Result> { + pub async fn webdav_move(&self, from: &str, to: &str) -> Result> { // Check if source file exists. let _ = self.webdav_stat(from).await?; // Make sure target's dir is exist. @@ -262,11 +258,7 @@ impl WebdavCore { self.client.send(req).await } - pub async fn webdav_list( - &self, - path: &str, - args: &OpList, - ) -> Result> { + pub async fn webdav_list(&self, path: &str, args: &OpList) -> Result> { let path = build_rooted_abs_path(&self.root, path); let url = format!("{}{}", self.endpoint, percent_encode_path(&path)); @@ -357,7 +349,7 @@ impl WebdavCore { // The MKCOL method can only be performed on a deleted or non-existent resource. // This error means the directory already exists which is allowed by create_dir. | StatusCode::METHOD_NOT_ALLOWED => { - resp.into_body().consume().await?; + Ok(()) } _ => Err(parse_error(resp).await?), diff --git a/core/src/services/webdav/error.rs b/core/src/services/webdav/error.rs index 967dc19f52a6..ee3210a4da0c 100644 --- a/core/src/services/webdav/error.rs +++ b/core/src/services/webdav/error.rs @@ -24,9 +24,9 @@ use crate::ErrorKind; use crate::Result; /// Parse error response into Error. -pub async fn parse_error(resp: Response) -> Result { +pub async fn parse_error(resp: Response) -> Result { let (parts, body) = resp.into_parts(); - let bs = body.bytes().await?; + let bs = body.copy_to_bytes(body.remaining()); let (kind, retryable) = match parts.status { StatusCode::NOT_FOUND => (ErrorKind::NotFound, false), diff --git a/core/src/services/webdav/writer.rs b/core/src/services/webdav/writer.rs index 525928adcd52..d51fd053a6f2 100644 --- a/core/src/services/webdav/writer.rs +++ b/core/src/services/webdav/writer.rs @@ -54,10 +54,7 @@ impl oio::OneShotWrite for WebdavWriter { let status = resp.status(); match status { - StatusCode::CREATED | StatusCode::OK | StatusCode::NO_CONTENT => { - resp.into_body().consume().await?; - Ok(()) - } + StatusCode::CREATED | StatusCode::OK | StatusCode::NO_CONTENT => Ok(()), _ => Err(parse_error(resp).await?), } } diff --git a/core/src/services/webhdfs/backend.rs b/core/src/services/webhdfs/backend.rs index 7944b03e3c2f..eff6dddffa76 100644 --- a/core/src/services/webhdfs/backend.rs +++ b/core/src/services/webhdfs/backend.rs @@ -267,7 +267,7 @@ impl WebhdfsBackend { return Err(parse_error(resp).await?); } - let bs = resp.into_body().bytes().await?; + let bs = resp.into_body(); let resp = serde_json::from_slice::(&bs).map_err(new_json_deserialize_error)?; @@ -307,7 +307,7 @@ impl WebhdfsBackend { match status { StatusCode::OK => { - let bs = resp.into_body().bytes().await?; + let bs = resp.into_body(); let resp: LocationResponse = serde_json::from_slice(&bs).map_err(new_json_deserialize_error)?; @@ -321,7 +321,7 @@ impl WebhdfsBackend { &self, from: &str, to: &str, - ) -> Result> { + ) -> Result> { let from = build_abs_path(&self.root, from); let to = build_rooted_abs_path(&self.root, to); @@ -431,10 +431,7 @@ impl WebhdfsBackend { Ok(req) } - pub async fn webhdfs_list_status_request( - &self, - path: &str, - ) -> Result> { + pub async fn webhdfs_list_status_request(&self, path: &str) -> Result> { let p = build_abs_path(&self.root, path); let mut url = format!( "{}/webhdfs/v1/{}?op=LISTSTATUS", @@ -455,7 +452,7 @@ impl WebhdfsBackend { &self, path: &str, start_after: &str, - ) -> Result> { + ) -> Result> { let p = build_abs_path(&self.root, path); let mut url = format!( @@ -480,7 +477,7 @@ impl WebhdfsBackend { &self, path: &str, range: BytesRange, - ) -> Result> { + ) -> Result> { let req = self.webhdfs_open_request(path, &range).await?; self.client.send(req).await } @@ -488,7 +485,7 @@ impl WebhdfsBackend { pub(super) async fn webhdfs_get_file_status( &self, path: &str, - ) -> Result> { + ) -> Result> { let p = build_abs_path(&self.root, path); let mut url = format!( "{}/webhdfs/v1/{}?op=GETFILESTATUS", @@ -507,7 +504,7 @@ impl WebhdfsBackend { self.client.send(req).await } - pub async fn webhdfs_delete(&self, path: &str) -> Result> { + pub async fn webhdfs_delete(&self, path: &str) -> Result> { let p = build_abs_path(&self.root, path); let mut url = format!( "{}/webhdfs/v1/{}?op=DELETE&recursive=false", @@ -529,7 +526,7 @@ impl WebhdfsBackend { let resp = self.webhdfs_get_file_status("/").await?; match resp.status() { StatusCode::OK => { - let bs = resp.into_body().bytes().await?; + let bs = resp.into_body(); let file_status = serde_json::from_slice::(&bs) .map_err(new_json_deserialize_error)? @@ -553,7 +550,7 @@ impl WebhdfsBackend { #[async_trait] impl Accessor for WebhdfsBackend { - type Reader = IncomingAsyncBody; + type Reader = oio::Buffer; type Writer = WebhdfsWriters; type Lister = oio::PageLister; type BlockingReader = (); @@ -599,7 +596,7 @@ impl Accessor for WebhdfsBackend { // the redirection should be done automatically. match status { StatusCode::CREATED | StatusCode::OK => { - let bs = resp.into_body().bytes().await?; + let bs = resp.into_body(); let resp = serde_json::from_slice::(&bs) .map_err(new_json_deserialize_error)?; @@ -627,7 +624,7 @@ impl Accessor for WebhdfsBackend { let status = resp.status(); match status { StatusCode::OK => { - let bs = resp.into_body().bytes().await?; + let bs = resp.into_body(); let file_status = serde_json::from_slice::(&bs) .map_err(new_json_deserialize_error)? @@ -664,17 +661,16 @@ impl Accessor for WebhdfsBackend { // WebHDFS will returns 403 when range is outside of the end. StatusCode::FORBIDDEN => { let (parts, body) = resp.into_parts(); - let bs = body.bytes().await?; + let bs = body.copy_to_bytes(body.remaining()); let s = String::from_utf8_lossy(&bs); if s.contains("out of the range") { - Ok((RpRead::new(), IncomingAsyncBody::empty())) + Ok((RpRead::new(), oio::Buffer::empty())) } else { Err(parse_error_msg(parts, &s)?) } } StatusCode::RANGE_NOT_SATISFIABLE => { - resp.into_body().consume().await?; - Ok((RpRead::new().with_size(Some(0)), IncomingAsyncBody::empty())) + Ok((RpRead::new().with_size(Some(0)), oio::Buffer::empty())) } _ => Err(parse_error(resp).await?), } @@ -696,10 +692,7 @@ impl Accessor for WebhdfsBackend { let resp = self.webhdfs_delete(path).await?; match resp.status() { - StatusCode::OK => { - resp.into_body().consume().await?; - Ok(RpDelete::default()) - } + StatusCode::OK => Ok(RpDelete::default()), _ => Err(parse_error(resp).await?), } } diff --git a/core/src/services/webhdfs/error.rs b/core/src/services/webhdfs/error.rs index d243b84811d1..5130c7705a7f 100644 --- a/core/src/services/webhdfs/error.rs +++ b/core/src/services/webhdfs/error.rs @@ -39,9 +39,9 @@ struct WebHdfsError { java_class_name: String, } -pub(super) async fn parse_error(resp: Response) -> Result { +pub(super) async fn parse_error(resp: Response) -> Result { let (parts, body) = resp.into_parts(); - let bs = body.bytes().await?; + let bs = body.copy_to_bytes(body.remaining()); let s = String::from_utf8_lossy(&bs); parse_error_msg(parts, &s) } @@ -99,7 +99,7 @@ mod tests { } "#, ); - let body = IncomingAsyncBody::new( + let body = oio::Buffer::new( Box::new(oio::into_stream(stream::iter(vec![Ok(ill_args.clone())]))), None, ); diff --git a/core/src/services/webhdfs/lister.rs b/core/src/services/webhdfs/lister.rs index cc7db189f257..c9e8036f14b6 100644 --- a/core/src/services/webhdfs/lister.rs +++ b/core/src/services/webhdfs/lister.rs @@ -45,7 +45,7 @@ impl oio::PageList for WebhdfsLister { StatusCode::OK => { ctx.done = true; - let bs = resp.into_body().bytes().await?; + let bs = resp.into_body(); serde_json::from_slice::(&bs) .map_err(new_json_deserialize_error)? .file_statuses @@ -64,7 +64,7 @@ impl oio::PageList for WebhdfsLister { .await?; match resp.status() { StatusCode::OK => { - let bs = resp.into_body().bytes().await?; + let bs = resp.into_body(); let directory_listing = serde_json::from_slice::(&bs) .map_err(new_json_deserialize_error)? .directory_listing; diff --git a/core/src/services/webhdfs/writer.rs b/core/src/services/webhdfs/writer.rs index 02757a57afb9..df553ac10fbb 100644 --- a/core/src/services/webhdfs/writer.rs +++ b/core/src/services/webhdfs/writer.rs @@ -50,10 +50,7 @@ impl oio::BlockWrite for WebhdfsWriter { let status = resp.status(); match status { - StatusCode::CREATED | StatusCode::OK => { - resp.into_body().consume().await?; - Ok(()) - } + StatusCode::CREATED | StatusCode::OK => Ok(()), _ => Err(parse_error(resp).await?), } } @@ -79,10 +76,7 @@ impl oio::BlockWrite for WebhdfsWriter { let status = resp.status(); match status { - StatusCode::CREATED | StatusCode::OK => { - resp.into_body().consume().await?; - Ok(()) - } + StatusCode::CREATED | StatusCode::OK => Ok(()), _ => Err(parse_error(resp).await?), } } @@ -129,10 +123,7 @@ impl oio::BlockWrite for WebhdfsWriter { let status = resp.status(); match status { - StatusCode::OK => { - resp.into_body().consume().await?; - Ok(()) - } + StatusCode::OK => Ok(()), _ => Err(parse_error(resp).await?), } } @@ -141,9 +132,7 @@ impl oio::BlockWrite for WebhdfsWriter { for block_id in block_ids { let resp = self.backend.webhdfs_delete(&block_id.to_string()).await?; match resp.status() { - StatusCode::OK => { - resp.into_body().consume().await?; - } + StatusCode::OK => {} _ => return Err(parse_error(resp).await?), } } @@ -179,8 +168,6 @@ impl oio::AppendWrite for WebhdfsWriter { match status { StatusCode::CREATED | StatusCode::OK => { - resp.into_body().consume().await?; - location = self.backend.webhdfs_init_append_request(&self.path).await?; } _ => return Err(parse_error(resp).await?), @@ -198,10 +185,7 @@ impl oio::AppendWrite for WebhdfsWriter { let status = resp.status(); match status { - StatusCode::OK => { - resp.into_body().consume().await?; - Ok(()) - } + StatusCode::OK => Ok(()), _ => Err(parse_error(resp).await?), } } diff --git a/core/src/services/yandex_disk/backend.rs b/core/src/services/yandex_disk/backend.rs index d581e8fd4d90..4af21a8024bf 100644 --- a/core/src/services/yandex_disk/backend.rs +++ b/core/src/services/yandex_disk/backend.rs @@ -179,7 +179,7 @@ pub struct YandexDiskBackend { #[async_trait] impl Accessor for YandexDiskBackend { - type Reader = IncomingAsyncBody; + type Reader = oio::Buffer; type Writer = YandexDiskWriters; type Lister = oio::PageLister; type BlockingReader = (); @@ -227,11 +227,7 @@ impl Accessor for YandexDiskBackend { let status = resp.status(); match status { - StatusCode::OK | StatusCode::CREATED => { - resp.into_body().consume().await?; - - Ok(RpRename::default()) - } + StatusCode::OK | StatusCode::CREATED => Ok(RpRename::default()), _ => Err(parse_error(resp).await?), } } @@ -244,11 +240,7 @@ impl Accessor for YandexDiskBackend { let status = resp.status(); match status { - StatusCode::OK | StatusCode::CREATED => { - resp.into_body().consume().await?; - - Ok(RpCopy::default()) - } + StatusCode::OK | StatusCode::CREATED => Ok(RpCopy::default()), _ => Err(parse_error(resp).await?), } } @@ -283,7 +275,7 @@ impl Accessor for YandexDiskBackend { match status { StatusCode::OK => { - let bs = resp.into_body().bytes().await?; + let bs = resp.into_body(); let mf: MetainformationResponse = serde_json::from_slice(&bs).map_err(new_json_deserialize_error)?; diff --git a/core/src/services/yandex_disk/core.rs b/core/src/services/yandex_disk/core.rs index f206a52c0eb9..76162913b8fb 100644 --- a/core/src/services/yandex_disk/core.rs +++ b/core/src/services/yandex_disk/core.rs @@ -49,7 +49,7 @@ impl Debug for YandexDiskCore { impl YandexDiskCore { #[inline] - pub async fn send(&self, req: Request) -> Result> { + pub async fn send(&self, req: Request) -> Result> { self.client.send(req).await } @@ -87,7 +87,7 @@ impl YandexDiskCore { match status { StatusCode::OK => { - let bytes = resp.into_body().bytes().await?; + let bytes = resp.into_body(); let resp: GetUploadUrlResponse = serde_json::from_slice(&bytes).map_err(new_json_deserialize_error)?; @@ -121,7 +121,7 @@ impl YandexDiskCore { match status { StatusCode::OK => { - let bytes = resp.into_body().bytes().await?; + let bytes = resp.into_body(); let resp: GetUploadUrlResponse = serde_json::from_slice(&bytes).map_err(new_json_deserialize_error)?; @@ -144,16 +144,14 @@ impl YandexDiskCore { let status = resp.status(); match status { - StatusCode::CREATED | StatusCode::CONFLICT => { - resp.into_body().consume().await?; - } + StatusCode::CREATED | StatusCode::CONFLICT => {} _ => return Err(parse_error(resp).await?), } } Ok(()) } - pub async fn create_dir(&self, path: &str) -> Result> { + pub async fn create_dir(&self, path: &str) -> Result> { let url = format!( "https://cloud-api.yandex.net/v1/disk/resources?path=/{}", percent_encode_path(path), @@ -171,7 +169,7 @@ impl YandexDiskCore { self.send(req).await } - pub async fn copy(&self, from: &str, to: &str) -> Result> { + pub async fn copy(&self, from: &str, to: &str) -> Result> { let from = build_rooted_abs_path(&self.root, from); let to = build_rooted_abs_path(&self.root, to); @@ -193,7 +191,7 @@ impl YandexDiskCore { self.send(req).await } - pub async fn move_object(&self, from: &str, to: &str) -> Result> { + pub async fn move_object(&self, from: &str, to: &str) -> Result> { let from = build_rooted_abs_path(&self.root, from); let to = build_rooted_abs_path(&self.root, to); @@ -215,7 +213,7 @@ impl YandexDiskCore { self.send(req).await } - pub async fn delete(&self, path: &str) -> Result> { + pub async fn delete(&self, path: &str) -> Result> { let path = build_rooted_abs_path(&self.root, path); let url = format!( @@ -240,7 +238,7 @@ impl YandexDiskCore { path: &str, limit: Option, offset: Option, - ) -> Result> { + ) -> Result> { let path = build_rooted_abs_path(&self.root, path); let mut url = format!( diff --git a/core/src/services/yandex_disk/error.rs b/core/src/services/yandex_disk/error.rs index 05020e64d799..170489c676e0 100644 --- a/core/src/services/yandex_disk/error.rs +++ b/core/src/services/yandex_disk/error.rs @@ -35,9 +35,9 @@ struct YandexDiskError { } /// Parse error response into Error. -pub async fn parse_error(resp: Response) -> Result { +pub async fn parse_error(resp: Response) -> Result { let (parts, body) = resp.into_parts(); - let bs = body.bytes().await?; + let bs = body.copy_to_bytes(body.remaining()); let (kind, retryable) = match parts.status.as_u16() { 400 => (ErrorKind::InvalidInput, false), @@ -97,7 +97,7 @@ mod test { for res in err_res { let bs = bytes::Bytes::from(res.0); - let body = IncomingAsyncBody::new( + let body = oio::Buffer::new( Box::new(oio::into_stream(stream::iter(vec![Ok(bs.clone())]))), None, ); diff --git a/core/src/services/yandex_disk/writer.rs b/core/src/services/yandex_disk/writer.rs index 7495f59ce799..0420809d3b79 100644 --- a/core/src/services/yandex_disk/writer.rs +++ b/core/src/services/yandex_disk/writer.rs @@ -54,10 +54,7 @@ impl oio::OneShotWrite for YandexDiskWriter { let status = resp.status(); match status { - StatusCode::CREATED => { - resp.into_body().consume().await?; - Ok(()) - } + StatusCode::CREATED => Ok(()), _ => Err(parse_error(resp).await?), } } From 1f8dd5ce26481c975c15be2722a030bdb551da91 Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Tue, 19 Mar 2024 18:19:43 +0800 Subject: [PATCH 010/111] Fix azdls Signed-off-by: Xuanwo --- core/src/services/azdls/backend.rs | 25 ++++--------- core/src/services/azdls/core.rs | 10 ------ core/src/services/azdls/error.rs | 2 +- core/src/services/azdls/lister.rs | 3 +- core/src/services/azdls/mod.rs | 1 + core/src/services/azdls/reader.rs | 58 ++++++++++++++++++++++++++++++ 6 files changed, 68 insertions(+), 31 deletions(-) create mode 100644 core/src/services/azdls/reader.rs diff --git a/core/src/services/azdls/backend.rs b/core/src/services/azdls/backend.rs index 710b6331ae63..70390dafa555 100644 --- a/core/src/services/azdls/backend.rs +++ b/core/src/services/azdls/backend.rs @@ -34,6 +34,7 @@ use super::lister::AzdlsLister; use super::writer::AzdlsWriter; use super::writer::AzdlsWriters; use crate::raw::*; +use crate::services::azdls::reader::AzdlsReader; use crate::*; /// Known endpoint suffix Azure Data Lake Storage Gen2 URI syntax. @@ -245,7 +246,7 @@ pub struct AzdlsBackend { #[async_trait] impl Accessor for AzdlsBackend { - type Reader = oio::Buffer; + type Reader = AzdlsReader; type Writer = AzdlsWriters; type Lister = oio::PageLister; type BlockingReader = (); @@ -345,24 +346,10 @@ impl Accessor for AzdlsBackend { } async fn read(&self, path: &str, args: OpRead) -> Result<(RpRead, Self::Reader)> { - let resp = self.core.azdls_read(path, args.range()).await?; - - let status = resp.status(); - - match status { - StatusCode::OK | StatusCode::PARTIAL_CONTENT => { - let size = parse_content_length(resp.headers())?; - let range = parse_content_range(resp.headers())?; - Ok(( - RpRead::new().with_size(size).with_range(range), - resp.into_body(), - )) - } - StatusCode::RANGE_NOT_SATISFIABLE => { - Ok((RpRead::new().with_size(Some(0)), oio::Buffer::empty())) - } - _ => Err(parse_error(resp).await?), - } + Ok(( + RpRead::default(), + AzdlsReader::new(self.core.clone(), path, args), + )) } async fn write(&self, path: &str, args: OpWrite) -> Result<(RpWrite, Self::Writer)> { diff --git a/core/src/services/azdls/core.rs b/core/src/services/azdls/core.rs index d14544264e42..5678361903e9 100644 --- a/core/src/services/azdls/core.rs +++ b/core/src/services/azdls/core.rs @@ -110,16 +110,6 @@ impl AzdlsCore { let mut req = Request::get(&url); if !range.is_full() { - // azblob doesn't support read with suffix range. - // - // ref: https://learn.microsoft.com/en-us/rest/api/storageservices/specifying-the-range-header-for-blob-service-operations - if range.offset().is_none() && range.size().is_some() { - return Err(Error::new( - ErrorKind::Unsupported, - "azblob doesn't support read with suffix range", - )); - } - req = req.header(http::header::RANGE, range.to_header()); } diff --git a/core/src/services/azdls/error.rs b/core/src/services/azdls/error.rs index 3e656e78eeb3..77b475c2954d 100644 --- a/core/src/services/azdls/error.rs +++ b/core/src/services/azdls/error.rs @@ -62,7 +62,7 @@ impl Debug for AzdlsError { /// Parse error response into Error. pub async fn parse_error(resp: Response) -> Result { - let (parts, body) = resp.into_parts(); + let (parts, mut body) = resp.into_parts(); let bs = body.copy_to_bytes(body.remaining()); let (kind, retryable) = match parts.status { diff --git a/core/src/services/azdls/lister.rs b/core/src/services/azdls/lister.rs index f76e68602129..a10b41af1b88 100644 --- a/core/src/services/azdls/lister.rs +++ b/core/src/services/azdls/lister.rs @@ -15,6 +15,7 @@ // specific language governing permissions and limitations // under the License. +use bytes::Buf; use std::sync::Arc; use serde::Deserialize; @@ -69,7 +70,7 @@ impl oio::PageList for AzdlsLister { let bs = resp.into_body(); - let output: Output = de::from_slice(&bs).map_err(new_json_deserialize_error)?; + let output: Output = de::from_reader(bs.reader()).map_err(new_json_deserialize_error)?; for object in output.paths { // Azdls will return `"true"` and `"false"` for is_directory. diff --git a/core/src/services/azdls/mod.rs b/core/src/services/azdls/mod.rs index 4284e4595ed0..e407c395eea8 100644 --- a/core/src/services/azdls/mod.rs +++ b/core/src/services/azdls/mod.rs @@ -21,4 +21,5 @@ pub use backend::AzdlsBuilder as Azdls; mod core; mod error; mod lister; +mod reader; mod writer; diff --git a/core/src/services/azdls/reader.rs b/core/src/services/azdls/reader.rs new file mode 100644 index 000000000000..7a42159bd39b --- /dev/null +++ b/core/src/services/azdls/reader.rs @@ -0,0 +1,58 @@ +// 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 super::core::AzdlsCore; +use super::error::parse_error; +use crate::raw::{oio, OpRead}; +use http::StatusCode; +use std::future::Future; +use std::sync::Arc; + +pub struct AzdlsReader { + core: Arc, + + path: String, + op: OpRead, +} + +impl AzdlsReader { + pub fn new(core: Arc, path: &str, op: OpRead) -> Self { + AzdlsReader { + core, + path: path.to_string(), + op: op, + } + } +} + +impl oio::Read for AzdlsReader { + async fn read_at(&self, offset: u64, limit: usize) -> crate::Result { + let Some(range) = self.op.range().apply_on_offset(offset, limit) else { + return Ok(oio::Buffer::new()); + }; + + let resp = self.core.azdls_read(&self.path, range).await?; + + let status = resp.status(); + + match status { + StatusCode::OK | StatusCode::PARTIAL_CONTENT => Ok(resp.into_body()), + StatusCode::RANGE_NOT_SATISFIABLE => Ok(oio::Buffer::new()), + _ => Err(parse_error(resp).await?), + } + } +} From c69fe3a15a38fa0e0ca41b906496ec7c3521b85c Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Tue, 19 Mar 2024 18:52:02 +0800 Subject: [PATCH 011/111] Fix azfile Signed-off-by: Xuanwo --- core/Cargo.toml | 6 ++- core/src/services/azfile/backend.rs | 25 +++---------- core/src/services/azfile/error.rs | 2 +- core/src/services/azfile/lister.rs | 7 ++-- core/src/services/azfile/mod.rs | 1 + core/src/services/azfile/reader.rs | 58 +++++++++++++++++++++++++++++ 6 files changed, 75 insertions(+), 24 deletions(-) create mode 100644 core/src/services/azfile/reader.rs diff --git a/core/Cargo.toml b/core/Cargo.toml index f59a25ab3b55..df57c1399b5d 100644 --- a/core/Cargo.toml +++ b/core/Cargo.toml @@ -104,7 +104,11 @@ services-azdls = [ "reqsign?/services-azblob", "reqsign?/reqwest_request", ] -services-azfile = [] +services-azfile = [ + "dep:reqsign", + "reqsign?/services-azblob", + "reqsign?/reqwest_request", +] services-b2 = [] services-cacache = ["dep:cacache"] services-chainsafe = [] diff --git a/core/src/services/azfile/backend.rs b/core/src/services/azfile/backend.rs index d6f2c1ad5924..4ed4dfe35fce 100644 --- a/core/src/services/azfile/backend.rs +++ b/core/src/services/azfile/backend.rs @@ -34,6 +34,7 @@ use super::writer::AzfileWriter; use super::writer::AzfileWriters; use crate::raw::*; use crate::services::azfile::lister::AzfileLister; +use crate::services::azfile::reader::AzfileReader; use crate::*; /// Default endpoint of Azure File services. @@ -266,7 +267,7 @@ pub struct AzfileBackend { #[async_trait] impl Accessor for AzfileBackend { - type Reader = oio::Buffer; + type Reader = AzfileReader; type Writer = AzfileWriters; type Lister = oio::PageLister; type BlockingReader = (); @@ -343,24 +344,10 @@ impl Accessor for AzfileBackend { } async fn read(&self, path: &str, args: OpRead) -> Result<(RpRead, Self::Reader)> { - let resp = self.core.azfile_read(path, args.range()).await?; - - let status = resp.status(); - - match status { - StatusCode::OK | StatusCode::PARTIAL_CONTENT => { - let size = parse_content_length(resp.headers())?; - let range = parse_content_range(resp.headers())?; - Ok(( - RpRead::new().with_size(size).with_range(range), - resp.into_body(), - )) - } - StatusCode::RANGE_NOT_SATISFIABLE => { - Ok((RpRead::new().with_size(Some(0)), oio::Buffer::empty())) - } - _ => Err(parse_error(resp).await?), - } + Ok(( + RpRead::default(), + AzfileReader::new(self.core.clone(), path, args), + )) } async fn write(&self, path: &str, args: OpWrite) -> Result<(RpWrite, Self::Writer)> { diff --git a/core/src/services/azfile/error.rs b/core/src/services/azfile/error.rs index e5bd569897dc..0e8992b7abe2 100644 --- a/core/src/services/azfile/error.rs +++ b/core/src/services/azfile/error.rs @@ -62,7 +62,7 @@ impl Debug for AzfileError { /// Parse error response into Error. pub async fn parse_error(resp: Response) -> Result { - let (parts, body) = resp.into_parts(); + let (parts, mut body) = resp.into_parts(); let bs = body.copy_to_bytes(body.remaining()); let (kind, retryable) = match parts.status { diff --git a/core/src/services/azfile/lister.rs b/core/src/services/azfile/lister.rs index ac65b123cbcd..daadad24b6c1 100644 --- a/core/src/services/azfile/lister.rs +++ b/core/src/services/azfile/lister.rs @@ -15,9 +15,11 @@ // specific language governing permissions and limitations // under the License. +use bytes::Buf; use std::sync::Arc; use http::StatusCode; +use quick_xml::de; use quick_xml::de::from_str; use serde::Deserialize; @@ -57,9 +59,8 @@ impl oio::PageList for AzfileLister { let bs = resp.into_body(); - let text = String::from_utf8(bs.to_vec()).expect("response convert to string must success"); - - let results: EnumerationResults = from_str(&text).map_err(new_xml_deserialize_error)?; + let results: EnumerationResults = + de::from_reader(bs.reader()).map_err(new_xml_deserialize_error)?; if results.next_marker.is_empty() { ctx.done = true; diff --git a/core/src/services/azfile/mod.rs b/core/src/services/azfile/mod.rs index 0ae9da72918f..a94ed0be8452 100644 --- a/core/src/services/azfile/mod.rs +++ b/core/src/services/azfile/mod.rs @@ -21,4 +21,5 @@ mod backend; mod core; mod error; mod lister; +mod reader; mod writer; diff --git a/core/src/services/azfile/reader.rs b/core/src/services/azfile/reader.rs new file mode 100644 index 000000000000..acbcee310bd3 --- /dev/null +++ b/core/src/services/azfile/reader.rs @@ -0,0 +1,58 @@ +// 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 super::core::AzfileCore; +use super::error::parse_error; +use crate::raw::*; +use crate::*; +use http::StatusCode; +use std::sync::Arc; + +pub struct AzfileReader { + core: Arc, + + path: String, + op: OpRead, +} + +impl AzfileReader { + pub fn new(core: Arc, path: &str, op: OpRead) -> Self { + Self { + core, + path: path.to_string(), + op, + } + } +} + +impl oio::Read for AzfileReader { + async fn read_at(&self, offset: u64, limit: usize) -> Result { + let Some(range) = self.op.range().apply_on_offset(offset, limit) else { + return Ok(oio::Buffer::new()); + }; + + let resp = self.core.azfile_read(&self.path, range).await?; + + let status = resp.status(); + + match status { + StatusCode::OK | StatusCode::PARTIAL_CONTENT => Ok(resp.into_body()), + StatusCode::RANGE_NOT_SATISFIABLE => Ok(oio::Buffer::new()), + _ => Err(parse_error(resp).await?), + } + } +} From 4ce89f15c58c3ab2adfa0689b69c48acec0c7ad0 Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Tue, 19 Mar 2024 18:58:53 +0800 Subject: [PATCH 012/111] Fix b2 Signed-off-by: Xuanwo --- core/src/services/b2/backend.rs | 30 ++++-------- core/src/services/b2/core.rs | 22 +++++---- core/src/services/b2/error.rs | 2 +- core/src/services/b2/mod.rs | 1 + core/src/services/b2/reader.rs | 61 ++++++++++++++++++++++++ core/src/services/b2/writer.rs | 5 +- core/src/services/chainsafe/backend.rs | 2 +- core/src/services/chainsafe/lister.rs | 2 +- core/src/services/gdrive/backend.rs | 2 +- core/src/services/gdrive/writer.rs | 2 +- core/src/services/icloud/core.rs | 2 +- core/src/services/ipmfs/backend.rs | 2 +- core/src/services/ipmfs/lister.rs | 2 +- core/src/services/koofr/backend.rs | 3 +- core/src/services/koofr/core.rs | 4 +- core/src/services/koofr/lister.rs | 2 +- core/src/services/libsql/backend.rs | 2 +- core/src/services/onedrive/writer.rs | 2 +- core/src/services/pcloud/backend.rs | 8 ++-- core/src/services/pcloud/core.rs | 4 +- core/src/services/pcloud/lister.rs | 2 +- core/src/services/pcloud/writer.rs | 2 +- core/src/services/vercel_blob/backend.rs | 3 +- core/src/services/webhdfs/backend.rs | 2 +- core/src/services/yandex_disk/backend.rs | 2 +- 25 files changed, 113 insertions(+), 58 deletions(-) create mode 100644 core/src/services/b2/reader.rs diff --git a/core/src/services/b2/backend.rs b/core/src/services/b2/backend.rs index 167dbab111af..f76849795f20 100644 --- a/core/src/services/b2/backend.rs +++ b/core/src/services/b2/backend.rs @@ -21,6 +21,7 @@ use std::fmt::Formatter; use std::sync::Arc; use async_trait::async_trait; +use bytes::Buf; use http::Request; use http::StatusCode; use log::debug; @@ -37,6 +38,7 @@ use super::writer::B2Writers; use crate::raw::*; use crate::services::b2::core::B2Signer; use crate::services::b2::core::ListFileNamesResponse; +use crate::services::b2::reader::B2Reader; use crate::*; /// Config for backblaze b2 services support. @@ -267,7 +269,7 @@ pub struct B2Backend { #[async_trait] impl Accessor for B2Backend { - type Reader = oio::Buffer; + type Reader = B2Reader; type Writer = B2Writers; type Lister = oio::PageLister; type BlockingReader = (); @@ -342,7 +344,7 @@ impl Accessor for B2Backend { let bs = resp.into_body(); let resp: ListFileNamesResponse = - serde_json::from_slice(&bs).map_err(new_json_deserialize_error)?; + serde_json::from_reader(bs.reader()).map_err(new_json_deserialize_error)?; if resp.files.is_empty() { return Err(Error::new(ErrorKind::NotFound, "no such file or directory")); } @@ -354,24 +356,10 @@ impl Accessor for B2Backend { } async fn read(&self, path: &str, args: OpRead) -> Result<(RpRead, Self::Reader)> { - let resp = self.core.download_file_by_name(path, &args).await?; - - let status = resp.status(); - - match status { - StatusCode::OK | StatusCode::PARTIAL_CONTENT => { - let size = parse_content_length(resp.headers())?; - let range = parse_content_range(resp.headers())?; - Ok(( - RpRead::new().with_size(size).with_range(range), - resp.into_body(), - )) - } - StatusCode::RANGE_NOT_SATISFIABLE => { - Ok((RpRead::new().with_size(Some(0)), oio::Buffer::empty())) - } - _ => Err(parse_error(resp).await?), - } + Ok(( + RpRead::default(), + B2Reader::new(self.core.clone(), path, args), + )) } async fn write(&self, path: &str, args: OpWrite) -> Result<(RpWrite, Self::Writer)> { @@ -428,7 +416,7 @@ impl Accessor for B2Backend { let bs = resp.into_body(); let resp: ListFileNamesResponse = - serde_json::from_slice(&bs).map_err(new_json_deserialize_error)?; + serde_json::from_reader(bs.reader()).map_err(new_json_deserialize_error)?; if resp.files.is_empty() { return Err(Error::new(ErrorKind::NotFound, "no such file or directory")); } diff --git a/core/src/services/b2/core.rs b/core/src/services/b2/core.rs index 17c76513a8b2..985b7e7cf795 100644 --- a/core/src/services/b2/core.rs +++ b/core/src/services/b2/core.rs @@ -15,6 +15,7 @@ // specific language governing permissions and limitations // under the License. +use bytes::Buf; use std::fmt::Debug; use std::fmt::Formatter; use std::sync::Arc; @@ -105,9 +106,10 @@ impl B2Core { match status { StatusCode::OK => { - let resp_body = &resp.into_body(); - let token = serde_json::from_slice::(resp_body) - .map_err(new_json_deserialize_error)?; + let resp_body = resp.into_body(); + let token: AuthorizeAccountResponse = + serde_json::from_reader(resp_body.reader()) + .map_err(new_json_deserialize_error)?; signer.auth_info = AuthInfo { authorization_token: token.authorization_token.clone(), api_url: token.api_url.clone(), @@ -129,6 +131,7 @@ impl B2Core { pub async fn download_file_by_name( &self, path: &str, + range: BytesRange, args: &OpRead, ) -> Result> { let path = build_abs_path(&self.root, path); @@ -147,7 +150,6 @@ impl B2Core { req = req.header(header::AUTHORIZATION, auth_info.authorization_token); - let range = args.range(); if !range.is_full() { req = req.header(header::RANGE, range.to_header()); } @@ -180,8 +182,8 @@ impl B2Core { let status = resp.status(); match status { StatusCode::OK => { - let resp_body = &resp.into_body(); - let resp = serde_json::from_slice::(resp_body) + let resp_body = resp.into_body(); + let resp = serde_json::from_reader(resp_body.reader()) .map_err(new_json_deserialize_error)?; Ok(resp) } @@ -229,8 +231,8 @@ impl B2Core { let status = resp.status(); match status { StatusCode::OK => { - let resp_body = &resp.into_body(); - let resp = serde_json::from_slice::(resp_body) + let resp_body = resp.into_body(); + let resp = serde_json::from_reader(resp_body.reader()) .map_err(new_json_deserialize_error)?; Ok(resp) } @@ -335,8 +337,8 @@ impl B2Core { let status = resp.status(); match status { StatusCode::OK => { - let resp_body = &resp.into_body(); - let resp = serde_json::from_slice::(resp_body) + let resp_body = resp.into_body(); + let resp = serde_json::from_reader(resp_body.reader()) .map_err(new_json_deserialize_error)?; Ok(resp) } diff --git a/core/src/services/b2/error.rs b/core/src/services/b2/error.rs index 80bde3761f44..209c6216c2aa 100644 --- a/core/src/services/b2/error.rs +++ b/core/src/services/b2/error.rs @@ -35,7 +35,7 @@ struct B2Error { /// Parse error response into Error. pub async fn parse_error(resp: Response) -> Result { - let (parts, body) = resp.into_parts(); + let (parts, mut body) = resp.into_parts(); let bs = body.copy_to_bytes(body.remaining()); let (mut kind, mut retryable) = match parts.status.as_u16() { diff --git a/core/src/services/b2/mod.rs b/core/src/services/b2/mod.rs index 80745f01e82a..025a80c3a831 100644 --- a/core/src/services/b2/mod.rs +++ b/core/src/services/b2/mod.rs @@ -22,4 +22,5 @@ pub use backend::B2Config; mod core; mod error; mod lister; +mod reader; mod writer; diff --git a/core/src/services/b2/reader.rs b/core/src/services/b2/reader.rs new file mode 100644 index 000000000000..0837279ad3d3 --- /dev/null +++ b/core/src/services/b2/reader.rs @@ -0,0 +1,61 @@ +// 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 super::core::*; +use super::error::*; +use crate::raw::*; +use crate::*; +use http::StatusCode; +use std::sync::Arc; + +pub struct B2Reader { + core: Arc, + + path: String, + op: OpRead, +} + +impl B2Reader { + pub fn new(core: Arc, path: &str, op: OpRead) -> Self { + B2Reader { + core, + path: path.to_string(), + op: op, + } + } +} + +impl oio::Read for B2Reader { + async fn read_at(&self, offset: u64, limit: usize) -> Result { + let Some(range) = self.op.range().apply_on_offset(offset, limit) else { + return Ok(oio::Buffer::new()); + }; + + let resp = self + .core + .download_file_by_name(&self.path, range, &self.op) + .await?; + + let status = resp.status(); + + match status { + StatusCode::OK | StatusCode::PARTIAL_CONTENT => Ok(resp.into_body()), + StatusCode::RANGE_NOT_SATISFIABLE => Ok(oio::Buffer::new()), + _ => Err(parse_error(resp).await?), + } + } +} diff --git a/core/src/services/b2/writer.rs b/core/src/services/b2/writer.rs index 9452fd07d7e7..66c7beabefc0 100644 --- a/core/src/services/b2/writer.rs +++ b/core/src/services/b2/writer.rs @@ -15,6 +15,7 @@ // specific language governing permissions and limitations // under the License. +use bytes::Buf; use std::sync::Arc; use http::StatusCode; @@ -70,7 +71,7 @@ impl oio::MultipartWrite for B2Writer { let bs = resp.into_body(); let result: StartLargeFileResponse = - serde_json::from_slice(&bs).map_err(new_json_deserialize_error)?; + serde_json::from_reader(bs.reader()).map_err(new_json_deserialize_error)?; Ok(result.file_id) } @@ -100,7 +101,7 @@ impl oio::MultipartWrite for B2Writer { let bs = resp.into_body(); let result: UploadPartResponse = - serde_json::from_slice(&bs).map_err(new_json_deserialize_error)?; + serde_json::from_reader(bs.reader()).map_err(new_json_deserialize_error)?; Ok(oio::MultipartPart { etag: result.content_sha1, diff --git a/core/src/services/chainsafe/backend.rs b/core/src/services/chainsafe/backend.rs index d3acb0e70701..cc51a5e0b7e0 100644 --- a/core/src/services/chainsafe/backend.rs +++ b/core/src/services/chainsafe/backend.rs @@ -261,7 +261,7 @@ impl Accessor for ChainsafeBackend { let bs = resp.into_body(); let output: ObjectInfoResponse = - serde_json::from_slice(&bs).map_err(new_json_deserialize_error)?; + serde_json::from_reader(bs.reader()).map_err(new_json_deserialize_error)?; Ok(RpStat::new(parse_info(output.content))) } _ => Err(parse_error(resp).await?), diff --git a/core/src/services/chainsafe/lister.rs b/core/src/services/chainsafe/lister.rs index 37a1a2dd4647..94dcfa4d5f9e 100644 --- a/core/src/services/chainsafe/lister.rs +++ b/core/src/services/chainsafe/lister.rs @@ -51,7 +51,7 @@ impl oio::PageList for ChainsafeLister { let bs = resp.into_body(); let output: Vec = - serde_json::from_slice(&bs).map_err(new_json_deserialize_error)?; + serde_json::from_reader(bs.reader()).map_err(new_json_deserialize_error)?; for info in output { let mut path = build_abs_path(&normalize_root(&self.path), &info.name); diff --git a/core/src/services/gdrive/backend.rs b/core/src/services/gdrive/backend.rs index d65b5f4978d1..8ce7ecad5e99 100644 --- a/core/src/services/gdrive/backend.rs +++ b/core/src/services/gdrive/backend.rs @@ -88,7 +88,7 @@ impl Accessor for GdriveBackend { let bs = resp.into_body(); let gdrive_file: GdriveFile = - serde_json::from_slice(&bs).map_err(new_json_deserialize_error)?; + serde_json::from_reader(bs.reader()).map_err(new_json_deserialize_error)?; if gdrive_file.mime_type == "application/vnd.google-apps.folder" { return Ok(RpStat::new(Metadata::new(EntryMode::DIR))); diff --git a/core/src/services/gdrive/writer.rs b/core/src/services/gdrive/writer.rs index b9a80b2789d0..ebafa7dfced9 100644 --- a/core/src/services/gdrive/writer.rs +++ b/core/src/services/gdrive/writer.rs @@ -66,7 +66,7 @@ impl oio::OneShotWrite for GdriveWriter { if self.file_id.is_none() { let bs = resp.into_body(); let file: GdriveFile = - serde_json::from_slice(&bs).map_err(new_json_deserialize_error)?; + serde_json::from_reader(bs.reader()).map_err(new_json_deserialize_error)?; self.core.path_cache.insert(&self.path, &file.id).await; } else { } diff --git a/core/src/services/icloud/core.rs b/core/src/services/icloud/core.rs index 2a2af72d5042..87d13e74985a 100644 --- a/core/src/services/icloud/core.rs +++ b/core/src/services/icloud/core.rs @@ -194,7 +194,7 @@ impl IcloudSigner { let bs = resp.into_body(); let auth_info: IcloudWebservicesResponse = - serde_json::from_slice(&bs).map_err(new_json_deserialize_error)?; + serde_json::from_reader(bs.reader()).map_err(new_json_deserialize_error)?; // Check if we have extra challenge to take. if auth_info.hsa_challenge_required && !auth_info.hsa_trusted_browser { diff --git a/core/src/services/ipmfs/backend.rs b/core/src/services/ipmfs/backend.rs index 250791475739..bab025555f15 100644 --- a/core/src/services/ipmfs/backend.rs +++ b/core/src/services/ipmfs/backend.rs @@ -118,7 +118,7 @@ impl Accessor for IpmfsBackend { let bs = resp.into_body(); let res: IpfsStatResponse = - serde_json::from_slice(&bs).map_err(new_json_deserialize_error)?; + serde_json::from_reader(bs.reader()).map_err(new_json_deserialize_error)?; let mode = match res.file_type.as_str() { "file" => EntryMode::FILE, diff --git a/core/src/services/ipmfs/lister.rs b/core/src/services/ipmfs/lister.rs index 7767b2518468..4288b284233c 100644 --- a/core/src/services/ipmfs/lister.rs +++ b/core/src/services/ipmfs/lister.rs @@ -53,7 +53,7 @@ impl oio::PageList for IpmfsLister { let bs = resp.into_body(); let entries_body: IpfsLsResponse = - serde_json::from_slice(&bs).map_err(new_json_deserialize_error)?; + serde_json::from_reader(bs.reader()).map_err(new_json_deserialize_error)?; // Mark dir stream has been consumed. ctx.done = true; diff --git a/core/src/services/koofr/backend.rs b/core/src/services/koofr/backend.rs index bf2ec885742f..b5cc6031b34d 100644 --- a/core/src/services/koofr/backend.rs +++ b/core/src/services/koofr/backend.rs @@ -288,7 +288,8 @@ impl Accessor for KoofrBackend { StatusCode::OK => { let bs = resp.into_body(); - let file: File = serde_json::from_slice(&bs).map_err(new_json_deserialize_error)?; + let file: File = + serde_json::from_reader(bs.reader()).map_err(new_json_deserialize_error)?; let mode = if file.ty == "dir" { EntryMode::DIR diff --git a/core/src/services/koofr/core.rs b/core/src/services/koofr/core.rs index 2e2f7d46282c..7daf730d1915 100644 --- a/core/src/services/koofr/core.rs +++ b/core/src/services/koofr/core.rs @@ -93,7 +93,7 @@ impl KoofrCore { let bs = resp.into_body(); let resp: MountsResponse = - serde_json::from_slice(&bs).map_err(new_json_deserialize_error)?; + serde_json::from_reader(bs.reader()).map_err(new_json_deserialize_error)?; for mount in resp.mounts { if mount.is_primary { @@ -139,7 +139,7 @@ impl KoofrCore { let bs = resp.into_body(); let resp: TokenResponse = - serde_json::from_slice(&bs).map_err(new_json_deserialize_error)?; + serde_json::from_reader(bs.reader()).map_err(new_json_deserialize_error)?; signer.token = resp.token; diff --git a/core/src/services/koofr/lister.rs b/core/src/services/koofr/lister.rs index 1815e9785e4a..09babe589c2c 100644 --- a/core/src/services/koofr/lister.rs +++ b/core/src/services/koofr/lister.rs @@ -60,7 +60,7 @@ impl oio::PageList for KoofrLister { let bs = resp.into_body(); let response: ListResponse = - serde_json::from_slice(&bs).map_err(new_json_deserialize_error)?; + serde_json::from_reader(bs.reader()).map_err(new_json_deserialize_error)?; for file in response.files { let path = build_abs_path(&normalize_root(&self.path), &file.name); diff --git a/core/src/services/libsql/backend.rs b/core/src/services/libsql/backend.rs index 069784e7546f..e1f3c461b4b1 100644 --- a/core/src/services/libsql/backend.rs +++ b/core/src/services/libsql/backend.rs @@ -311,7 +311,7 @@ impl Adapter { let bs = resp.into_body(); - let resp: ServerMsg = serde_json::from_slice(&bs).map_err(|e| { + let resp: ServerMsg = serde_json::from_reader(bs.reader()).map_err(|e| { Error::new(ErrorKind::Unexpected, "deserialize json from response").set_source(e) })?; diff --git a/core/src/services/onedrive/writer.rs b/core/src/services/onedrive/writer.rs index 374a037b8be9..b626a7bf4cd2 100644 --- a/core/src/services/onedrive/writer.rs +++ b/core/src/services/onedrive/writer.rs @@ -146,7 +146,7 @@ impl OneDriveWriter { StatusCode::OK => { let bs = resp.into_body(); let result: OneDriveUploadSessionCreationResponseBody = - serde_json::from_slice(&bs).map_err(new_json_deserialize_error)?; + serde_json::from_reader(bs.reader()).map_err(new_json_deserialize_error)?; Ok(result) } _ => Err(parse_error(resp).await?), diff --git a/core/src/services/pcloud/backend.rs b/core/src/services/pcloud/backend.rs index b31f08a00089..8b60bcf2a712 100644 --- a/core/src/services/pcloud/backend.rs +++ b/core/src/services/pcloud/backend.rs @@ -275,7 +275,7 @@ impl Accessor for PcloudBackend { StatusCode::OK => { let bs = resp.into_body(); let resp: StatResponse = - serde_json::from_slice(&bs).map_err(new_json_deserialize_error)?; + serde_json::from_reader(bs.reader()).map_err(new_json_deserialize_error)?; let result = resp.result; if result == 2010 || result == 2055 || result == 2002 { return Err(Error::new(ErrorKind::NotFound, &format!("{resp:?}"))); @@ -336,7 +336,7 @@ impl Accessor for PcloudBackend { StatusCode::OK => { let bs = resp.into_body(); let resp: PcloudError = - serde_json::from_slice(&bs).map_err(new_json_deserialize_error)?; + serde_json::from_reader(bs.reader()).map_err(new_json_deserialize_error)?; let result = resp.result; // pCloud returns 2005 or 2009 if the file or folder is not found @@ -370,7 +370,7 @@ impl Accessor for PcloudBackend { StatusCode::OK => { let bs = resp.into_body(); let resp: PcloudError = - serde_json::from_slice(&bs).map_err(new_json_deserialize_error)?; + serde_json::from_reader(bs.reader()).map_err(new_json_deserialize_error)?; let result = resp.result; if result == 2009 || result == 2010 || result == 2055 || result == 2002 { return Err(Error::new(ErrorKind::NotFound, &format!("{resp:?}"))); @@ -400,7 +400,7 @@ impl Accessor for PcloudBackend { StatusCode::OK => { let bs = resp.into_body(); let resp: PcloudError = - serde_json::from_slice(&bs).map_err(new_json_deserialize_error)?; + serde_json::from_reader(bs.reader()).map_err(new_json_deserialize_error)?; let result = resp.result; if result == 2009 || result == 2010 || result == 2055 || result == 2002 { return Err(Error::new(ErrorKind::NotFound, &format!("{resp:?}"))); diff --git a/core/src/services/pcloud/core.rs b/core/src/services/pcloud/core.rs index 7defb2b88610..59c455312d23 100644 --- a/core/src/services/pcloud/core.rs +++ b/core/src/services/pcloud/core.rs @@ -86,7 +86,7 @@ impl PcloudCore { StatusCode::OK => { let bs = resp.into_body(); let resp: GetFileLinkResponse = - serde_json::from_slice(&bs).map_err(new_json_deserialize_error)?; + serde_json::from_reader(bs.reader()).map_err(new_json_deserialize_error)?; let result = resp.result; if result == 2010 || result == 2055 || result == 2002 { return Err(Error::new(ErrorKind::NotFound, &format!("{resp:?}"))); @@ -134,7 +134,7 @@ impl PcloudCore { StatusCode::OK => { let bs = resp.into_body(); let resp: PcloudError = - serde_json::from_slice(&bs).map_err(new_json_deserialize_error)?; + serde_json::from_reader(bs.reader()).map_err(new_json_deserialize_error)?; let result = resp.result; if result == 2010 || result == 2055 || result == 2002 { return Err(Error::new(ErrorKind::NotFound, &format!("{resp:?}"))); diff --git a/core/src/services/pcloud/lister.rs b/core/src/services/pcloud/lister.rs index 1a2f810d7221..aed358ff4ec3 100644 --- a/core/src/services/pcloud/lister.rs +++ b/core/src/services/pcloud/lister.rs @@ -51,7 +51,7 @@ impl oio::PageList for PcloudLister { let bs = resp.into_body(); let resp: ListFolderResponse = - serde_json::from_slice(&bs).map_err(new_json_deserialize_error)?; + serde_json::from_reader(bs.reader()).map_err(new_json_deserialize_error)?; let result = resp.result; if result == 2005 { diff --git a/core/src/services/pcloud/writer.rs b/core/src/services/pcloud/writer.rs index 08750e50fd5f..6ec18d05abe3 100644 --- a/core/src/services/pcloud/writer.rs +++ b/core/src/services/pcloud/writer.rs @@ -51,7 +51,7 @@ impl oio::OneShotWrite for PcloudWriter { StatusCode::OK => { let bs = resp.into_body(); let resp: PcloudError = - serde_json::from_slice(&bs).map_err(new_json_deserialize_error)?; + serde_json::from_reader(bs.reader()).map_err(new_json_deserialize_error)?; let result = resp.result; if result != 0 { diff --git a/core/src/services/vercel_blob/backend.rs b/core/src/services/vercel_blob/backend.rs index 9bec6341a3b6..d0c034142f3e 100644 --- a/core/src/services/vercel_blob/backend.rs +++ b/core/src/services/vercel_blob/backend.rs @@ -222,7 +222,8 @@ impl Accessor for VercelBlobBackend { StatusCode::OK => { let bs = resp.into_body(); - let resp: Blob = serde_json::from_slice(&bs).map_err(new_json_deserialize_error)?; + let resp: Blob = + serde_json::from_reader(bs.reader()).map_err(new_json_deserialize_error)?; parse_blob(&resp).map(RpStat::new) } diff --git a/core/src/services/webhdfs/backend.rs b/core/src/services/webhdfs/backend.rs index eff6dddffa76..0f70e8c1cf7c 100644 --- a/core/src/services/webhdfs/backend.rs +++ b/core/src/services/webhdfs/backend.rs @@ -309,7 +309,7 @@ impl WebhdfsBackend { StatusCode::OK => { let bs = resp.into_body(); let resp: LocationResponse = - serde_json::from_slice(&bs).map_err(new_json_deserialize_error)?; + serde_json::from_reader(bs.reader()).map_err(new_json_deserialize_error)?; Ok(resp.location) } diff --git a/core/src/services/yandex_disk/backend.rs b/core/src/services/yandex_disk/backend.rs index 4af21a8024bf..c515ca511375 100644 --- a/core/src/services/yandex_disk/backend.rs +++ b/core/src/services/yandex_disk/backend.rs @@ -278,7 +278,7 @@ impl Accessor for YandexDiskBackend { let bs = resp.into_body(); let mf: MetainformationResponse = - serde_json::from_slice(&bs).map_err(new_json_deserialize_error)?; + serde_json::from_reader(bs.reader()).map_err(new_json_deserialize_error)?; parse_info(mf).map(RpStat::new) } From 4de8d7814299544da886b7f3ade65a45659bf06a Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Tue, 19 Mar 2024 19:23:07 +0800 Subject: [PATCH 013/111] FIx chainsafe Signed-off-by: Xuanwo --- core/src/services/chainsafe/backend.rs | 30 ++++--------- core/src/services/chainsafe/core.rs | 7 +++- core/src/services/chainsafe/error.rs | 2 +- core/src/services/chainsafe/lister.rs | 1 + core/src/services/chainsafe/mod.rs | 1 + core/src/services/chainsafe/reader.rs | 58 ++++++++++++++++++++++++++ 6 files changed, 75 insertions(+), 24 deletions(-) create mode 100644 core/src/services/chainsafe/reader.rs diff --git a/core/src/services/chainsafe/backend.rs b/core/src/services/chainsafe/backend.rs index cc51a5e0b7e0..fc300f62fb7f 100644 --- a/core/src/services/chainsafe/backend.rs +++ b/core/src/services/chainsafe/backend.rs @@ -21,6 +21,7 @@ use std::fmt::Formatter; use std::sync::Arc; use async_trait::async_trait; +use bytes::Buf; use http::StatusCode; use log::debug; use serde::Deserialize; @@ -30,6 +31,7 @@ use super::core::ChainsafeCore; use super::core::ObjectInfoResponse; use super::error::parse_error; use super::lister::ChainsafeLister; +use super::reader::ChainsafeReader; use super::writer::ChainsafeWriter; use super::writer::ChainsafeWriters; use crate::raw::*; @@ -203,16 +205,11 @@ pub struct ChainsafeBackend { #[async_trait] impl Accessor for ChainsafeBackend { - type Reader = oio::Buffer; - + type Reader = ChainsafeReader; type Writer = ChainsafeWriters; - type Lister = oio::PageLister; - type BlockingReader = (); - type BlockingWriter = (); - type BlockingLister = (); fn info(&self) -> AccessorInfo { @@ -268,22 +265,11 @@ impl Accessor for ChainsafeBackend { } } - async fn read(&self, path: &str, _args: OpRead) -> Result<(RpRead, Self::Reader)> { - let resp = self.core.download_object(path).await?; - - let status = resp.status(); - - match status { - StatusCode::OK => { - let size = parse_content_length(resp.headers())?; - let range = parse_content_range(resp.headers())?; - Ok(( - RpRead::new().with_size(size).with_range(range), - resp.into_body(), - )) - } - _ => Err(parse_error(resp).await?), - } + async fn read(&self, path: &str, args: OpRead) -> Result<(RpRead, Self::Reader)> { + Ok(( + RpRead::default(), + ChainsafeReader::new(self.core.clone(), path, args), + )) } async fn write(&self, path: &str, args: OpWrite) -> Result<(RpWrite, Self::Writer)> { diff --git a/core/src/services/chainsafe/core.rs b/core/src/services/chainsafe/core.rs index c367dd96e65c..e38dcb4c792b 100644 --- a/core/src/services/chainsafe/core.rs +++ b/core/src/services/chainsafe/core.rs @@ -58,7 +58,11 @@ impl ChainsafeCore { } impl ChainsafeCore { - pub async fn download_object(&self, path: &str) -> Result> { + pub async fn download_object( + &self, + path: &str, + range: BytesRange, + ) -> Result> { let path = build_abs_path(&self.root, path); let url = format!( @@ -76,6 +80,7 @@ impl ChainsafeCore { header::AUTHORIZATION, format_authorization_by_bearer(&self.api_key)?, ) + .header(header::RANGE, range.to_header()) .header(header::CONTENT_TYPE, "application/json") .body(body) .map_err(new_request_build_error)?; diff --git a/core/src/services/chainsafe/error.rs b/core/src/services/chainsafe/error.rs index aa54209baaa9..1344212f1a85 100644 --- a/core/src/services/chainsafe/error.rs +++ b/core/src/services/chainsafe/error.rs @@ -39,7 +39,7 @@ struct ChainsafeSubError { /// Parse error response into Error. pub async fn parse_error(resp: Response) -> Result { - let (parts, body) = resp.into_parts(); + let (parts, mut body) = resp.into_parts(); let bs = body.copy_to_bytes(body.remaining()); let (kind, retryable) = match parts.status.as_u16() { diff --git a/core/src/services/chainsafe/lister.rs b/core/src/services/chainsafe/lister.rs index 94dcfa4d5f9e..95d8beb6afd2 100644 --- a/core/src/services/chainsafe/lister.rs +++ b/core/src/services/chainsafe/lister.rs @@ -15,6 +15,7 @@ // specific language governing permissions and limitations // under the License. +use bytes::Buf; use std::sync::Arc; use http::StatusCode; diff --git a/core/src/services/chainsafe/mod.rs b/core/src/services/chainsafe/mod.rs index ed5335a738bf..cecf6a300201 100644 --- a/core/src/services/chainsafe/mod.rs +++ b/core/src/services/chainsafe/mod.rs @@ -22,4 +22,5 @@ pub use backend::ChainsafeConfig; mod core; mod error; mod lister; +mod reader; mod writer; diff --git a/core/src/services/chainsafe/reader.rs b/core/src/services/chainsafe/reader.rs new file mode 100644 index 000000000000..1a59eede51f1 --- /dev/null +++ b/core/src/services/chainsafe/reader.rs @@ -0,0 +1,58 @@ +// 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 super::core::ChainsafeCore; +use super::error::parse_error; +use crate::raw::{oio, OpRead}; +use http::StatusCode; +use std::future::Future; +use std::sync::Arc; + +pub struct ChainsafeReader { + core: Arc, + + path: String, + op: OpRead, +} + +impl ChainsafeReader { + pub fn new(core: Arc, path: &str, op: OpRead) -> Self { + ChainsafeReader { + core, + path: path.to_string(), + op, + } + } +} + +impl oio::Read for ChainsafeReader { + async fn read_at(&self, offset: u64, limit: usize) -> crate::Result { + let Some(range) = self.op.range().apply_on_offset(offset, limit) else { + return Ok(oio::Buffer::new()); + }; + + let resp = self.core.download_object(&self.path, range).await?; + + let status = resp.status(); + + match status { + StatusCode::OK | StatusCode::PARTIAL_CONTENT => Ok(resp.into_body()), + StatusCode::RANGE_NOT_SATISFIABLE => Ok(oio::Buffer::new()), + _ => Err(parse_error(resp).await?), + } + } +} From 9742968be9bfdaff523d3ab2f894fefa541102f1 Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Tue, 19 Mar 2024 19:25:57 +0800 Subject: [PATCH 014/111] Fix cloudflare kv Signed-off-by: Xuanwo --- core/src/services/cloudflare_kv/backend.rs | 18 ++++++++++-------- core/src/services/cloudflare_kv/error.rs | 2 +- 2 files changed, 11 insertions(+), 9 deletions(-) diff --git a/core/src/services/cloudflare_kv/backend.rs b/core/src/services/cloudflare_kv/backend.rs index 075e7f6ac5e5..f55c34e79c11 100644 --- a/core/src/services/cloudflare_kv/backend.rs +++ b/core/src/services/cloudflare_kv/backend.rs @@ -20,6 +20,7 @@ use std::fmt::Debug; use std::fmt::Formatter; use async_trait::async_trait; +use bytes::Buf; use http::header; use http::Request; use http::StatusCode; @@ -236,8 +237,8 @@ impl kv::Adapter for Adapter { let status = resp.status(); match status { StatusCode::OK => { - let body = resp.into_body(); - Ok(Some(body.into())) + let mut body = resp.into_body(); + Ok(Some(body.copy_to_bytes(body.remaining()).to_vec())) } _ => Err(parse_error(resp).await?), } @@ -292,12 +293,13 @@ impl kv::Adapter for Adapter { match status { StatusCode::OK => { let body = resp.into_body(); - let response: CfKvScanResponse = serde_json::from_slice(&body).map_err(|e| { - Error::new( - ErrorKind::Unexpected, - &format!("failed to parse error response: {}", e), - ) - })?; + let response: CfKvScanResponse = + serde_json::from_reader(body.reader()).map_err(|e| { + Error::new( + ErrorKind::Unexpected, + &format!("failed to parse error response: {}", e), + ) + })?; Ok(response.result.into_iter().map(|r| r.name).collect()) } _ => Err(parse_error(resp).await?), diff --git a/core/src/services/cloudflare_kv/error.rs b/core/src/services/cloudflare_kv/error.rs index 0041adbe45cf..34c81953516e 100644 --- a/core/src/services/cloudflare_kv/error.rs +++ b/core/src/services/cloudflare_kv/error.rs @@ -29,7 +29,7 @@ use crate::Result; /// Parse error response into Error. pub(crate) async fn parse_error(resp: Response) -> Result { - let (parts, body) = resp.into_parts(); + let (parts, mut body) = resp.into_parts(); let bs = body.copy_to_bytes(body.remaining()); let (mut kind, mut retryable) = match parts.status { From 12e451f81325aaa87721c7ebf0f3815ed579b0a8 Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Tue, 19 Mar 2024 19:46:34 +0800 Subject: [PATCH 015/111] Fix cos Signed-off-by: Xuanwo --- core/src/services/cos/backend.rs | 30 ++++------- core/src/services/cos/core.rs | 17 ++++-- core/src/services/cos/error.rs | 2 +- core/src/services/cos/mod.rs | 1 + core/src/services/cos/reader.rs | 60 +++++++++++++++++++++ core/src/services/d1/error.rs | 2 +- core/src/services/dbfs/error.rs | 2 +- core/src/services/dropbox/error.rs | 2 +- core/src/services/gcs/error.rs | 2 +- core/src/services/gdrive/error.rs | 2 +- core/src/services/ghac/error.rs | 2 +- core/src/services/github/error.rs | 2 +- core/src/services/http/error.rs | 2 +- core/src/services/huggingface/error.rs | 2 +- core/src/services/icloud/core.rs | 2 +- core/src/services/ipfs/error.rs | 2 +- core/src/services/ipmfs/error.rs | 2 +- core/src/services/koofr/error.rs | 2 +- core/src/services/libsql/error.rs | 2 +- core/src/services/obs/error.rs | 2 +- core/src/services/onedrive/error.rs | 2 +- core/src/services/oss/error.rs | 2 +- core/src/services/pcloud/error.rs | 2 +- core/src/services/seafile/error.rs | 2 +- core/src/services/supabase/error.rs | 2 +- core/src/services/swift/error.rs | 2 +- core/src/services/upyun/error.rs | 2 +- core/src/services/vercel_artifacts/error.rs | 2 +- core/src/services/vercel_blob/error.rs | 2 +- core/src/services/webdav/error.rs | 2 +- core/src/services/webhdfs/backend.rs | 2 +- core/src/services/webhdfs/error.rs | 2 +- core/src/services/yandex_disk/error.rs | 2 +- 33 files changed, 113 insertions(+), 53 deletions(-) create mode 100644 core/src/services/cos/reader.rs diff --git a/core/src/services/cos/backend.rs b/core/src/services/cos/backend.rs index b83922d1e1a1..a86709fb9ac4 100644 --- a/core/src/services/cos/backend.rs +++ b/core/src/services/cos/backend.rs @@ -33,6 +33,7 @@ use super::error::parse_error; use super::lister::CosLister; use super::writer::CosWriter; use crate::raw::*; +use crate::services::cos::reader::CosReader; use crate::services::cos::writer::CosWriters; use crate::*; @@ -254,7 +255,7 @@ pub struct CosBackend { #[async_trait] impl Accessor for CosBackend { - type Reader = oio::Buffer; + type Reader = CosReader; type Writer = CosWriters; type Lister = oio::PageLister; type BlockingReader = (); @@ -326,24 +327,10 @@ impl Accessor for CosBackend { } async fn read(&self, path: &str, args: OpRead) -> Result<(RpRead, Self::Reader)> { - let resp = self.core.cos_get_object(path, &args).await?; - - let status = resp.status(); - - match status { - StatusCode::OK | StatusCode::PARTIAL_CONTENT => { - let size = parse_content_length(resp.headers())?; - let range = parse_content_range(resp.headers())?; - Ok(( - RpRead::new().with_size(size).with_range(range), - resp.into_body(), - )) - } - StatusCode::RANGE_NOT_SATISFIABLE => { - Ok((RpRead::new().with_size(Some(0)), oio::Buffer::empty())) - } - _ => Err(parse_error(resp).await?), - } + Ok(( + RpRead::default(), + CosReader::new(self.core.clone(), path, args), + )) } async fn write(&self, path: &str, args: OpWrite) -> Result<(RpWrite, Self::Writer)> { @@ -390,7 +377,10 @@ impl Accessor for CosBackend { async fn presign(&self, path: &str, args: OpPresign) -> Result { let mut req = match args.operation() { PresignOperation::Stat(v) => self.core.cos_head_object_request(path, v)?, - PresignOperation::Read(v) => self.core.cos_get_object_request(path, v)?, + PresignOperation::Read(v) => { + self.core + .cos_get_object_request(path, BytesRange::default(), v)? + } PresignOperation::Write(v) => { self.core .cos_put_object_request(path, None, v, AsyncBody::Empty)? diff --git a/core/src/services/cos/core.rs b/core/src/services/cos/core.rs index 208fc2721bd0..003be69b319b 100644 --- a/core/src/services/cos/core.rs +++ b/core/src/services/cos/core.rs @@ -101,15 +101,25 @@ impl CosCore { } impl CosCore { - pub async fn cos_get_object(&self, path: &str, args: &OpRead) -> Result> { - let mut req = self.cos_get_object_request(path, args)?; + pub async fn cos_get_object( + &self, + path: &str, + range: BytesRange, + args: &OpRead, + ) -> Result> { + let mut req = self.cos_get_object_request(path, range, args)?; self.sign(&mut req).await?; self.send(req).await } - pub fn cos_get_object_request(&self, path: &str, args: &OpRead) -> Result> { + pub fn cos_get_object_request( + &self, + path: &str, + range: BytesRange, + args: &OpRead, + ) -> Result> { let p = build_abs_path(&self.root, path); let url = format!("{}/{}", self.endpoint, percent_encode_path(&p)); @@ -120,7 +130,6 @@ impl CosCore { req = req.header(IF_MATCH, if_match); } - let range = args.range(); if !range.is_full() { req = req.header(http::header::RANGE, range.to_header()) } diff --git a/core/src/services/cos/error.rs b/core/src/services/cos/error.rs index 2f7c2ee670f6..95d5e347fbaa 100644 --- a/core/src/services/cos/error.rs +++ b/core/src/services/cos/error.rs @@ -39,7 +39,7 @@ struct CosError { /// Parse error response into Error. pub async fn parse_error(resp: Response) -> Result { - let (parts, body) = resp.into_parts(); + let (parts, mut body) = resp.into_parts(); let bs = body.copy_to_bytes(body.remaining()); let (kind, retryable) = match parts.status { diff --git a/core/src/services/cos/mod.rs b/core/src/services/cos/mod.rs index 780c2c4796cf..07f1e54ce94b 100644 --- a/core/src/services/cos/mod.rs +++ b/core/src/services/cos/mod.rs @@ -21,4 +21,5 @@ pub use backend::CosBuilder as Cos; mod core; mod error; mod lister; +mod reader; mod writer; diff --git a/core/src/services/cos/reader.rs b/core/src/services/cos/reader.rs new file mode 100644 index 000000000000..e830d23aba35 --- /dev/null +++ b/core/src/services/cos/reader.rs @@ -0,0 +1,60 @@ +// 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 super::core::CosCore; +use super::error::parse_error; +use crate::raw::{oio, OpRead}; +use http::StatusCode; +use std::sync::Arc; + +pub struct CosReader { + core: Arc, + + path: String, + op: OpRead, +} + +impl CosReader { + pub fn new(core: Arc, path: &str, op: OpRead) -> Self { + CosReader { + core, + path: path.to_string(), + op: op, + } + } +} + +impl oio::Read for CosReader { + async fn read_at(&self, offset: u64, limit: usize) -> crate::Result { + let Some(range) = self.op.range().apply_on_offset(offset, limit) else { + return Ok(oio::Buffer::new()); + }; + + let resp = self + .core + .cos_get_object(&self.path, range, &self.op) + .await?; + + let status = resp.status(); + + match status { + StatusCode::OK | StatusCode::PARTIAL_CONTENT => Ok(resp.into_body()), + StatusCode::RANGE_NOT_SATISFIABLE => Ok(oio::Buffer::new()), + _ => Err(parse_error(resp).await?), + } + } +} diff --git a/core/src/services/d1/error.rs b/core/src/services/d1/error.rs index 98c774381611..ada419ef4b3d 100644 --- a/core/src/services/d1/error.rs +++ b/core/src/services/d1/error.rs @@ -29,7 +29,7 @@ use crate::Result; /// Parse error response into Error. pub async fn parse_error(resp: Response) -> Result { - let (parts, body) = resp.into_parts(); + let (parts, mut body) = resp.into_parts(); let bs = body.copy_to_bytes(body.remaining()); let (mut kind, mut retryable) = match parts.status { diff --git a/core/src/services/dbfs/error.rs b/core/src/services/dbfs/error.rs index 6b5b8fac0a0b..7b8704fc2973 100644 --- a/core/src/services/dbfs/error.rs +++ b/core/src/services/dbfs/error.rs @@ -45,7 +45,7 @@ impl Debug for DbfsError { } pub async fn parse_error(resp: Response) -> Result { - let (parts, body) = resp.into_parts(); + let (parts, mut body) = resp.into_parts(); let bs = body.copy_to_bytes(body.remaining()); let (kind, retryable) = match parts.status { diff --git a/core/src/services/dropbox/error.rs b/core/src/services/dropbox/error.rs index fe69da739671..2996f44453c4 100644 --- a/core/src/services/dropbox/error.rs +++ b/core/src/services/dropbox/error.rs @@ -32,7 +32,7 @@ pub struct DropboxErrorResponse { /// Parse error response into Error. pub async fn parse_error(resp: Response) -> Result { - let (parts, body) = resp.into_parts(); + let (parts, mut body) = resp.into_parts(); let bs = body.copy_to_bytes(body.remaining()); let (mut kind, mut retryable) = match parts.status { diff --git a/core/src/services/gcs/error.rs b/core/src/services/gcs/error.rs index 74a6649741dc..f270fc7bd0db 100644 --- a/core/src/services/gcs/error.rs +++ b/core/src/services/gcs/error.rs @@ -51,7 +51,7 @@ struct GcsErrorDetail { /// Parse error response into Error. pub async fn parse_error(resp: Response) -> Result { - let (parts, body) = resp.into_parts(); + let (parts, mut body) = resp.into_parts(); let bs = body.copy_to_bytes(body.remaining()); let (kind, retryable) = match parts.status { diff --git a/core/src/services/gdrive/error.rs b/core/src/services/gdrive/error.rs index 5fdd634a5233..5a2e78a34375 100644 --- a/core/src/services/gdrive/error.rs +++ b/core/src/services/gdrive/error.rs @@ -36,7 +36,7 @@ struct GdriveInnerError { /// Parse error response into Error. pub async fn parse_error(resp: Response) -> Result { - let (parts, body) = resp.into_parts(); + let (parts, mut body) = resp.into_parts(); let bs = body.copy_to_bytes(body.remaining()); let (mut kind, mut retryable) = match parts.status { diff --git a/core/src/services/ghac/error.rs b/core/src/services/ghac/error.rs index dae4c334edcd..27f99902cae3 100644 --- a/core/src/services/ghac/error.rs +++ b/core/src/services/ghac/error.rs @@ -25,7 +25,7 @@ use crate::Result; /// Parse error response into Error. pub async fn parse_error(resp: Response) -> Result { - let (parts, body) = resp.into_parts(); + let (parts, mut body) = resp.into_parts(); let (kind, retryable) = match parts.status { StatusCode::NOT_FOUND | StatusCode::NO_CONTENT => (ErrorKind::NotFound, false), diff --git a/core/src/services/github/error.rs b/core/src/services/github/error.rs index 4fcedf851013..e2dd87909655 100644 --- a/core/src/services/github/error.rs +++ b/core/src/services/github/error.rs @@ -39,7 +39,7 @@ struct GithubSubError { /// Parse error response into Error. pub async fn parse_error(resp: Response) -> Result { - let (parts, body) = resp.into_parts(); + let (parts, mut body) = resp.into_parts(); let bs = body.copy_to_bytes(body.remaining()); let (kind, retryable) = match parts.status.as_u16() { diff --git a/core/src/services/http/error.rs b/core/src/services/http/error.rs index 96386a946525..336ca8d8798d 100644 --- a/core/src/services/http/error.rs +++ b/core/src/services/http/error.rs @@ -25,7 +25,7 @@ use crate::Result; /// Parse error response into Error. pub async fn parse_error(resp: Response) -> Result { - let (parts, body) = resp.into_parts(); + let (parts, mut body) = resp.into_parts(); let bs = body.copy_to_bytes(body.remaining()); let (kind, retryable) = match parts.status { diff --git a/core/src/services/huggingface/error.rs b/core/src/services/huggingface/error.rs index 0057b9b3e33c..c69c03e5e5a9 100644 --- a/core/src/services/huggingface/error.rs +++ b/core/src/services/huggingface/error.rs @@ -40,7 +40,7 @@ impl Debug for HuggingfaceError { } pub async fn parse_error(resp: Response) -> Result { - let (parts, body) = resp.into_parts(); + let (parts, mut body) = resp.into_parts(); let bs = body.copy_to_bytes(body.remaining()); let (kind, retryable) = match parts.status { diff --git a/core/src/services/icloud/core.rs b/core/src/services/icloud/core.rs index 87d13e74985a..697f235e8da6 100644 --- a/core/src/services/icloud/core.rs +++ b/core/src/services/icloud/core.rs @@ -562,7 +562,7 @@ impl PathQuery for IcloudPathQuery { } pub async fn parse_error(resp: Response) -> Result { - let (parts, body) = resp.into_parts(); + let (parts, mut body) = resp.into_parts(); let bs = body.copy_to_bytes(body.remaining()); let mut kind = match parts.status.as_u16() { diff --git a/core/src/services/ipfs/error.rs b/core/src/services/ipfs/error.rs index d5211cd2f3e0..a526369c2e3f 100644 --- a/core/src/services/ipfs/error.rs +++ b/core/src/services/ipfs/error.rs @@ -25,7 +25,7 @@ use crate::Result; /// Parse error response into Error. pub async fn parse_error(resp: Response) -> Result { - let (parts, body) = resp.into_parts(); + let (parts, mut body) = resp.into_parts(); let bs = body.copy_to_bytes(body.remaining()); let (kind, retryable) = match parts.status { diff --git a/core/src/services/ipmfs/error.rs b/core/src/services/ipmfs/error.rs index 118ae17adeb1..edf3e0f4f23e 100644 --- a/core/src/services/ipmfs/error.rs +++ b/core/src/services/ipmfs/error.rs @@ -46,7 +46,7 @@ struct IpfsError { /// /// ref: https://docs.ipfs.tech/reference/kubo/rpc/#http-status-codes pub async fn parse_error(resp: Response) -> Result { - let (parts, body) = resp.into_parts(); + let (parts, mut body) = resp.into_parts(); let bs = body.copy_to_bytes(body.remaining()); let ipfs_error = de::from_slice::(&bs).ok(); diff --git a/core/src/services/koofr/error.rs b/core/src/services/koofr/error.rs index 559377549b60..bbea17cf9268 100644 --- a/core/src/services/koofr/error.rs +++ b/core/src/services/koofr/error.rs @@ -24,7 +24,7 @@ use crate::Result; /// Parse error response into Error. pub async fn parse_error(resp: Response) -> Result { - let (parts, body) = resp.into_parts(); + let (parts, mut body) = resp.into_parts(); let bs = body.copy_to_bytes(body.remaining()); let (kind, retryable) = match parts.status.as_u16() { diff --git a/core/src/services/libsql/error.rs b/core/src/services/libsql/error.rs index f55b4d26b0a8..2ab1000c6f1d 100644 --- a/core/src/services/libsql/error.rs +++ b/core/src/services/libsql/error.rs @@ -25,7 +25,7 @@ use crate::Result; /// Parse error response into Error. pub async fn parse_error(resp: Response) -> Result { - let (parts, body) = resp.into_parts(); + let (parts, mut body) = resp.into_parts(); let bs = body.copy_to_bytes(body.remaining()); let (kind, retryable) = match parts.status { diff --git a/core/src/services/obs/error.rs b/core/src/services/obs/error.rs index bb33631fa542..034def4d2aa4 100644 --- a/core/src/services/obs/error.rs +++ b/core/src/services/obs/error.rs @@ -39,7 +39,7 @@ struct ObsError { /// Parse error response into Error. pub async fn parse_error(resp: Response) -> Result { - let (parts, body) = resp.into_parts(); + let (parts, mut body) = resp.into_parts(); let bs = body.copy_to_bytes(body.remaining()); let (kind, retryable) = match parts.status { diff --git a/core/src/services/onedrive/error.rs b/core/src/services/onedrive/error.rs index 1c8b8218c6c2..94d592fd5652 100644 --- a/core/src/services/onedrive/error.rs +++ b/core/src/services/onedrive/error.rs @@ -25,7 +25,7 @@ use crate::Result; /// Parse error response into Error. pub async fn parse_error(resp: Response) -> Result { - let (parts, body) = resp.into_parts(); + let (parts, mut body) = resp.into_parts(); let bs = body.copy_to_bytes(body.remaining()); let (kind, retryable) = match parts.status { diff --git a/core/src/services/oss/error.rs b/core/src/services/oss/error.rs index 6aeef66cf491..e1d8fcf2a200 100644 --- a/core/src/services/oss/error.rs +++ b/core/src/services/oss/error.rs @@ -38,7 +38,7 @@ struct OssError { /// Parse error response into Error. pub async fn parse_error(resp: Response) -> Result { - let (parts, body) = resp.into_parts(); + let (parts, mut body) = resp.into_parts(); let bs = body.copy_to_bytes(body.remaining()); let (kind, retryable) = match parts.status { diff --git a/core/src/services/pcloud/error.rs b/core/src/services/pcloud/error.rs index 877709b75658..b4900689e41a 100644 --- a/core/src/services/pcloud/error.rs +++ b/core/src/services/pcloud/error.rs @@ -44,7 +44,7 @@ impl Debug for PcloudError { /// Parse error response into Error. pub async fn parse_error(resp: Response) -> Result { - let (parts, body) = resp.into_parts(); + let (parts, mut body) = resp.into_parts(); let bs = body.copy_to_bytes(body.remaining()); let message = String::from_utf8_lossy(&bs).into_owned(); diff --git a/core/src/services/seafile/error.rs b/core/src/services/seafile/error.rs index dba17133d78a..949d083fdad7 100644 --- a/core/src/services/seafile/error.rs +++ b/core/src/services/seafile/error.rs @@ -33,7 +33,7 @@ struct SeafileError { /// Parse error response into Error. pub async fn parse_error(resp: Response) -> Result { - let (parts, body) = resp.into_parts(); + let (parts, mut body) = resp.into_parts(); let bs = body.copy_to_bytes(body.remaining()); let (kind, _retryable) = match parts.status.as_u16() { diff --git a/core/src/services/supabase/error.rs b/core/src/services/supabase/error.rs index 239090c212bc..bfc5038e4587 100644 --- a/core/src/services/supabase/error.rs +++ b/core/src/services/supabase/error.rs @@ -36,7 +36,7 @@ struct SupabaseError { /// Parse the supabase error type to the OpenDAL error type pub async fn parse_error(resp: Response) -> Result { - let (parts, body) = resp.into_parts(); + let (parts, mut body) = resp.into_parts(); let bs = body.copy_to_bytes(body.remaining()); // Check HTTP status code first/ diff --git a/core/src/services/swift/error.rs b/core/src/services/swift/error.rs index c73a43c01ba3..6c956b20f442 100644 --- a/core/src/services/swift/error.rs +++ b/core/src/services/swift/error.rs @@ -35,7 +35,7 @@ struct ErrorResponse { } pub async fn parse_error(resp: Response) -> Result { - let (parts, body) = resp.into_parts(); + let (parts, mut body) = resp.into_parts(); let bs = body.copy_to_bytes(body.remaining()); let (kind, retryable) = match parts.status { diff --git a/core/src/services/upyun/error.rs b/core/src/services/upyun/error.rs index 63614e8a9659..2ecaadc2a4e1 100644 --- a/core/src/services/upyun/error.rs +++ b/core/src/services/upyun/error.rs @@ -36,7 +36,7 @@ struct UpyunError { /// Parse error response into Error. pub async fn parse_error(resp: Response) -> Result { - let (parts, body) = resp.into_parts(); + let (parts, mut body) = resp.into_parts(); let bs = body.copy_to_bytes(body.remaining()); let (kind, retryable) = match parts.status.as_u16() { diff --git a/core/src/services/vercel_artifacts/error.rs b/core/src/services/vercel_artifacts/error.rs index 1c8b8218c6c2..94d592fd5652 100644 --- a/core/src/services/vercel_artifacts/error.rs +++ b/core/src/services/vercel_artifacts/error.rs @@ -25,7 +25,7 @@ use crate::Result; /// Parse error response into Error. pub async fn parse_error(resp: Response) -> Result { - let (parts, body) = resp.into_parts(); + let (parts, mut body) = resp.into_parts(); let bs = body.copy_to_bytes(body.remaining()); let (kind, retryable) = match parts.status { diff --git a/core/src/services/vercel_blob/error.rs b/core/src/services/vercel_blob/error.rs index b781183d5434..d733071246f7 100644 --- a/core/src/services/vercel_blob/error.rs +++ b/core/src/services/vercel_blob/error.rs @@ -41,7 +41,7 @@ struct VercelBlobErrorDetail { /// Parse error response into Error. pub async fn parse_error(resp: Response) -> Result { - let (parts, body) = resp.into_parts(); + let (parts, mut body) = resp.into_parts(); let bs = body.copy_to_bytes(body.remaining()); let (kind, retryable) = match parts.status.as_u16() { diff --git a/core/src/services/webdav/error.rs b/core/src/services/webdav/error.rs index ee3210a4da0c..08e463b8aa33 100644 --- a/core/src/services/webdav/error.rs +++ b/core/src/services/webdav/error.rs @@ -25,7 +25,7 @@ use crate::Result; /// Parse error response into Error. pub async fn parse_error(resp: Response) -> Result { - let (parts, body) = resp.into_parts(); + let (parts, mut body) = resp.into_parts(); let bs = body.copy_to_bytes(body.remaining()); let (kind, retryable) = match parts.status { diff --git a/core/src/services/webhdfs/backend.rs b/core/src/services/webhdfs/backend.rs index 0f70e8c1cf7c..a0cdfd5de160 100644 --- a/core/src/services/webhdfs/backend.rs +++ b/core/src/services/webhdfs/backend.rs @@ -660,7 +660,7 @@ impl Accessor for WebhdfsBackend { } // WebHDFS will returns 403 when range is outside of the end. StatusCode::FORBIDDEN => { - let (parts, body) = resp.into_parts(); + let (parts, mut body) = resp.into_parts(); let bs = body.copy_to_bytes(body.remaining()); let s = String::from_utf8_lossy(&bs); if s.contains("out of the range") { diff --git a/core/src/services/webhdfs/error.rs b/core/src/services/webhdfs/error.rs index 5130c7705a7f..bb0b6f846e0d 100644 --- a/core/src/services/webhdfs/error.rs +++ b/core/src/services/webhdfs/error.rs @@ -40,7 +40,7 @@ struct WebHdfsError { } pub(super) async fn parse_error(resp: Response) -> Result { - let (parts, body) = resp.into_parts(); + let (parts, mut body) = resp.into_parts(); let bs = body.copy_to_bytes(body.remaining()); let s = String::from_utf8_lossy(&bs); parse_error_msg(parts, &s) diff --git a/core/src/services/yandex_disk/error.rs b/core/src/services/yandex_disk/error.rs index 170489c676e0..bd82eaed5e9f 100644 --- a/core/src/services/yandex_disk/error.rs +++ b/core/src/services/yandex_disk/error.rs @@ -36,7 +36,7 @@ struct YandexDiskError { /// Parse error response into Error. pub async fn parse_error(resp: Response) -> Result { - let (parts, body) = resp.into_parts(); + let (parts, mut body) = resp.into_parts(); let bs = body.copy_to_bytes(body.remaining()); let (kind, retryable) = match parts.status.as_u16() { From 0d4ba9aa8b8e4d079d4c262223482d4c9fd359b2 Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Tue, 19 Mar 2024 19:48:01 +0800 Subject: [PATCH 016/111] Fix d1 Signed-off-by: Xuanwo --- core/src/services/d1/backend.rs | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/core/src/services/d1/backend.rs b/core/src/services/d1/backend.rs index ce16a6df7949..9cf3c38ccbcd 100644 --- a/core/src/services/d1/backend.rs +++ b/core/src/services/d1/backend.rs @@ -20,6 +20,7 @@ use std::fmt::Debug; use std::fmt::Formatter; use async_trait::async_trait; +use bytes::Buf; use http::header; use http::Request; use http::StatusCode; @@ -313,8 +314,9 @@ impl kv::Adapter for Adapter { let status = resp.status(); match status { StatusCode::OK | StatusCode::PARTIAL_CONTENT => { - let body = resp.into_body(); - let d1_response = D1Response::parse(&body)?; + let mut body = resp.into_body(); + let bs = body.copy_to_bytes(body.remaining()); + let d1_response = D1Response::parse(&bs)?; Ok(d1_response.get_result(&self.value_field)) } _ => Err(parse_error(resp).await?), From c4f5a2412a993a86e4f4a3e0a3aa214a4a27c438 Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Tue, 19 Mar 2024 19:50:44 +0800 Subject: [PATCH 017/111] Fix dbfs Signed-off-by: Xuanwo --- core/src/services/dbfs/backend.rs | 5 +++-- core/src/services/dbfs/error.rs | 1 + core/src/services/dbfs/lister.rs | 7 ++++--- core/src/services/dbfs/reader.rs | 20 ++++---------------- 4 files changed, 12 insertions(+), 21 deletions(-) diff --git a/core/src/services/dbfs/backend.rs b/core/src/services/dbfs/backend.rs index 59d54ebffb76..2ee42040bab2 100644 --- a/core/src/services/dbfs/backend.rs +++ b/core/src/services/dbfs/backend.rs @@ -21,6 +21,7 @@ use std::fmt::Formatter; use std::sync::Arc; use async_trait::async_trait; +use bytes::Buf; use http::StatusCode; use log::debug; use serde::Deserialize; @@ -222,8 +223,8 @@ impl Accessor for DbfsBackend { StatusCode::OK => { let mut meta = parse_into_metadata(path, resp.headers())?; let bs = resp.into_body(); - let decoded_response = serde_json::from_slice::(&bs) - .map_err(new_json_deserialize_error)?; + let decoded_response: DbfsStatus = + serde_json::from_reader(bs.reader()).map_err(new_json_deserialize_error)?; meta.set_last_modified(parse_datetime_from_from_timestamp_millis( decoded_response.modification_time, )?); diff --git a/core/src/services/dbfs/error.rs b/core/src/services/dbfs/error.rs index 7b8704fc2973..53a5e4d4eba2 100644 --- a/core/src/services/dbfs/error.rs +++ b/core/src/services/dbfs/error.rs @@ -15,6 +15,7 @@ // specific language governing permissions and limitations // under the License. +use bytes::Buf; use std::fmt::Debug; use http::Response; diff --git a/core/src/services/dbfs/lister.rs b/core/src/services/dbfs/lister.rs index 11841bdae842..f6b31091d6f0 100644 --- a/core/src/services/dbfs/lister.rs +++ b/core/src/services/dbfs/lister.rs @@ -15,6 +15,7 @@ // specific language governing permissions and limitations // under the License. +use bytes::Buf; use std::sync::Arc; use http::StatusCode; @@ -50,9 +51,9 @@ impl oio::PageList for DbfsLister { return Err(error); } - let bytes = response.into_body().bytes().await?; - let decoded_response = - serde_json::from_slice::(&bytes).map_err(new_json_deserialize_error)?; + let bytes = response.into_body(); + let decoded_response: DbfsOutputList = + serde_json::from_reader(bytes.reader()).map_err(new_json_deserialize_error)?; ctx.done = true; diff --git a/core/src/services/dbfs/reader.rs b/core/src/services/dbfs/reader.rs index 856d0eaed490..4a4bbd4c7a23 100644 --- a/core/src/services/dbfs/reader.rs +++ b/core/src/services/dbfs/reader.rs @@ -15,6 +15,7 @@ // specific language governing permissions and limitations // under the License. +use std::future::Future; use std::io::SeekFrom; use std::sync::Arc; @@ -24,6 +25,7 @@ use bytes::Bytes; use serde::Deserialize; use super::core::DbfsCore; +use crate::raw::oio::Buffer; use crate::raw::*; use crate::*; @@ -86,22 +88,8 @@ impl DbfsReader { unsafe impl Sync for DbfsReader {} impl oio::Read for DbfsReader { - async fn read(&mut self, limit: usize) -> Result { - let _ = limit; - - Err(Error::new( - ErrorKind::Unsupported, - "output reader doesn't support seeking", - )) - } - - async fn seek(&mut self, pos: SeekFrom) -> Result { - let _ = pos; - - Err(Error::new( - ErrorKind::Unsupported, - "output reader doesn't support seeking", - )) + async fn read_at(&self, offset: u64, limit: usize) -> Result { + todo!() } } From 6b7217f18c1ad8236c19a4a31045cb8a0c3961a7 Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Tue, 19 Mar 2024 19:55:25 +0800 Subject: [PATCH 018/111] Fix dropbox Signed-off-by: Xuanwo --- core/src/services/dropbox/backend.rs | 29 +++++++------- core/src/services/dropbox/core.rs | 16 +++++--- core/src/services/dropbox/error.rs | 1 + core/src/services/dropbox/mod.rs | 1 + core/src/services/dropbox/reader.rs | 58 ++++++++++++++++++++++++++++ 5 files changed, 83 insertions(+), 22 deletions(-) create mode 100644 core/src/services/dropbox/reader.rs diff --git a/core/src/services/dropbox/backend.rs b/core/src/services/dropbox/backend.rs index 0cbe9be64e9f..fc27c5b210cc 100644 --- a/core/src/services/dropbox/backend.rs +++ b/core/src/services/dropbox/backend.rs @@ -20,10 +20,12 @@ use std::sync::Arc; use async_trait::async_trait; use backon::Retryable; +use bytes::Buf; use http::StatusCode; use super::core::*; use super::error::*; +use super::reader::DropboxReader; use super::writer::DropboxWriter; use crate::raw::*; use crate::*; @@ -35,7 +37,7 @@ pub struct DropboxBackend { #[async_trait] impl Accessor for DropboxBackend { - type Reader = oio::Buffer; + type Reader = DropboxReader; type Writer = oio::OneShotWriter; type Lister = (); type BlockingReader = (); @@ -71,8 +73,8 @@ impl Accessor for DropboxBackend { let resp = self.core.dropbox_get_metadata(path).await?; if StatusCode::OK == resp.status() { let bytes = resp.into_body(); - let decoded_response = serde_json::from_slice::(&bytes) - .map_err(new_json_deserialize_error)?; + let decoded_response: DropboxMetadataResponse = + serde_json::from_reader(bytes.reader()).map_err(new_json_deserialize_error)?; if "folder" == decoded_response.tag { return Ok(RpCreateDir::default()); } @@ -103,8 +105,8 @@ impl Accessor for DropboxBackend { match status { StatusCode::OK => { let bytes = resp.into_body(); - let decoded_response = serde_json::from_slice::(&bytes) - .map_err(new_json_deserialize_error)?; + let decoded_response: DropboxMetadataResponse = + serde_json::from_reader(bytes.reader()).map_err(new_json_deserialize_error)?; let entry_mode: EntryMode = match decoded_response.tag.as_str() { "file" => EntryMode::FILE, "folder" => EntryMode::DIR, @@ -136,15 +138,10 @@ impl Accessor for DropboxBackend { } async fn read(&self, path: &str, args: OpRead) -> Result<(RpRead, Self::Reader)> { - let resp = self.core.dropbox_get(path, args).await?; - let status = resp.status(); - match status { - StatusCode::OK | StatusCode::PARTIAL_CONTENT => Ok((RpRead::new(), resp.into_body())), - StatusCode::RANGE_NOT_SATISFIABLE => { - Ok((RpRead::new().with_size(Some(0)), oio::Buffer::empty())) - } - _ => Err(parse_error(resp).await?), - } + Ok(( + RpRead::default(), + DropboxReader::new(self.core.clone(), path, args), + )) } async fn write(&self, path: &str, args: OpWrite) -> Result<(RpWrite, Self::Writer)> { @@ -193,8 +190,8 @@ impl Accessor for DropboxBackend { } let bs = resp.into_body(); - let decoded_response = serde_json::from_slice::(&bs) - .map_err(new_json_deserialize_error)?; + let decoded_response: DropboxDeleteBatchResponse = + serde_json::from_reader(bs.reader()).map_err(new_json_deserialize_error)?; match decoded_response.tag.as_str() { "complete" => { diff --git a/core/src/services/dropbox/core.rs b/core/src/services/dropbox/core.rs index 054f630ef510..9301d37de580 100644 --- a/core/src/services/dropbox/core.rs +++ b/core/src/services/dropbox/core.rs @@ -23,7 +23,7 @@ use std::sync::Arc; use std::time::Duration; use backon::ExponentialBuilder; -use bytes::Bytes; +use bytes::{Buf, Bytes}; use chrono::DateTime; use chrono::Utc; use http::header; @@ -104,7 +104,7 @@ impl DropboxCore { let body = resp.into_body(); let token: DropboxTokenResponse = - serde_json::from_slice(&body).map_err(new_json_deserialize_error)?; + serde_json::from_reader(body.reader()).map_err(new_json_deserialize_error)?; // Update signer after token refreshed. signer.access_token = token.access_token.clone(); @@ -121,7 +121,12 @@ impl DropboxCore { Ok(()) } - pub async fn dropbox_get(&self, path: &str, args: OpRead) -> Result> { + pub async fn dropbox_get( + &self, + path: &str, + range: BytesRange, + _: &OpRead, + ) -> Result> { let url: String = "https://content.dropboxapi.com/2/files/download".to_string(); let download_args = DropboxDownloadArgs { path: build_rooted_abs_path(&self.root, path), @@ -133,7 +138,6 @@ impl DropboxCore { .header("Dropbox-API-Arg", request_payload) .header(CONTENT_LENGTH, 0); - let range = args.range(); if !range.is_full() { req = req.header(header::RANGE, range.to_header()); } @@ -241,8 +245,8 @@ impl DropboxCore { let bs = resp.into_body(); - let decoded_response = serde_json::from_slice::(&bs) - .map_err(new_json_deserialize_error)?; + let decoded_response: DropboxDeleteBatchResponse = + serde_json::from_reader(bs.reader()).map_err(new_json_deserialize_error)?; match decoded_response.tag.as_str() { "in_progress" => Err(Error::new( ErrorKind::Unexpected, diff --git a/core/src/services/dropbox/error.rs b/core/src/services/dropbox/error.rs index 2996f44453c4..a956a6f714fc 100644 --- a/core/src/services/dropbox/error.rs +++ b/core/src/services/dropbox/error.rs @@ -15,6 +15,7 @@ // specific language governing permissions and limitations // under the License. +use bytes::Buf; use http::Response; use http::StatusCode; use serde::Deserialize; diff --git a/core/src/services/dropbox/mod.rs b/core/src/services/dropbox/mod.rs index e056778cb746..3ba91497498d 100644 --- a/core/src/services/dropbox/mod.rs +++ b/core/src/services/dropbox/mod.rs @@ -19,6 +19,7 @@ mod backend; mod builder; mod core; mod error; +mod reader; mod writer; pub use builder::DropboxBuilder as Dropbox; diff --git a/core/src/services/dropbox/reader.rs b/core/src/services/dropbox/reader.rs new file mode 100644 index 000000000000..39d04733fe6c --- /dev/null +++ b/core/src/services/dropbox/reader.rs @@ -0,0 +1,58 @@ +// 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 super::core::DropboxCore; +use super::error::parse_error; +use crate::raw::{oio, OpRead}; +use http::StatusCode; +use std::future::Future; +use std::sync::Arc; + +pub struct DropboxReader { + core: Arc, + + path: String, + op: OpRead, +} + +impl DropboxReader { + pub fn new(core: Arc, path: &str, op: OpRead) -> Self { + DropboxReader { + core, + path: path.to_string(), + op, + } + } +} + +impl oio::Read for DropboxReader { + async fn read_at(&self, offset: u64, limit: usize) -> crate::Result { + let Some(range) = self.op.range().apply_on_offset(offset, limit) else { + return Ok(oio::Buffer::new()); + }; + + let resp = self.core.dropbox_get(&self.path, range, &self.op).await?; + + let status = resp.status(); + + match status { + StatusCode::OK | StatusCode::PARTIAL_CONTENT => Ok(resp.into_body()), + StatusCode::RANGE_NOT_SATISFIABLE => Ok(oio::Buffer::new()), + _ => Err(parse_error(resp).await?), + } + } +} From 9bf2f50736eb8058e6f9013ce5abae13e379bcc3 Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Tue, 19 Mar 2024 20:08:27 +0800 Subject: [PATCH 019/111] Fix ftp Signed-off-by: Xuanwo --- core/src/raw/oio/buf/buffer.rs | 6 ++ core/src/services/ftp/backend.rs | 49 ++------------- core/src/services/ftp/mod.rs | 2 +- core/src/services/ftp/reader.rs | 82 ++++++++++++++++++++++++ core/src/services/ftp/util.rs | 105 ------------------------------- 5 files changed, 93 insertions(+), 151 deletions(-) create mode 100644 core/src/services/ftp/reader.rs delete mode 100644 core/src/services/ftp/util.rs diff --git a/core/src/raw/oio/buf/buffer.rs b/core/src/raw/oio/buf/buffer.rs index 9ec1f65dbc9f..8376f9d14095 100644 --- a/core/src/raw/oio/buf/buffer.rs +++ b/core/src/raw/oio/buf/buffer.rs @@ -32,6 +32,12 @@ impl Buffer { } } +impl From> for Buffer { + fn from(bs: Vec) -> Self { + Self(Inner::Contiguous(bs.into())) + } +} + impl From for Buffer { fn from(bs: Bytes) -> Self { Self(Inner::Contiguous(bs)) diff --git a/core/src/services/ftp/backend.rs b/core/src/services/ftp/backend.rs index e51b64384ba6..6a96caaa1051 100644 --- a/core/src/services/ftp/backend.rs +++ b/core/src/services/ftp/backend.rs @@ -42,10 +42,10 @@ use tokio::sync::OnceCell; use super::err::parse_error; use super::lister::FtpLister; -use super::util::FtpReader; +use super::reader::FtpReader; use super::writer::FtpWriter; +use super::writer::FtpWriters; use crate::raw::*; -use crate::services::ftp::writer::FtpWriters; use crate::*; /// Config for Ftpservices support. @@ -359,49 +359,8 @@ impl Accessor for FtpBackend { Ok(RpStat::new(meta)) } - /// TODO: migrate to FileReader maybe? async fn read(&self, path: &str, args: OpRead) -> Result<(RpRead, Self::Reader)> { - let mut ftp_stream = self.ftp_connect(Operation::Read).await?; - - let meta = self.ftp_stat(path).await?; - - let br = args.range(); - let r: Box = match (br.offset(), br.size()) { - (Some(offset), Some(size)) => { - ftp_stream - .resume_transfer(offset as usize) - .await - .map_err(parse_error)?; - let ds = ftp_stream - .retr_as_stream(path) - .await - .map_err(parse_error)? - .take(size); - Box::new(ds) - } - (Some(offset), None) => { - ftp_stream - .resume_transfer(offset as usize) - .await - .map_err(parse_error)?; - let ds = ftp_stream.retr_as_stream(path).await.map_err(parse_error)?; - Box::new(ds) - } - (None, Some(size)) => { - ftp_stream - .resume_transfer((meta.size() as u64 - size) as usize) - .await - .map_err(parse_error)?; - let ds = ftp_stream.retr_as_stream(path).await.map_err(parse_error)?; - Box::new(ds) - } - (None, None) => { - let ds = ftp_stream.retr_as_stream(path).await.map_err(parse_error)?; - Box::new(ds) - } - }; - - Ok((RpRead::new(), FtpReader::new(r, ftp_stream))) + Ok((RpRead::new(), FtpReader::new(self.clone(), path, args))) } async fn write(&self, path: &str, _: OpWrite) -> Result<(RpWrite, Self::Writer)> { @@ -497,7 +456,7 @@ impl FtpBackend { }) } - async fn ftp_stat(&self, path: &str) -> Result { + pub async fn ftp_stat(&self, path: &str) -> Result { let mut ftp_stream = self.ftp_connect(Operation::Stat).await?; let (parent, basename) = (get_parent(path), get_basename(path)); diff --git a/core/src/services/ftp/mod.rs b/core/src/services/ftp/mod.rs index 85d3d8da9234..35a0ba5734ac 100644 --- a/core/src/services/ftp/mod.rs +++ b/core/src/services/ftp/mod.rs @@ -21,5 +21,5 @@ pub use backend::FtpConfig; mod err; mod lister; -mod util; +mod reader; mod writer; diff --git a/core/src/services/ftp/reader.rs b/core/src/services/ftp/reader.rs new file mode 100644 index 000000000000..7a34165a69a0 --- /dev/null +++ b/core/src/services/ftp/reader.rs @@ -0,0 +1,82 @@ +// 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 super::backend::FtpBackend; +use super::err::parse_error; +use crate::raw::{new_std_io_error, oio, OpRead, Operation}; +use crate::*; +use futures::AsyncReadExt; + +pub struct FtpReader { + core: FtpBackend, + + path: String, + op: OpRead, +} + +impl FtpReader { + pub fn new(core: FtpBackend, path: &str, op: OpRead) -> Self { + FtpReader { + core, + path: path.to_string(), + op: op, + } + } +} + +impl oio::Read for FtpReader { + async fn read_at(&self, offset: u64, limit: usize) -> Result { + let Some(range) = self.op.range().apply_on_offset(offset, limit) else { + return Ok(oio::Buffer::new()); + }; + + let mut ftp_stream = self.core.ftp_connect(Operation::Read).await?; + + let meta = self.core.ftp_stat(&self.path).await?; + + match (range.offset(), range.size()) { + (Some(offset), Some(size)) => { + ftp_stream + .resume_transfer(offset as usize) + .await + .map_err(parse_error)?; + let mut ds = ftp_stream + .retr_as_stream(&self.path) + .await + .map_err(parse_error)? + .take(size); + let mut bs = Vec::with_capacity(size as usize); + ds.read_to_end(&mut bs).await.map_err(new_std_io_error)?; + Ok(oio::Buffer::from(bs)) + } + (Some(offset), None) => { + ftp_stream + .resume_transfer(offset as usize) + .await + .map_err(parse_error)?; + let mut ds = ftp_stream + .retr_as_stream(&self.path) + .await + .map_err(parse_error)?; + let mut bs = vec![]; + ds.read_to_end(&mut bs).await.map_err(new_std_io_error)?; + Ok(oio::Buffer::from(bs)) + } + _ => unimplemented!(), + } + } +} diff --git a/core/src/services/ftp/util.rs b/core/src/services/ftp/util.rs deleted file mode 100644 index 3a7e5abac9fd..000000000000 --- a/core/src/services/ftp/util.rs +++ /dev/null @@ -1,105 +0,0 @@ -// 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; - -use bb8::PooledConnection; -use bytes::Bytes; -use futures::AsyncRead; -use futures::AsyncReadExt; -use suppaftp::Status; -use tokio::io::ReadBuf; - -use super::backend::Manager; -use crate::raw::*; -use crate::services::ftp::err::parse_error; -use crate::*; - -/// Wrapper for ftp data stream and command stream. -pub struct FtpReader { - reader: Box, - conn: Option>, - buf: Vec, -} - -unsafe impl Sync for FtpReader {} - -impl FtpReader { - /// Create an instance of FtpReader. - pub fn new( - reader: Box, - conn: PooledConnection<'static, Manager>, - ) -> Self { - Self { - reader, - conn: Some(conn), - buf: Vec::with_capacity(64 * 1024), - } - } -} - -impl oio::Read for FtpReader { - async fn read(&mut self, limit: usize) -> Result { - if self.conn.is_none() { - return Err(Error::new( - ErrorKind::Unexpected, - "ftp reader is already closed", - )); - } - - // Make sure buf has enough space. - if self.buf.capacity() < limit { - self.buf.reserve(limit); - } - let buf = self.buf.spare_capacity_mut(); - let mut read_buf: ReadBuf = ReadBuf::uninit(buf); - - // SAFETY: Read at most `size` bytes into `read_buf`. - unsafe { - read_buf.assume_init(limit); - } - - let data = self.reader.read(read_buf.initialize_unfilled()).await; - - // Data read with success, copy and return it. - if let Ok(n) = data { - if n > 0 { - read_buf.set_filled(n); - return Ok(Bytes::copy_from_slice(&self.buf[..n])); - } - } - - // While hitting Error or EOF, we should end this ftp stream. - let _ = self - .conn - .take() - .expect("connection must be valid during read") - .read_response_in(&[Status::ClosingDataConnection, Status::RequestedFileActionOk]) - .await - .map_err(parse_error)?; - Ok(Bytes::new()) - } - - async fn seek(&mut self, pos: io::SeekFrom) -> Result { - let _ = pos; - - Err(Error::new( - ErrorKind::Unsupported, - "ftp reader doesn't support seeking", - )) - } -} From 51e0edf302189dd161acce8b2c147c956a7090c7 Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Tue, 19 Mar 2024 20:13:51 +0800 Subject: [PATCH 020/111] Fix gcs Signed-off-by: Xuanwo --- core/src/raw/oio/buf/buffer.rs | 10 +++++- core/src/services/gcs/backend.rs | 22 +++++------- core/src/services/gcs/core.rs | 20 ++++++++--- core/src/services/gcs/error.rs | 1 + core/src/services/gcs/lister.rs | 3 +- core/src/services/gcs/mod.rs | 1 + core/src/services/gcs/reader.rs | 61 ++++++++++++++++++++++++++++++++ 7 files changed, 98 insertions(+), 20 deletions(-) create mode 100644 core/src/services/gcs/reader.rs diff --git a/core/src/raw/oio/buf/buffer.rs b/core/src/raw/oio/buf/buffer.rs index 8376f9d14095..2fa7d960504d 100644 --- a/core/src/raw/oio/buf/buffer.rs +++ b/core/src/raw/oio/buf/buffer.rs @@ -15,11 +15,13 @@ // specific language governing permissions and limitations // under the License. -use bytes::{BufMut, Bytes, BytesMut}; +use bytes::{Buf, BufMut, Bytes, BytesMut}; use std::collections::VecDeque; +#[derive(Clone)] pub struct Buffer(Inner); +#[derive(Clone)] enum Inner { Contiguous(Bytes), NonContiguous(VecDeque), @@ -30,6 +32,12 @@ impl Buffer { pub const fn new() -> Self { Self(Inner::Contiguous(Bytes::new())) } + + #[inline] + pub fn to_bytes(&self) -> Bytes { + let mut bs = self.clone(); + bs.copy_to_bytes(bs.remaining()) + } } impl From> for Buffer { diff --git a/core/src/services/gcs/backend.rs b/core/src/services/gcs/backend.rs index 2c2ac070f928..2d584f222316 100644 --- a/core/src/services/gcs/backend.rs +++ b/core/src/services/gcs/backend.rs @@ -21,6 +21,7 @@ use std::fmt::Formatter; use std::sync::Arc; use async_trait::async_trait; +use bytes::Buf; use http::StatusCode; use log::debug; use reqsign::GoogleCredentialLoader; @@ -33,6 +34,7 @@ use serde_json; use super::core::*; use super::error::parse_error; use super::lister::GcsLister; +use super::reader::GcsReader; use super::writer::GcsWriter; use super::writer::GcsWriters; use crate::raw::*; @@ -332,7 +334,7 @@ pub struct GcsBackend { #[cfg_attr(not(target_arch = "wasm32"), async_trait)] #[cfg_attr(target_arch = "wasm32", async_trait(?Send))] impl Accessor for GcsBackend { - type Reader = oio::Buffer; + type Reader = GcsReader; type Writer = GcsWriters; type Lister = oio::PageLister; type BlockingReader = (); @@ -396,7 +398,7 @@ impl Accessor for GcsBackend { let slc = resp.into_body(); let meta: GetObjectJsonResponse = - serde_json::from_slice(&slc).map_err(new_json_deserialize_error)?; + serde_json::from_reader(slc.reader()).map_err(new_json_deserialize_error)?; let mut m = Metadata::new(EntryMode::FILE); @@ -418,16 +420,10 @@ impl Accessor for GcsBackend { } async fn read(&self, path: &str, args: OpRead) -> Result<(RpRead, Self::Reader)> { - let resp = self.core.gcs_get_object(path, &args).await?; - - if resp.status().is_success() { - let size = parse_content_length(resp.headers())?; - Ok((RpRead::new().with_size(size), resp.into_body())) - } else if resp.status() == StatusCode::RANGE_NOT_SATISFIABLE { - Ok((RpRead::new(), oio::Buffer::empty())) - } else { - Err(parse_error(resp).await?) - } + Ok(( + RpRead::default(), + GcsReader::new(self.core.clone(), path, args), + )) } async fn write(&self, path: &str, args: OpWrite) -> Result<(RpWrite, Self::Writer)> { @@ -527,7 +523,7 @@ impl Accessor for GcsBackend { .trim_matches('"'); let multipart: Multipart = Multipart::new() .with_boundary(boundary) - .parse(resp.into_body().bytes().await?)?; + .parse(resp.into_body().to_bytes())?; let parts = multipart.into_parts(); let mut batched_result = Vec::with_capacity(parts.len()); diff --git a/core/src/services/gcs/core.rs b/core/src/services/gcs/core.rs index 01d9f8d204fb..8935c29cf0e8 100644 --- a/core/src/services/gcs/core.rs +++ b/core/src/services/gcs/core.rs @@ -147,7 +147,12 @@ impl GcsCore { } impl GcsCore { - pub fn gcs_get_object_request(&self, path: &str, args: &OpRead) -> Result> { + pub fn gcs_get_object_request( + &self, + path: &str, + range: BytesRange, + args: &OpRead, + ) -> Result> { let p = build_abs_path(&self.root, path); let url = format!( @@ -165,8 +170,8 @@ impl GcsCore { if let Some(if_none_match) = args.if_none_match() { req = req.header(IF_NONE_MATCH, if_none_match); } - if !args.range().is_full() { - req = req.header(http::header::RANGE, args.range().to_header()); + if !range.is_full() { + req = req.header(http::header::RANGE, range.to_header()); } let req = req @@ -205,8 +210,13 @@ impl GcsCore { Ok(req) } - pub async fn gcs_get_object(&self, path: &str, args: &OpRead) -> Result> { - let mut req = self.gcs_get_object_request(path, args)?; + pub async fn gcs_get_object( + &self, + path: &str, + range: BytesRange, + args: &OpRead, + ) -> Result> { + let mut req = self.gcs_get_object_request(path, range, args)?; self.sign(&mut req).await?; self.send(req).await diff --git a/core/src/services/gcs/error.rs b/core/src/services/gcs/error.rs index f270fc7bd0db..38ffa0486ba4 100644 --- a/core/src/services/gcs/error.rs +++ b/core/src/services/gcs/error.rs @@ -15,6 +15,7 @@ // specific language governing permissions and limitations // under the License. +use bytes::Buf; use http::Response; use http::StatusCode; use serde::Deserialize; diff --git a/core/src/services/gcs/lister.rs b/core/src/services/gcs/lister.rs index 31fbfb3894a4..79ef5c98a4d6 100644 --- a/core/src/services/gcs/lister.rs +++ b/core/src/services/gcs/lister.rs @@ -15,6 +15,7 @@ // specific language governing permissions and limitations // under the License. +use bytes::Buf; use std::sync::Arc; use serde_json; @@ -82,7 +83,7 @@ impl oio::PageList for GcsLister { let bytes = resp.into_body(); let output: ListResponse = - serde_json::from_slice(&bytes).map_err(new_json_deserialize_error)?; + serde_json::from_reader(bytes.reader()).map_err(new_json_deserialize_error)?; if let Some(token) = &output.next_page_token { ctx.token = token.clone(); diff --git a/core/src/services/gcs/mod.rs b/core/src/services/gcs/mod.rs index 1dcaa9ca0632..756b2474728d 100644 --- a/core/src/services/gcs/mod.rs +++ b/core/src/services/gcs/mod.rs @@ -22,5 +22,6 @@ pub use backend::GcsConfig; mod core; mod error; mod lister; +mod reader; mod uri; mod writer; diff --git a/core/src/services/gcs/reader.rs b/core/src/services/gcs/reader.rs new file mode 100644 index 000000000000..60dd7d785a0a --- /dev/null +++ b/core/src/services/gcs/reader.rs @@ -0,0 +1,61 @@ +// 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 super::core::GcsCore; +use super::error::parse_error; +use crate::raw::{oio, OpRead}; +use http::StatusCode; +use std::future::Future; +use std::sync::Arc; + +pub struct GcsReader { + core: Arc, + + path: String, + op: OpRead, +} + +impl GcsReader { + pub fn new(core: Arc, path: &str, op: OpRead) -> Self { + GcsReader { + core, + path: path.to_string(), + op: op, + } + } +} + +impl oio::Read for GcsReader { + async fn read_at(&self, offset: u64, limit: usize) -> crate::Result { + let Some(range) = self.op.range().apply_on_offset(offset, limit) else { + return Ok(oio::Buffer::new()); + }; + + let resp = self + .core + .gcs_get_object(&self.path, range, &self.op) + .await?; + + let status = resp.status(); + + match status { + StatusCode::OK | StatusCode::PARTIAL_CONTENT => Ok(resp.into_body()), + StatusCode::RANGE_NOT_SATISFIABLE => Ok(oio::Buffer::new()), + _ => Err(parse_error(resp).await?), + } + } +} From d9617b3d7df4bd16fee959dfda5375f4d866ba51 Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Tue, 19 Mar 2024 20:19:35 +0800 Subject: [PATCH 021/111] Fix gdrive Signed-off-by: Xuanwo --- core/src/services/gdrive/backend.rs | 33 ++++++---------- core/src/services/gdrive/core.rs | 14 ++++--- core/src/services/gdrive/error.rs | 1 + core/src/services/gdrive/lister.rs | 2 +- core/src/services/gdrive/mod.rs | 1 + core/src/services/gdrive/reader.rs | 58 +++++++++++++++++++++++++++++ core/src/services/gdrive/writer.rs | 2 +- 7 files changed, 81 insertions(+), 30 deletions(-) create mode 100644 core/src/services/gdrive/reader.rs diff --git a/core/src/services/gdrive/backend.rs b/core/src/services/gdrive/backend.rs index 8ce7ecad5e99..247683b862fb 100644 --- a/core/src/services/gdrive/backend.rs +++ b/core/src/services/gdrive/backend.rs @@ -19,19 +19,19 @@ use std::fmt::Debug; use std::sync::Arc; use async_trait::async_trait; -use bytes::Bytes; +use bytes::{Buf, Bytes}; use chrono::Utc; use http::Request; use http::StatusCode; use serde_json::json; use super::core::GdriveCore; +use super::core::GdriveFile; use super::error::parse_error; use super::lister::GdriveLister; +use super::reader::GdriveReader; use super::writer::GdriveWriter; use crate::raw::*; -use crate::services::gdrive::core::GdriveFile; -use crate::types::Result; use crate::*; #[derive(Clone, Debug)] @@ -42,7 +42,7 @@ pub struct GdriveBackend { #[cfg_attr(not(target_arch = "wasm32"), async_trait)] #[cfg_attr(target_arch = "wasm32", async_trait(?Send))] impl Accessor for GdriveBackend { - type Reader = oio::Buffer; + type Reader = GdriveReader; type Writer = oio::OneShotWriter; type Lister = oio::PageLister; type BlockingReader = (); @@ -108,22 +108,11 @@ impl Accessor for GdriveBackend { Ok(RpStat::new(meta)) } - async fn read(&self, path: &str, _args: OpRead) -> Result<(RpRead, Self::Reader)> { - let resp = self.core.gdrive_get(path).await?; - - let status = resp.status(); - - match status { - StatusCode::OK => { - let size = parse_content_length(resp.headers())?; - let range = parse_content_range(resp.headers())?; - Ok(( - RpRead::new().with_size(size).with_range(range), - resp.into_body(), - )) - } - _ => Err(parse_error(resp).await?), - } + async fn read(&self, path: &str, args: OpRead) -> Result<(RpRead, Self::Reader)> { + Ok(( + RpRead::default(), + GdriveReader::new(self.core.clone(), path, args), + )) } async fn write(&self, path: &str, _: OpWrite) -> Result<(RpWrite, Self::Writer)> { @@ -241,8 +230,8 @@ impl Accessor for GdriveBackend { match status { StatusCode::OK => { let body = resp.into_body(); - let meta = serde_json::from_slice::(&body) - .map_err(new_json_deserialize_error)?; + let meta: GdriveFile = + serde_json::from_reader(body.reader()).map_err(new_json_deserialize_error)?; let cache = &self.core.path_cache; diff --git a/core/src/services/gdrive/core.rs b/core/src/services/gdrive/core.rs index e2aa317acedc..3df9e05f453e 100644 --- a/core/src/services/gdrive/core.rs +++ b/core/src/services/gdrive/core.rs @@ -21,7 +21,7 @@ use std::sync::Arc; use async_trait::async_trait; use bytes; -use bytes::Bytes; +use bytes::{Buf, Bytes}; use chrono::DateTime; use chrono::Utc; use http::header; @@ -78,7 +78,7 @@ impl GdriveCore { self.client.send(req).await } - pub async fn gdrive_get(&self, path: &str) -> Result> { + pub async fn gdrive_get(&self, path: &str, range: BytesRange) -> Result> { let path = build_abs_path(&self.root, path); let path_id = self.path_cache.get(&path).await?.ok_or(Error::new( ErrorKind::NotFound, @@ -91,6 +91,7 @@ impl GdriveCore { ); let mut req = Request::get(&url) + .header(header::RANGE, range.to_header()) .body(AsyncBody::Empty) .map_err(new_request_build_error)?; self.sign(&mut req).await?; @@ -310,8 +311,8 @@ impl GdriveSigner { match status { StatusCode::OK => { - let resp_body = &resp.into_body(); - let token = serde_json::from_slice::(resp_body) + let resp_body = resp.into_body(); + let token: GdriveTokenResponse = serde_json::from_reader(resp_body.reader()) .map_err(new_json_deserialize_error)?; self.access_token = token.access_token.clone(); self.expires_in = Utc::now() + chrono::Duration::seconds(token.expires_in) @@ -381,7 +382,7 @@ impl PathQuery for GdrivePathQuery { StatusCode::OK => { let body = resp.into_body(); let meta: GdriveFileList = - serde_json::from_slice(&body).map_err(new_json_deserialize_error)?; + serde_json::from_reader(body.reader()).map_err(new_json_deserialize_error)?; if let Some(f) = meta.files.first() { Ok(Some(f.id.clone())) @@ -417,7 +418,8 @@ impl PathQuery for GdrivePathQuery { } let body = resp.into_body(); - let file: GdriveFile = serde_json::from_slice(&body).map_err(new_json_deserialize_error)?; + let file: GdriveFile = + serde_json::from_reader(body.reader()).map_err(new_json_deserialize_error)?; Ok(file.id) } } diff --git a/core/src/services/gdrive/error.rs b/core/src/services/gdrive/error.rs index 5a2e78a34375..f9e57ecef804 100644 --- a/core/src/services/gdrive/error.rs +++ b/core/src/services/gdrive/error.rs @@ -15,6 +15,7 @@ // specific language governing permissions and limitations // under the License. +use bytes::Buf; use http::Response; use http::StatusCode; use serde::Deserialize; diff --git a/core/src/services/gdrive/lister.rs b/core/src/services/gdrive/lister.rs index 34df9073f388..e77a5022da7a 100644 --- a/core/src/services/gdrive/lister.rs +++ b/core/src/services/gdrive/lister.rs @@ -54,7 +54,7 @@ impl oio::PageList for GdriveLister { .await?; let bytes = match resp.status() { - StatusCode::OK => resp.into_body().bytes().await?, + StatusCode::OK => resp.into_body().to_bytes(), _ => return Err(parse_error(resp).await?), }; diff --git a/core/src/services/gdrive/mod.rs b/core/src/services/gdrive/mod.rs index 015f177b28e0..2b9b906cbc46 100644 --- a/core/src/services/gdrive/mod.rs +++ b/core/src/services/gdrive/mod.rs @@ -22,4 +22,5 @@ mod error; pub use builder::GdriveBuilder as Gdrive; mod lister; +mod reader; mod writer; diff --git a/core/src/services/gdrive/reader.rs b/core/src/services/gdrive/reader.rs new file mode 100644 index 000000000000..afad3cba4176 --- /dev/null +++ b/core/src/services/gdrive/reader.rs @@ -0,0 +1,58 @@ +// 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 super::core::GdriveCore; +use super::error::parse_error; +use crate::raw::{oio, OpRead}; +use http::StatusCode; +use std::future::Future; +use std::sync::Arc; + +pub struct GdriveReader { + core: Arc, + + path: String, + op: OpRead, +} + +impl GdriveReader { + pub fn new(core: Arc, path: &str, op: OpRead) -> Self { + GdriveReader { + core, + path: path.to_string(), + op: op, + } + } +} + +impl oio::Read for GdriveReader { + async fn read_at(&self, offset: u64, limit: usize) -> crate::Result { + let Some(range) = self.op.range().apply_on_offset(offset, limit) else { + return Ok(oio::Buffer::new()); + }; + + let resp = self.core.gdrive_get(&self.path, range).await?; + + let status = resp.status(); + + match status { + StatusCode::OK | StatusCode::PARTIAL_CONTENT => Ok(resp.into_body()), + StatusCode::RANGE_NOT_SATISFIABLE => Ok(oio::Buffer::new()), + _ => Err(parse_error(resp).await?), + } + } +} diff --git a/core/src/services/gdrive/writer.rs b/core/src/services/gdrive/writer.rs index ebafa7dfced9..ed4689cfcb9f 100644 --- a/core/src/services/gdrive/writer.rs +++ b/core/src/services/gdrive/writer.rs @@ -17,7 +17,7 @@ use std::sync::Arc; -use bytes::Bytes; +use bytes::{Buf, Bytes}; use http::StatusCode; use super::core::GdriveCore; From f5a58224093601da7d52ecfb4f07c453f4c77cdf Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Tue, 19 Mar 2024 20:25:18 +0800 Subject: [PATCH 022/111] Fix ghac Signed-off-by: Xuanwo --- core/src/services/ghac/backend.rs | 35 +++++---------- core/src/services/ghac/error.rs | 1 + core/src/services/ghac/mod.rs | 1 + core/src/services/ghac/reader.rs | 59 +++++++++++++++++++++++++ core/src/services/github/core.rs | 2 +- core/src/services/swift/lister.rs | 2 +- core/src/services/vercel_blob/core.rs | 2 +- core/src/services/yandex_disk/core.rs | 4 +- core/src/services/yandex_disk/lister.rs | 2 +- 9 files changed, 78 insertions(+), 30 deletions(-) create mode 100644 core/src/services/ghac/reader.rs diff --git a/core/src/services/ghac/backend.rs b/core/src/services/ghac/backend.rs index 09c29988fd8d..4fc3adbf7754 100644 --- a/core/src/services/ghac/backend.rs +++ b/core/src/services/ghac/backend.rs @@ -19,7 +19,7 @@ use std::collections::HashMap; use std::env; use async_trait::async_trait; -use bytes::Bytes; +use bytes::{Buf, Bytes}; use http::header; use http::header::ACCEPT; use http::header::AUTHORIZATION; @@ -37,6 +37,7 @@ use serde::Serialize; use super::error::parse_error; use super::writer::GhacWriter; use crate::raw::*; +use crate::services::ghac::reader::GhacReader; use crate::*; /// The base url for cache url. @@ -227,7 +228,7 @@ pub struct GhacBackend { #[async_trait] impl Accessor for GhacBackend { - type Reader = oio::Buffer; + type Reader = GhacReader; type Writer = GhacWriter; type Lister = (); type BlockingReader = (); @@ -268,7 +269,7 @@ impl Accessor for GhacBackend { let location = if resp.status() == StatusCode::OK { let slc = resp.into_body(); let query_resp: GhacQueryResponse = - serde_json::from_slice(&slc).map_err(new_json_deserialize_error)?; + serde_json::from_reader(slc.reader()).map_err(new_json_deserialize_error)?; query_resp.archive_location } else { return Err(parse_error(resp).await?); @@ -306,30 +307,16 @@ impl Accessor for GhacBackend { let location = if resp.status() == StatusCode::OK { let slc = resp.into_body(); let query_resp: GhacQueryResponse = - serde_json::from_slice(&slc).map_err(new_json_deserialize_error)?; + serde_json::from_reader(slc.reader()).map_err(new_json_deserialize_error)?; query_resp.archive_location } else { return Err(parse_error(resp).await?); }; - let req = self.ghac_get_location(&location, args.range()).await?; - let resp = self.client.send(req).await?; - - let status = resp.status(); - match status { - StatusCode::OK | StatusCode::PARTIAL_CONTENT => { - let size = parse_content_length(resp.headers())?; - let range = parse_content_range(resp.headers())?; - Ok(( - RpRead::new().with_size(size).with_range(range), - resp.into_body(), - )) - } - StatusCode::RANGE_NOT_SATISFIABLE => { - Ok((RpRead::new().with_size(Some(0)), oio::Buffer::empty())) - } - _ => Err(parse_error(resp).await?), - } + Ok(( + RpRead::default(), + GhacReader::new(self.clone(), &location, args), + )) } async fn write(&self, path: &str, _: OpWrite) -> Result<(RpWrite, Self::Writer)> { @@ -340,7 +327,7 @@ impl Accessor for GhacBackend { let cache_id = if resp.status().is_success() { let slc = resp.into_body(); let reserve_resp: GhacReserveResponse = - serde_json::from_slice(&slc).map_err(new_json_deserialize_error)?; + serde_json::from_reader(slc.reader()).map_err(new_json_deserialize_error)?; reserve_resp.cache_id } else { return Err(parse_error(resp) @@ -392,7 +379,7 @@ impl GhacBackend { Ok(req) } - async fn ghac_get_location( + pub async fn ghac_get_location( &self, location: &str, range: BytesRange, diff --git a/core/src/services/ghac/error.rs b/core/src/services/ghac/error.rs index 27f99902cae3..c84004e6cd29 100644 --- a/core/src/services/ghac/error.rs +++ b/core/src/services/ghac/error.rs @@ -15,6 +15,7 @@ // specific language governing permissions and limitations // under the License. +use bytes::Buf; use http::Response; use http::StatusCode; diff --git a/core/src/services/ghac/mod.rs b/core/src/services/ghac/mod.rs index b819c409dd78..8d5fe5623d69 100644 --- a/core/src/services/ghac/mod.rs +++ b/core/src/services/ghac/mod.rs @@ -19,4 +19,5 @@ mod backend; pub use backend::GhacBuilder as Ghac; mod error; +mod reader; mod writer; diff --git a/core/src/services/ghac/reader.rs b/core/src/services/ghac/reader.rs new file mode 100644 index 000000000000..590ad0903e2b --- /dev/null +++ b/core/src/services/ghac/reader.rs @@ -0,0 +1,59 @@ +// 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 super::error::parse_error; +use crate::raw::{oio, OpRead}; +use crate::services::ghac::backend::GhacBackend; +use http::StatusCode; +use std::future::Future; +use std::sync::Arc; + +pub struct GhacReader { + core: GhacBackend, + + location: String, + op: OpRead, +} + +impl GhacReader { + pub fn new(core: GhacBackend, location: &str, op: OpRead) -> Self { + GhacReader { + core, + location: location.to_string(), + op: op, + } + } +} + +impl oio::Read for GhacReader { + async fn read_at(&self, offset: u64, limit: usize) -> crate::Result { + let Some(range) = self.op.range().apply_on_offset(offset, limit) else { + return Ok(oio::Buffer::new()); + }; + + let req = self.core.ghac_get_location(&self.location, range).await?; + let resp = self.core.client.send(req).await?; + + let status = resp.status(); + + match status { + StatusCode::OK | StatusCode::PARTIAL_CONTENT => Ok(resp.into_body()), + StatusCode::RANGE_NOT_SATISFIABLE => Ok(oio::Buffer::new()), + _ => Err(parse_error(resp).await?), + } + } +} diff --git a/core/src/services/github/core.rs b/core/src/services/github/core.rs index 454a4c7d1dd9..56a02156aa6e 100644 --- a/core/src/services/github/core.rs +++ b/core/src/services/github/core.rs @@ -243,7 +243,7 @@ impl GithubCore { StatusCode::OK => { let body = resp.into_body(); let resp: ListResponse = - serde_json::from_slice(&body).map_err(new_json_deserialize_error)?; + serde_json::from_reader(body.reader()).map_err(new_json_deserialize_error)?; Ok(resp.entries) } diff --git a/core/src/services/swift/lister.rs b/core/src/services/swift/lister.rs index 79b8fdfa6f92..3dda024144ca 100644 --- a/core/src/services/swift/lister.rs +++ b/core/src/services/swift/lister.rs @@ -57,7 +57,7 @@ impl oio::PageList for SwiftLister { let bytes = response.into_body().bytes().await?; let decoded_response: Vec = - serde_json::from_slice(&bytes).map_err(new_json_deserialize_error)?; + serde_json::from_reader(bytes.reader()).map_err(new_json_deserialize_error)?; // Update token and done based on resp. if let Some(entry) = decoded_response.last() { diff --git a/core/src/services/vercel_blob/core.rs b/core/src/services/vercel_blob/core.rs index 0b52bc9655aa..2ee1ad5c42d0 100644 --- a/core/src/services/vercel_blob/core.rs +++ b/core/src/services/vercel_blob/core.rs @@ -268,7 +268,7 @@ impl VercelBlobCore { let body = resp.into_body(); let resp: ListResponse = - serde_json::from_slice(&body).map_err(new_json_deserialize_error)?; + serde_json::from_reader(body.reader()).map_err(new_json_deserialize_error)?; Ok(resp) } diff --git a/core/src/services/yandex_disk/core.rs b/core/src/services/yandex_disk/core.rs index 76162913b8fb..ab658b3498bc 100644 --- a/core/src/services/yandex_disk/core.rs +++ b/core/src/services/yandex_disk/core.rs @@ -90,7 +90,7 @@ impl YandexDiskCore { let bytes = resp.into_body(); let resp: GetUploadUrlResponse = - serde_json::from_slice(&bytes).map_err(new_json_deserialize_error)?; + serde_json::from_reader(bytes.reader()).map_err(new_json_deserialize_error)?; Ok(resp.href) } @@ -124,7 +124,7 @@ impl YandexDiskCore { let bytes = resp.into_body(); let resp: GetUploadUrlResponse = - serde_json::from_slice(&bytes).map_err(new_json_deserialize_error)?; + serde_json::from_reader(bytes.reader()).map_err(new_json_deserialize_error)?; Ok(resp.href) } diff --git a/core/src/services/yandex_disk/lister.rs b/core/src/services/yandex_disk/lister.rs index f51d8da7966e..26addb5433fa 100644 --- a/core/src/services/yandex_disk/lister.rs +++ b/core/src/services/yandex_disk/lister.rs @@ -65,7 +65,7 @@ impl oio::PageList for YandexDiskLister { let body = resp.into_body(); let resp: MetainformationResponse = - serde_json::from_slice(&body).map_err(new_json_deserialize_error)?; + serde_json::from_reader(body.reader()).map_err(new_json_deserialize_error)?; if let Some(embedded) = resp.embedded { let n = embedded.items.len(); From 7b56daccfa802ec1afa884b852b02f86528b5a10 Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Tue, 19 Mar 2024 20:27:43 +0800 Subject: [PATCH 023/111] fix github Signed-off-by: Xuanwo --- core/src/services/github/backend.rs | 24 ++++-------- core/src/services/github/core.rs | 7 ++-- core/src/services/github/mod.rs | 1 + core/src/services/github/reader.rs | 58 +++++++++++++++++++++++++++++ 4 files changed, 70 insertions(+), 20 deletions(-) create mode 100644 core/src/services/github/reader.rs diff --git a/core/src/services/github/backend.rs b/core/src/services/github/backend.rs index 796e345e8b3a..b9a3d6e5c37e 100644 --- a/core/src/services/github/backend.rs +++ b/core/src/services/github/backend.rs @@ -31,6 +31,7 @@ use super::lister::GithubLister; use super::writer::GithubWriter; use super::writer::GithubWriters; use crate::raw::*; +use crate::services::github::reader::GithubReader; use crate::*; /// Config for backblaze Github services support. @@ -221,7 +222,7 @@ pub struct GithubBackend { #[async_trait] impl Accessor for GithubBackend { - type Reader = oio::Buffer; + type Reader = GithubReader; type Writer = GithubWriters; @@ -284,22 +285,11 @@ impl Accessor for GithubBackend { } } - async fn read(&self, path: &str, _args: OpRead) -> Result<(RpRead, Self::Reader)> { - let resp = self.core.get(path).await?; - - let status = resp.status(); - - match status { - StatusCode::OK => { - let size = parse_content_length(resp.headers())?; - let range = parse_content_range(resp.headers())?; - Ok(( - RpRead::new().with_size(size).with_range(range), - resp.into_body(), - )) - } - _ => Err(parse_error(resp).await?), - } + async fn read(&self, path: &str, args: OpRead) -> Result<(RpRead, Self::Reader)> { + Ok(( + RpRead::default(), + GithubReader::new(self.core.clone(), path, args), + )) } async fn write(&self, path: &str, _args: OpWrite) -> Result<(RpWrite, Self::Writer)> { diff --git a/core/src/services/github/core.rs b/core/src/services/github/core.rs index 56a02156aa6e..a5419da9bb23 100644 --- a/core/src/services/github/core.rs +++ b/core/src/services/github/core.rs @@ -19,7 +19,7 @@ use std::fmt::Debug; use std::fmt::Formatter; use base64::Engine; -use bytes::Bytes; +use bytes::{Buf, Bytes}; use http::header; use http::request; use http::Request; @@ -122,7 +122,7 @@ impl GithubCore { self.send(req).await } - pub async fn get(&self, path: &str) -> Result> { + pub async fn get(&self, path: &str, range: BytesRange) -> Result> { let path = build_abs_path(&self.root, path); let url = format!( @@ -137,7 +137,8 @@ impl GithubCore { let req = self.sign(req)?; let req = req - .header("Accept", "application/vnd.github.raw+json") + .header(header::ACCEPT, "application/vnd.github.raw+json") + .header(header::RANGE, range.to_header()) .body(AsyncBody::Empty) .map_err(new_request_build_error)?; diff --git a/core/src/services/github/mod.rs b/core/src/services/github/mod.rs index 06418c19347b..9855cb0808d4 100644 --- a/core/src/services/github/mod.rs +++ b/core/src/services/github/mod.rs @@ -22,4 +22,5 @@ pub use backend::GithubConfig; mod core; mod error; mod lister; +mod reader; mod writer; diff --git a/core/src/services/github/reader.rs b/core/src/services/github/reader.rs new file mode 100644 index 000000000000..31561188e43d --- /dev/null +++ b/core/src/services/github/reader.rs @@ -0,0 +1,58 @@ +// 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 super::core::GithubCore; +use super::error::parse_error; +use crate::raw::{oio, OpRead}; +use http::StatusCode; +use std::future::Future; +use std::sync::Arc; + +pub struct GithubReader { + core: Arc, + + path: String, + op: OpRead, +} + +impl GithubReader { + pub fn new(core: Arc, path: &str, op: OpRead) -> Self { + GithubReader { + core, + path: path.to_string(), + op: op, + } + } +} + +impl oio::Read for GithubReader { + async fn read_at(&self, offset: u64, limit: usize) -> crate::Result { + let Some(range) = self.op.range().apply_on_offset(offset, limit) else { + return Ok(oio::Buffer::new()); + }; + + let resp = self.core.get(&self.path, range).await?; + + let status = resp.status(); + + match status { + StatusCode::OK | StatusCode::PARTIAL_CONTENT => Ok(resp.into_body()), + StatusCode::RANGE_NOT_SATISFIABLE => Ok(oio::Buffer::new()), + _ => Err(parse_error(resp).await?), + } + } +} From c017f4b98aa012ef05ed99b6d2318a63a6d06259 Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Tue, 19 Mar 2024 20:32:22 +0800 Subject: [PATCH 024/111] Fix hdfs Signed-off-by: Xuanwo --- core/src/services/hdfs/backend.rs | 7 +++--- core/src/services/hdfs/mod.rs | 1 + core/src/services/hdfs/reader.rs | 41 +++++++++++++++++++++++++++++++ core/src/services/mod.rs | 4 +-- 4 files changed, 47 insertions(+), 6 deletions(-) create mode 100644 core/src/services/hdfs/reader.rs diff --git a/core/src/services/hdfs/backend.rs b/core/src/services/hdfs/backend.rs index b1eb2e57a5ab..9d94f048bfef 100644 --- a/core/src/services/hdfs/backend.rs +++ b/core/src/services/hdfs/backend.rs @@ -31,6 +31,7 @@ use uuid::Uuid; use super::lister::HdfsLister; use super::writer::HdfsWriter; use crate::raw::*; +use crate::services::hdfs::reader::HdfsReader; use crate::*; /// [Hadoop Distributed File System (HDFS™)](https://hadoop.apache.org/) support. @@ -245,7 +246,7 @@ unsafe impl Sync for HdfsBackend {} #[async_trait] impl Accessor for HdfsBackend { - type Reader = oio::FuturesReader; + type Reader = HdfsReader; type Writer = HdfsWriter; type Lister = Option; type BlockingReader = oio::StdReader; @@ -317,9 +318,7 @@ impl Accessor for HdfsBackend { .await .map_err(new_std_io_error)?; - let r = oio::FuturesReader::new(f); - - Ok((RpRead::new(), r)) + todo!() } async fn write(&self, path: &str, op: OpWrite) -> Result<(RpWrite, Self::Writer)> { diff --git a/core/src/services/hdfs/mod.rs b/core/src/services/hdfs/mod.rs index f5d6506fb4c6..df55c9c3e01a 100644 --- a/core/src/services/hdfs/mod.rs +++ b/core/src/services/hdfs/mod.rs @@ -20,3 +20,4 @@ pub use backend::HdfsBuilder as Hdfs; pub use backend::HdfsConfig; mod lister; mod writer; +mod r#reader diff --git a/core/src/services/hdfs/reader.rs b/core/src/services/hdfs/reader.rs new file mode 100644 index 000000000000..2566b802e9a7 --- /dev/null +++ b/core/src/services/hdfs/reader.rs @@ -0,0 +1,41 @@ +// 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 crate::raw::{oio, OpRead}; +use http::StatusCode; +use std::future::Future; +use std::sync::Arc; + +pub struct HdfsReader { + path: String, + op: OpRead, +} + +impl HdfsReader { + pub fn new(path: &str, op: OpRead) -> Self { + HdfsReader { + path: path.to_string(), + op, + } + } +} + +impl oio::Read for HdfsReader { + async fn read_at(&self, offset: u64, limit: usize) -> crate::Result { + todo!() + } +} diff --git a/core/src/services/mod.rs b/core/src/services/mod.rs index 780dab54f4d0..a009bd538a5c 100644 --- a/core/src/services/mod.rs +++ b/core/src/services/mod.rs @@ -85,9 +85,9 @@ pub use gridfs::Gridfs; #[cfg(feature = "services-hdfs")] mod hdfs; #[cfg(feature = "services-hdfs")] -pub use hdfs::Hdfs; +pub use self::hdfs::Hdfs; #[cfg(feature = "services-hdfs")] -pub use hdfs::HdfsConfig; +pub use self::hdfs::HdfsConfig; #[cfg(feature = "services-http")] mod http; From 75a0f5bd048931df66edfb83e52f78eb4ccddbbb Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Tue, 19 Mar 2024 20:35:39 +0800 Subject: [PATCH 025/111] Fix http Signed-off-by: Xuanwo --- core/src/services/hdfs_native/reader.rs | 17 ++------ core/src/services/http/backend.rs | 33 +++++--------- core/src/services/http/error.rs | 1 + core/src/services/http/mod.rs | 1 + core/src/services/http/reader.rs | 58 +++++++++++++++++++++++++ 5 files changed, 74 insertions(+), 36 deletions(-) create mode 100644 core/src/services/http/reader.rs diff --git a/core/src/services/hdfs_native/reader.rs b/core/src/services/hdfs_native/reader.rs index 9ae5ed020a9c..f6e078cba182 100644 --- a/core/src/services/hdfs_native/reader.rs +++ b/core/src/services/hdfs_native/reader.rs @@ -20,7 +20,7 @@ use std::io::SeekFrom; use bytes::Bytes; use hdfs_native::file::FileReader; -use crate::raw::oio::Read; +use crate::raw::oio; use crate::*; pub struct HdfsNativeReader { @@ -33,19 +33,8 @@ impl HdfsNativeReader { } } -impl Read for HdfsNativeReader { - async fn read(&mut self, limit: usize) -> Result { - let _ = limit; - +impl oio::Read for HdfsNativeReader { + async fn read_at(&self, offset: u64, limit: usize) -> crate::Result { todo!() } - - async fn seek(&mut self, pos: SeekFrom) -> Result { - let _ = pos; - - Err(Error::new( - ErrorKind::Unsupported, - "HdfsNativeReader doesn't support seeking", - )) - } } diff --git a/core/src/services/http/backend.rs b/core/src/services/http/backend.rs index 92293b9c2b5c..60e80cef448c 100644 --- a/core/src/services/http/backend.rs +++ b/core/src/services/http/backend.rs @@ -31,6 +31,7 @@ use serde::Deserialize; use super::error::parse_error; use crate::raw::*; +use crate::services::http::reader::HttpReader; use crate::*; /// Config for Http service support. @@ -223,7 +224,7 @@ impl Debug for HttpBackend { #[async_trait] impl Accessor for HttpBackend { - type Reader = oio::Buffer; + type Reader = HttpReader; type Writer = (); type Lister = (); type BlockingReader = (); @@ -273,29 +274,17 @@ impl Accessor for HttpBackend { } async fn read(&self, path: &str, args: OpRead) -> Result<(RpRead, Self::Reader)> { - let resp = self.http_get(path, &args).await?; - - let status = resp.status(); - - match status { - StatusCode::OK | StatusCode::PARTIAL_CONTENT => { - let size = parse_content_length(resp.headers())?; - let range = parse_content_range(resp.headers())?; - Ok(( - RpRead::new().with_size(size).with_range(range), - resp.into_body(), - )) - } - StatusCode::RANGE_NOT_SATISFIABLE => { - Ok((RpRead::new().with_size(Some(0)), oio::Buffer::empty())) - } - _ => Err(parse_error(resp).await?), - } + Ok((RpRead::default(), HttpReader::new(self.clone(), path, args))) } } impl HttpBackend { - async fn http_get(&self, path: &str, args: &OpRead) -> Result> { + pub async fn http_get( + &self, + path: &str, + range: BytesRange, + args: &OpRead, + ) -> Result> { let p = build_rooted_abs_path(&self.root, path); let url = format!("{}{}", self.endpoint, percent_encode_path(&p)); @@ -314,8 +303,8 @@ impl HttpBackend { req = req.header(header::AUTHORIZATION, auth.clone()) } - if !args.range().is_full() { - req = req.header(header::RANGE, args.range().to_header()); + if !range.is_full() { + req = req.header(header::RANGE, range.to_header()); } let req = req diff --git a/core/src/services/http/error.rs b/core/src/services/http/error.rs index 336ca8d8798d..451f89d10cd1 100644 --- a/core/src/services/http/error.rs +++ b/core/src/services/http/error.rs @@ -15,6 +15,7 @@ // specific language governing permissions and limitations // under the License. +use bytes::Buf; use http::Response; use http::StatusCode; diff --git a/core/src/services/http/mod.rs b/core/src/services/http/mod.rs index 7424d97c2b9b..4bab91f879ae 100644 --- a/core/src/services/http/mod.rs +++ b/core/src/services/http/mod.rs @@ -20,3 +20,4 @@ pub use backend::HttpBuilder as Http; pub use backend::HttpConfig; mod error; +mod reader; diff --git a/core/src/services/http/reader.rs b/core/src/services/http/reader.rs new file mode 100644 index 000000000000..f6266b30d688 --- /dev/null +++ b/core/src/services/http/reader.rs @@ -0,0 +1,58 @@ +// 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 super::error::parse_error; +use crate::raw::{oio, OpRead}; +use crate::services::http::backend::HttpBackend; +use http::StatusCode; +use std::future::Future; +use std::sync::Arc; + +pub struct HttpReader { + core: HttpBackend, + + path: String, + op: OpRead, +} + +impl HttpReader { + pub fn new(core: HttpBackend, path: &str, op: OpRead) -> Self { + HttpReader { + core, + path: path.to_string(), + op: op, + } + } +} + +impl oio::Read for HttpReader { + async fn read_at(&self, offset: u64, limit: usize) -> crate::Result { + let Some(range) = self.op.range().apply_on_offset(offset, limit) else { + return Ok(oio::Buffer::new()); + }; + + let resp = self.core.http_get(&self.path, range, &self.op).await?; + + let status = resp.status(); + + match status { + StatusCode::OK | StatusCode::PARTIAL_CONTENT => Ok(resp.into_body()), + StatusCode::RANGE_NOT_SATISFIABLE => Ok(oio::Buffer::new()), + _ => Err(parse_error(resp).await?), + } + } +} From 1c1da49968adf6ec75eb36bedb505921357e195c Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Tue, 19 Mar 2024 20:38:41 +0800 Subject: [PATCH 026/111] Fix hf Signed-off-by: Xuanwo --- core/src/services/huggingface/backend.rs | 30 ++++-------- core/src/services/huggingface/core.rs | 10 ++-- core/src/services/huggingface/error.rs | 1 + core/src/services/huggingface/lister.rs | 7 +-- core/src/services/huggingface/mod.rs | 1 + core/src/services/huggingface/reader.rs | 58 ++++++++++++++++++++++++ 6 files changed, 80 insertions(+), 27 deletions(-) create mode 100644 core/src/services/huggingface/reader.rs diff --git a/core/src/services/huggingface/backend.rs b/core/src/services/huggingface/backend.rs index 3f9187efff17..671f6e5755af 100644 --- a/core/src/services/huggingface/backend.rs +++ b/core/src/services/huggingface/backend.rs @@ -21,6 +21,7 @@ use std::fmt::Formatter; use std::sync::Arc; use async_trait::async_trait; +use bytes::Buf; use http::StatusCode; use log::debug; use serde::Deserialize; @@ -30,6 +31,7 @@ use super::core::HuggingfaceStatus; use super::error::parse_error; use super::lister::HuggingfaceLister; use crate::raw::*; +use crate::services::huggingface::reader::HuggingfaceReader; use crate::*; /// Configuration for Huggingface service support. @@ -243,7 +245,7 @@ pub struct HuggingfaceBackend { #[async_trait] impl Accessor for HuggingfaceBackend { - type Reader = oio::Buffer; + type Reader = HuggingfaceReader; type Writer = (); type Lister = oio::PageLister; type BlockingReader = (); @@ -283,8 +285,8 @@ impl Accessor for HuggingfaceBackend { let mut meta = parse_into_metadata(path, resp.headers())?; let bs = resp.into_body(); - let decoded_response = serde_json::from_slice::>(&bs) - .map_err(new_json_deserialize_error)?; + let decoded_response: Vec = + serde_json::from_reader(bs.reader()).map_err(new_json_deserialize_error)?; // NOTE: if the file is not found, the server will return 200 with an empty array if let Some(status) = decoded_response.first() { @@ -312,24 +314,10 @@ impl Accessor for HuggingfaceBackend { } async fn read(&self, path: &str, args: OpRead) -> Result<(RpRead, Self::Reader)> { - let resp = self.core.hf_resolve(path, args).await?; - - let status = resp.status(); - - match status { - StatusCode::OK | StatusCode::PARTIAL_CONTENT => { - let size = parse_content_length(resp.headers())?; - let range = parse_content_range(resp.headers())?; - Ok(( - RpRead::new().with_size(size).with_range(range), - resp.into_body(), - )) - } - StatusCode::RANGE_NOT_SATISFIABLE => { - Ok((RpRead::new().with_size(Some(0)), oio::Buffer::empty())) - } - _ => Err(parse_error(resp).await?), - } + Ok(( + RpRead::default(), + HuggingfaceReader::new(self.core.clone(), path, args), + )) } async fn list(&self, path: &str, args: OpList) -> Result<(RpList, Self::Lister)> { diff --git a/core/src/services/huggingface/core.rs b/core/src/services/huggingface/core.rs index f7aab77acec3..2557b552d8b6 100644 --- a/core/src/services/huggingface/core.rs +++ b/core/src/services/huggingface/core.rs @@ -121,7 +121,12 @@ impl HuggingfaceCore { self.client.send(req).await } - pub async fn hf_resolve(&self, path: &str, arg: OpRead) -> Result> { + pub async fn hf_resolve( + &self, + path: &str, + range: BytesRange, + args: &OpRead, + ) -> Result> { let p = build_abs_path(&self.root, path) .trim_end_matches('/') .to_string(); @@ -148,9 +153,8 @@ impl HuggingfaceCore { req = req.header(header::AUTHORIZATION, auth_header_content); } - let range = arg.range(); if !range.is_full() { - req = req.header(header::RANGE, &range.to_header()); + req = req.header(header::RANGE, range.to_header()); } let req = req diff --git a/core/src/services/huggingface/error.rs b/core/src/services/huggingface/error.rs index c69c03e5e5a9..2a2e49e55bdd 100644 --- a/core/src/services/huggingface/error.rs +++ b/core/src/services/huggingface/error.rs @@ -15,6 +15,7 @@ // specific language governing permissions and limitations // under the License. +use bytes::Buf; use std::fmt::Debug; use http::Response; diff --git a/core/src/services/huggingface/lister.rs b/core/src/services/huggingface/lister.rs index 836b62481a41..9c7555e7967b 100644 --- a/core/src/services/huggingface/lister.rs +++ b/core/src/services/huggingface/lister.rs @@ -15,6 +15,7 @@ // specific language governing permissions and limitations // under the License. +use bytes::Buf; use std::sync::Arc; use super::core::HuggingfaceCore; @@ -49,9 +50,9 @@ impl oio::PageList for HuggingfaceLister { return Err(error); } - let bytes = response.into_body().bytes().await?; - let decoded_response = serde_json::from_slice::>(&bytes) - .map_err(new_json_deserialize_error)?; + let bytes = response.into_body(); + let decoded_response: Vec = + serde_json::from_reader(bytes.reader()).map_err(new_json_deserialize_error)?; ctx.done = true; diff --git a/core/src/services/huggingface/mod.rs b/core/src/services/huggingface/mod.rs index 3a692fa45e53..73149c15ad3f 100644 --- a/core/src/services/huggingface/mod.rs +++ b/core/src/services/huggingface/mod.rs @@ -22,3 +22,4 @@ pub use backend::HuggingfaceConfig; mod core; mod error; mod lister; +mod reader; diff --git a/core/src/services/huggingface/reader.rs b/core/src/services/huggingface/reader.rs new file mode 100644 index 000000000000..db2d06374398 --- /dev/null +++ b/core/src/services/huggingface/reader.rs @@ -0,0 +1,58 @@ +// 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 +// +// Huggingface://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 super::core::HuggingfaceCore; +use super::error::parse_error; +use crate::raw::{oio, OpRead}; +use http::StatusCode; +use std::future::Future; +use std::sync::Arc; + +pub struct HuggingfaceReader { + core: Arc, + + path: String, + op: OpRead, +} + +impl HuggingfaceReader { + pub fn new(core: Arc, path: &str, op: OpRead) -> Self { + HuggingfaceReader { + core, + path: path.to_string(), + op: op, + } + } +} + +impl oio::Read for HuggingfaceReader { + async fn read_at(&self, offset: u64, limit: usize) -> crate::Result { + let Some(range) = self.op.range().apply_on_offset(offset, limit) else { + return Ok(oio::Buffer::new()); + }; + + let resp = self.core.hf_resolve(&self.path, range, &self.op).await?; + + let status = resp.status(); + + match status { + StatusCode::OK | StatusCode::PARTIAL_CONTENT => Ok(resp.into_body()), + StatusCode::RANGE_NOT_SATISFIABLE => Ok(oio::Buffer::new()), + _ => Err(parse_error(resp).await?), + } + } +} From c08f2dc3ef65b6f9469fd5e2ff3e84e610972da9 Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Tue, 19 Mar 2024 20:41:45 +0800 Subject: [PATCH 027/111] Fix icloud Signed-off-by: Xuanwo --- core/src/services/huggingface/reader.rs | 2 +- core/src/services/icloud/backend.rs | 24 +++-------- core/src/services/icloud/core.rs | 13 ++++-- core/src/services/icloud/mod.rs | 1 + core/src/services/icloud/reader.rs | 57 +++++++++++++++++++++++++ 5 files changed, 74 insertions(+), 23 deletions(-) create mode 100644 core/src/services/icloud/reader.rs diff --git a/core/src/services/huggingface/reader.rs b/core/src/services/huggingface/reader.rs index db2d06374398..4b2172db8b67 100644 --- a/core/src/services/huggingface/reader.rs +++ b/core/src/services/huggingface/reader.rs @@ -6,7 +6,7 @@ // "License"); you may not use this file except in compliance // with the License. You may obtain a copy of the License at // -// Huggingface://www.apache.org/licenses/LICENSE-2.0 +// 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 diff --git a/core/src/services/icloud/backend.rs b/core/src/services/icloud/backend.rs index c2c7b174ff4f..bc94a39a7df0 100644 --- a/core/src/services/icloud/backend.rs +++ b/core/src/services/icloud/backend.rs @@ -28,6 +28,7 @@ use tokio::sync::Mutex; use super::core::*; use crate::raw::*; use crate::*; +use crate::services::icloud::reader::IcloudReader; /// Config for icloud services support. #[derive(Default, Deserialize)] @@ -267,7 +268,7 @@ pub struct IcloudBackend { #[async_trait] impl Accessor for IcloudBackend { - type Reader = oio::Buffer; + type Reader = IcloudReader; type BlockingReader = (); type Writer = (); type BlockingWriter = (); @@ -312,22 +313,9 @@ impl Accessor for IcloudBackend { } async fn read(&self, path: &str, args: OpRead) -> Result<(RpRead, Self::Reader)> { - let resp = self.core.read(path, &args).await?; - let status = resp.status(); - - match status { - StatusCode::OK | StatusCode::PARTIAL_CONTENT => { - let size = parse_content_length(resp.headers())?; - let range = parse_content_range(resp.headers())?; - Ok(( - RpRead::new().with_size(size).with_range(range), - resp.into_body(), - )) - } - StatusCode::RANGE_NOT_SATISFIABLE => { - Ok((RpRead::new().with_size(Some(0)), oio::Buffer::empty())) - } - _ => Err(parse_error(resp).await?), - } + Ok(( + RpRead::default(), + IcloudReader::new(self.core.clone(), path, args), + )) } } diff --git a/core/src/services/icloud/core.rs b/core/src/services/icloud/core.rs index 697f235e8da6..6cef03f9624d 100644 --- a/core/src/services/icloud/core.rs +++ b/core/src/services/icloud/core.rs @@ -366,6 +366,7 @@ impl IcloudCore { &self, id: &str, zone: &str, + range: BytesRange, args: OpRead, ) -> Result> { let mut signer = self.signer.lock().await; @@ -398,8 +399,7 @@ impl IcloudCore { req = req.header(IF_MATCH, if_match); } - let range = args.range(); - if !range.is_full() { + if range.is_full() { req = req.header(header::RANGE, range.to_header()) } if let Some(if_none_match) = args.if_none_match() { @@ -415,7 +415,12 @@ impl IcloudCore { Ok(resp) } - pub async fn read(&self, path: &str, args: &OpRead) -> Result> { + pub async fn read( + &self, + path: &str, + range: BytesRange, + args: &OpRead, + ) -> Result> { let path = build_rooted_abs_path(&self.root, path); let base = get_basename(&path); @@ -426,7 +431,7 @@ impl IcloudCore { if let Some(docwsid) = path_id.strip_prefix("FILE::com.apple.CloudDocs::") { Ok(self - .get_file(docwsid, "com.apple.CloudDocs", args.clone()) + .get_file(docwsid, "com.apple.CloudDocs", range, args.clone()) .await?) } else { Err(Error::new( diff --git a/core/src/services/icloud/mod.rs b/core/src/services/icloud/mod.rs index d76c934ddde2..9366484735e8 100644 --- a/core/src/services/icloud/mod.rs +++ b/core/src/services/icloud/mod.rs @@ -19,3 +19,4 @@ mod backend; pub use backend::IcloudBuilder as Icloud; mod core; +mod reader; diff --git a/core/src/services/icloud/reader.rs b/core/src/services/icloud/reader.rs new file mode 100644 index 000000000000..4309ee0f1525 --- /dev/null +++ b/core/src/services/icloud/reader.rs @@ -0,0 +1,57 @@ +// 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 super::core::{parse_error, IcloudCore}; +use crate::raw::{oio, OpRead}; +use http::StatusCode; +use std::future::Future; +use std::sync::Arc; + +pub struct IcloudReader { + core: Arc, + + path: String, + op: OpRead, +} + +impl IcloudReader { + pub fn new(core: Arc, path: &str, op: OpRead) -> Self { + IcloudReader { + core, + path: path.to_string(), + op: op, + } + } +} + +impl oio::Read for IcloudReader { + async fn read_at(&self, offset: u64, limit: usize) -> crate::Result { + let Some(range) = self.op.range().apply_on_offset(offset, limit) else { + return Ok(oio::Buffer::new()); + }; + + let resp = self.core.read(&self.path, range, &self.op).await?; + + let status = resp.status(); + + match status { + StatusCode::OK | StatusCode::PARTIAL_CONTENT => Ok(resp.into_body()), + StatusCode::RANGE_NOT_SATISFIABLE => Ok(oio::Buffer::new()), + _ => Err(parse_error(resp).await?), + } + } +} From 22bdad63bc0b677657ae3c57d3fe21de11d99914 Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Tue, 19 Mar 2024 20:43:36 +0800 Subject: [PATCH 028/111] Fix ipfs Signed-off-by: Xuanwo --- core/src/services/icloud/backend.rs | 2 +- core/src/services/ipfs/backend.rs | 14 ++----- core/src/services/ipfs/error.rs | 1 + core/src/services/ipfs/mod.rs | 1 + core/src/services/ipfs/reader.rs | 58 +++++++++++++++++++++++++++++ 5 files changed, 65 insertions(+), 11 deletions(-) create mode 100644 core/src/services/ipfs/reader.rs diff --git a/core/src/services/icloud/backend.rs b/core/src/services/icloud/backend.rs index bc94a39a7df0..a414f92ca228 100644 --- a/core/src/services/icloud/backend.rs +++ b/core/src/services/icloud/backend.rs @@ -27,8 +27,8 @@ use tokio::sync::Mutex; use super::core::*; use crate::raw::*; -use crate::*; use crate::services::icloud::reader::IcloudReader; +use crate::*; /// Config for icloud services support. #[derive(Default, Deserialize)] diff --git a/core/src/services/ipfs/backend.rs b/core/src/services/ipfs/backend.rs index aa83e5d8de26..88e89fda1c88 100644 --- a/core/src/services/ipfs/backend.rs +++ b/core/src/services/ipfs/backend.rs @@ -30,6 +30,7 @@ use prost::Message; use super::error::parse_error; use super::ipld::PBNode; use crate::raw::*; +use crate::services::ipfs::reader::IpfsReader; use crate::*; /// IPFS file system support based on [IPFS HTTP Gateway](https://docs.ipfs.tech/concepts/ipfs-gateway/). @@ -161,7 +162,7 @@ impl Debug for IpfsBackend { #[async_trait] impl Accessor for IpfsBackend { - type Reader = oio::Buffer; + type Reader = IpfsReader; type Writer = (); type Lister = oio::PageLister; type BlockingReader = (); @@ -339,14 +340,7 @@ impl Accessor for IpfsBackend { } async fn read(&self, path: &str, args: OpRead) -> Result<(RpRead, Self::Reader)> { - let resp = self.ipfs_get(path, args.range()).await?; - - let status = resp.status(); - - match status { - StatusCode::OK | StatusCode::PARTIAL_CONTENT => Ok((RpRead::new(), resp.into_body())), - _ => Err(parse_error(resp).await?), - } + Ok((RpRead::default(), IpfsReader::new(self.clone(), path, args))) } async fn list(&self, path: &str, _: OpList) -> Result<(RpList, Self::Lister)> { @@ -356,7 +350,7 @@ impl Accessor for IpfsBackend { } impl IpfsBackend { - async fn ipfs_get(&self, path: &str, range: BytesRange) -> Result> { + pub async fn ipfs_get(&self, path: &str, range: BytesRange) -> Result> { let p = build_rooted_abs_path(&self.root, path); let url = format!("{}{}", self.endpoint, percent_encode_path(&p)); diff --git a/core/src/services/ipfs/error.rs b/core/src/services/ipfs/error.rs index a526369c2e3f..99177e9b388c 100644 --- a/core/src/services/ipfs/error.rs +++ b/core/src/services/ipfs/error.rs @@ -15,6 +15,7 @@ // specific language governing permissions and limitations // under the License. +use bytes::Buf; use http::Response; use http::StatusCode; diff --git a/core/src/services/ipfs/mod.rs b/core/src/services/ipfs/mod.rs index 4bd46822fbb3..85aeb674f001 100644 --- a/core/src/services/ipfs/mod.rs +++ b/core/src/services/ipfs/mod.rs @@ -20,3 +20,4 @@ pub use backend::IpfsBuilder as Ipfs; mod error; mod ipld; +mod reader; diff --git a/core/src/services/ipfs/reader.rs b/core/src/services/ipfs/reader.rs new file mode 100644 index 000000000000..c43ee0f1c23d --- /dev/null +++ b/core/src/services/ipfs/reader.rs @@ -0,0 +1,58 @@ +// 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 super::error::parse_error; +use crate::raw::{oio, OpRead}; +use crate::services::ipfs::backend::IpfsBackend; +use http::StatusCode; +use std::future::Future; +use std::sync::Arc; + +pub struct IpfsReader { + core: IpfsBackend, + + path: String, + op: OpRead, +} + +impl IpfsReader { + pub fn new(core: IpfsBackend, path: &str, op: OpRead) -> Self { + IpfsReader { + core, + path: path.to_string(), + op: op, + } + } +} + +impl oio::Read for IpfsReader { + async fn read_at(&self, offset: u64, limit: usize) -> crate::Result { + let Some(range) = self.op.range().apply_on_offset(offset, limit) else { + return Ok(oio::Buffer::new()); + }; + + let resp = self.core.ipfs_get(&self.path, range).await?; + + let status = resp.status(); + + match status { + StatusCode::OK | StatusCode::PARTIAL_CONTENT => Ok(resp.into_body()), + StatusCode::RANGE_NOT_SATISFIABLE => Ok(oio::Buffer::new()), + _ => Err(parse_error(resp).await?), + } + } +} From 40ab28a18ea47529d36963bf06ee4aec792a4f5d Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Tue, 19 Mar 2024 20:45:19 +0800 Subject: [PATCH 029/111] Fix ipmfs Signed-off-by: Xuanwo --- core/src/services/ipmfs/backend.rs | 19 +++++----- core/src/services/ipmfs/error.rs | 1 + core/src/services/ipmfs/lister.rs | 1 + core/src/services/ipmfs/mod.rs | 1 + core/src/services/ipmfs/reader.rs | 58 ++++++++++++++++++++++++++++++ 5 files changed, 69 insertions(+), 11 deletions(-) create mode 100644 core/src/services/ipmfs/reader.rs diff --git a/core/src/services/ipmfs/backend.rs b/core/src/services/ipmfs/backend.rs index bab025555f15..ed561e9663e4 100644 --- a/core/src/services/ipmfs/backend.rs +++ b/core/src/services/ipmfs/backend.rs @@ -21,7 +21,7 @@ use std::str; use std::sync::Arc; use async_trait::async_trait; -use bytes::Bytes; +use bytes::{Buf, Bytes}; use http::Request; use http::Response; use http::StatusCode; @@ -31,6 +31,7 @@ use super::error::parse_error; use super::lister::IpmfsLister; use super::writer::IpmfsWriter; use crate::raw::*; +use crate::services::ipmfs::reader::IpmfsReader; use crate::*; /// IPFS Mutable File System (IPMFS) backend. @@ -63,7 +64,7 @@ impl IpmfsBackend { #[async_trait] impl Accessor for IpmfsBackend { - type Reader = oio::Buffer; + type Reader = IpmfsReader; type Writer = oio::OneShotWriter; type Lister = oio::PageLister; type BlockingReader = (); @@ -136,14 +137,10 @@ impl Accessor for IpmfsBackend { } async fn read(&self, path: &str, args: OpRead) -> Result<(RpRead, Self::Reader)> { - let resp = self.ipmfs_read(path, args.range()).await?; - - let status = resp.status(); - - match status { - StatusCode::OK => Ok((RpRead::new(), resp.into_body())), - _ => Err(parse_error(resp).await?), - } + Ok(( + RpRead::default(), + IpmfsReader::new(self.clone(), path, args), + )) } async fn write(&self, path: &str, _: OpWrite) -> Result<(RpWrite, Self::Writer)> { @@ -188,7 +185,7 @@ impl IpmfsBackend { self.client.send(req).await } - async fn ipmfs_read(&self, path: &str, range: BytesRange) -> Result> { + pub async fn ipmfs_read(&self, path: &str, range: BytesRange) -> Result> { let p = build_rooted_abs_path(&self.root, path); let mut url = format!( diff --git a/core/src/services/ipmfs/error.rs b/core/src/services/ipmfs/error.rs index edf3e0f4f23e..ee07566e1bdf 100644 --- a/core/src/services/ipmfs/error.rs +++ b/core/src/services/ipmfs/error.rs @@ -15,6 +15,7 @@ // specific language governing permissions and limitations // under the License. +use bytes::Buf; use http::Response; use http::StatusCode; use serde::Deserialize; diff --git a/core/src/services/ipmfs/lister.rs b/core/src/services/ipmfs/lister.rs index 4288b284233c..5a66c7ba9cae 100644 --- a/core/src/services/ipmfs/lister.rs +++ b/core/src/services/ipmfs/lister.rs @@ -15,6 +15,7 @@ // specific language governing permissions and limitations // under the License. +use bytes::Buf; use std::sync::Arc; use http::StatusCode; diff --git a/core/src/services/ipmfs/mod.rs b/core/src/services/ipmfs/mod.rs index 2a7ce5c0ad71..2e511206ca52 100644 --- a/core/src/services/ipmfs/mod.rs +++ b/core/src/services/ipmfs/mod.rs @@ -21,4 +21,5 @@ pub use builder::IpmfsBuilder as Ipmfs; mod error; mod lister; +mod reader; mod writer; diff --git a/core/src/services/ipmfs/reader.rs b/core/src/services/ipmfs/reader.rs new file mode 100644 index 000000000000..ecb7f41dbba6 --- /dev/null +++ b/core/src/services/ipmfs/reader.rs @@ -0,0 +1,58 @@ +// 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 super::error::parse_error; +use crate::raw::{oio, OpRead}; +use crate::services::ipmfs::backend::IpmfsBackend; +use http::StatusCode; +use std::future::Future; +use std::sync::Arc; + +pub struct IpmfsReader { + core: IpmfsBackend, + + path: String, + op: OpRead, +} + +impl IpmfsReader { + pub fn new(core: IpmfsBackend, path: &str, op: OpRead) -> Self { + IpmfsReader { + core, + path: path.to_string(), + op: op, + } + } +} + +impl oio::Read for IpmfsReader { + async fn read_at(&self, offset: u64, limit: usize) -> crate::Result { + let Some(range) = self.op.range().apply_on_offset(offset, limit) else { + return Ok(oio::Buffer::new()); + }; + + let resp = self.core.ipmfs_read(&self.path, range).await?; + + let status = resp.status(); + + match status { + StatusCode::OK | StatusCode::PARTIAL_CONTENT => Ok(resp.into_body()), + StatusCode::RANGE_NOT_SATISFIABLE => Ok(oio::Buffer::new()), + _ => Err(parse_error(resp).await?), + } + } +} From f8c57055628c8a7e308b8f7aedeea799862b941d Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Tue, 19 Mar 2024 20:49:32 +0800 Subject: [PATCH 030/111] Fix koofr Signed-off-by: Xuanwo --- core/src/services/koofr/backend.rs | 27 +++++--------- core/src/services/koofr/core.rs | 6 ++-- core/src/services/koofr/error.rs | 1 + core/src/services/koofr/lister.rs | 1 + core/src/services/koofr/mod.rs | 1 + core/src/services/koofr/reader.rs | 58 ++++++++++++++++++++++++++++++ 6 files changed, 73 insertions(+), 21 deletions(-) create mode 100644 core/src/services/koofr/reader.rs diff --git a/core/src/services/koofr/backend.rs b/core/src/services/koofr/backend.rs index b5cc6031b34d..91e08a1e7507 100644 --- a/core/src/services/koofr/backend.rs +++ b/core/src/services/koofr/backend.rs @@ -12,7 +12,7 @@ // 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 +// specific language governing permissions and limi tations // under the License. use std::collections::HashMap; @@ -21,6 +21,7 @@ use std::fmt::Formatter; use std::sync::Arc; use async_trait::async_trait; +use bytes::Buf; use http::StatusCode; use log::debug; use serde::Deserialize; @@ -35,6 +36,7 @@ use super::lister::KoofrLister; use super::writer::KoofrWriter; use super::writer::KoofrWriters; use crate::raw::*; +use crate::services::koofr::reader::KoofrReader; use crate::*; /// Config for backblaze Koofr services support. @@ -235,7 +237,7 @@ pub struct KoofrBackend { #[async_trait] impl Accessor for KoofrBackend { - type Reader = oio::Buffer; + type Reader = KoofrReader; type Writer = KoofrWriters; type Lister = oio::PageLister; type BlockingReader = (); @@ -309,22 +311,11 @@ impl Accessor for KoofrBackend { } } - async fn read(&self, path: &str, _args: OpRead) -> Result<(RpRead, Self::Reader)> { - let resp = self.core.get(path).await?; - - let status = resp.status(); - - match status { - StatusCode::OK | StatusCode::PARTIAL_CONTENT => { - let size = parse_content_length(resp.headers())?; - let range = parse_content_range(resp.headers())?; - Ok(( - RpRead::new().with_size(size).with_range(range), - resp.into_body(), - )) - } - _ => Err(parse_error(resp).await?), - } + async fn read(&self, path: &str, args: OpRead) -> Result<(RpRead, Self::Reader)> { + Ok(( + RpRead::default(), + KoofrReader::new(self.core.clone(), path, args), + )) } async fn write(&self, path: &str, _args: OpWrite) -> Result<(RpWrite, Self::Writer)> { diff --git a/core/src/services/koofr/core.rs b/core/src/services/koofr/core.rs index 7daf730d1915..bc836b3affd6 100644 --- a/core/src/services/koofr/core.rs +++ b/core/src/services/koofr/core.rs @@ -20,7 +20,7 @@ use std::fmt::Debug; use std::fmt::Formatter; use std::sync::Arc; -use bytes::Bytes; +use bytes::{Buf, Bytes}; use http::header; use http::request; use http::Request; @@ -241,7 +241,7 @@ impl KoofrCore { self.send(req).await } - pub async fn get(&self, path: &str) -> Result> { + pub async fn get(&self, path: &str, range: BytesRange) -> Result> { let path = build_rooted_abs_path(&self.root, path); let mount_id = self.get_mount_id().await?; @@ -253,7 +253,7 @@ impl KoofrCore { percent_encode_path(&path) ); - let req = Request::get(url); + let req = Request::get(url).header(header::RANGE, range.to_header()); let req = self.sign(req).await?; diff --git a/core/src/services/koofr/error.rs b/core/src/services/koofr/error.rs index bbea17cf9268..ec09b7640cee 100644 --- a/core/src/services/koofr/error.rs +++ b/core/src/services/koofr/error.rs @@ -15,6 +15,7 @@ // specific language governing permissions and limitations // under the License. +use bytes::Buf; use http::Response; use crate::raw::*; diff --git a/core/src/services/koofr/lister.rs b/core/src/services/koofr/lister.rs index 09babe589c2c..e2dcd9a997ac 100644 --- a/core/src/services/koofr/lister.rs +++ b/core/src/services/koofr/lister.rs @@ -15,6 +15,7 @@ // specific language governing permissions and limitations // under the License. +use bytes::Buf; use std::sync::Arc; use super::core::KoofrCore; diff --git a/core/src/services/koofr/mod.rs b/core/src/services/koofr/mod.rs index 445d69fbb82e..fce486b246b9 100644 --- a/core/src/services/koofr/mod.rs +++ b/core/src/services/koofr/mod.rs @@ -22,4 +22,5 @@ pub use backend::KoofrConfig; mod core; mod error; mod lister; +mod reader; mod writer; diff --git a/core/src/services/koofr/reader.rs b/core/src/services/koofr/reader.rs new file mode 100644 index 000000000000..081034afbfe1 --- /dev/null +++ b/core/src/services/koofr/reader.rs @@ -0,0 +1,58 @@ +// 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 super::core::KoofrCore; +use super::error::parse_error; +use crate::raw::{oio, OpRead}; +use http::StatusCode; +use std::future::Future; +use std::sync::Arc; + +pub struct KoofrReader { + core: Arc, + + path: String, + op: OpRead, +} + +impl KoofrReader { + pub fn new(core: Arc, path: &str, op: OpRead) -> Self { + KoofrReader { + core, + path: path.to_string(), + op: op, + } + } +} + +impl oio::Read for KoofrReader { + async fn read_at(&self, offset: u64, limit: usize) -> crate::Result { + let Some(range) = self.op.range().apply_on_offset(offset, limit) else { + return Ok(oio::Buffer::new()); + }; + + let resp = self.core.get(&self.path, range).await?; + + let status = resp.status(); + + match status { + StatusCode::OK | StatusCode::PARTIAL_CONTENT => Ok(resp.into_body()), + StatusCode::RANGE_NOT_SATISFIABLE => Ok(oio::Buffer::new()), + _ => Err(parse_error(resp).await?), + } + } +} From 62ef0ec41e9df31224a3313a0a646d8641ccb8a8 Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Tue, 19 Mar 2024 20:50:10 +0800 Subject: [PATCH 031/111] Fix libsql Signed-off-by: Xuanwo --- core/src/services/libsql/backend.rs | 2 +- core/src/services/libsql/error.rs | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/core/src/services/libsql/backend.rs b/core/src/services/libsql/backend.rs index e1f3c461b4b1..e9f406ecb02a 100644 --- a/core/src/services/libsql/backend.rs +++ b/core/src/services/libsql/backend.rs @@ -20,7 +20,7 @@ use std::fmt::Debug; use std::str; use async_trait::async_trait; -use bytes::Bytes; +use bytes::{Buf, Bytes}; use hrana_client_proto::pipeline::ClientMsg; use hrana_client_proto::pipeline::Response; use hrana_client_proto::pipeline::ServerMsg; diff --git a/core/src/services/libsql/error.rs b/core/src/services/libsql/error.rs index 2ab1000c6f1d..59804a6e4fd5 100644 --- a/core/src/services/libsql/error.rs +++ b/core/src/services/libsql/error.rs @@ -15,6 +15,7 @@ // specific language governing permissions and limitations // under the License. +use bytes::Buf; use http::Response; use http::StatusCode; From 98558fbee456d290428fd4a77140f8709b08d037 Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Tue, 19 Mar 2024 20:55:51 +0800 Subject: [PATCH 032/111] Fix obs and onedrive Signed-off-by: Xuanwo --- core/src/services/obs/backend.rs | 30 +++++-------- core/src/services/obs/core.rs | 19 ++++++--- core/src/services/obs/mod.rs | 1 + core/src/services/obs/reader.rs | 61 +++++++++++++++++++++++++++ core/src/services/onedrive/backend.rs | 38 +++++++---------- core/src/services/onedrive/error.rs | 1 + core/src/services/onedrive/lister.rs | 5 ++- core/src/services/onedrive/mod.rs | 1 + core/src/services/onedrive/reader.rs | 58 +++++++++++++++++++++++++ core/src/services/onedrive/writer.rs | 2 +- 10 files changed, 166 insertions(+), 50 deletions(-) create mode 100644 core/src/services/obs/reader.rs create mode 100644 core/src/services/onedrive/reader.rs diff --git a/core/src/services/obs/backend.rs b/core/src/services/obs/backend.rs index fcac8b94aea6..7f4ff2633628 100644 --- a/core/src/services/obs/backend.rs +++ b/core/src/services/obs/backend.rs @@ -32,6 +32,7 @@ use super::error::parse_error; use super::lister::ObsLister; use super::writer::ObsWriter; use crate::raw::*; +use crate::services::obs::reader::ObsReader; use crate::services::obs::writer::ObsWriters; use crate::*; @@ -248,7 +249,7 @@ pub struct ObsBackend { #[async_trait] impl Accessor for ObsBackend { - type Reader = oio::Buffer; + type Reader = ObsReader; type Writer = ObsWriters; type Lister = oio::PageLister; type BlockingReader = (); @@ -323,24 +324,10 @@ impl Accessor for ObsBackend { } async fn read(&self, path: &str, args: OpRead) -> Result<(RpRead, Self::Reader)> { - let resp = self.core.obs_get_object(path, &args).await?; - - let status = resp.status(); - - match status { - StatusCode::OK | StatusCode::PARTIAL_CONTENT => { - let size = parse_content_length(resp.headers())?; - let range = parse_content_range(resp.headers())?; - Ok(( - RpRead::new().with_size(size).with_range(range), - resp.into_body(), - )) - } - StatusCode::RANGE_NOT_SATISFIABLE => { - Ok((RpRead::new().with_size(Some(0)), oio::Buffer::empty())) - } - _ => Err(parse_error(resp).await?), - } + Ok(( + RpRead::default(), + ObsReader::new(self.core.clone(), path, args), + )) } async fn write(&self, path: &str, args: OpWrite) -> Result<(RpWrite, Self::Writer)> { @@ -387,7 +374,10 @@ impl Accessor for ObsBackend { async fn presign(&self, path: &str, args: OpPresign) -> Result { let mut req = match args.operation() { PresignOperation::Stat(v) => self.core.obs_head_object_request(path, v)?, - PresignOperation::Read(v) => self.core.obs_get_object_request(path, v)?, + PresignOperation::Read(v) => { + self.core + .obs_get_object_request(path, BytesRange::default(), v)? + } PresignOperation::Write(v) => { self.core .obs_put_object_request(path, None, v, AsyncBody::Empty)? diff --git a/core/src/services/obs/core.rs b/core/src/services/obs/core.rs index 71633d603d06..ca01ee4135c5 100644 --- a/core/src/services/obs/core.rs +++ b/core/src/services/obs/core.rs @@ -101,15 +101,25 @@ impl ObsCore { } impl ObsCore { - pub async fn obs_get_object(&self, path: &str, args: &OpRead) -> Result> { - let mut req = self.obs_get_object_request(path, args)?; + pub async fn obs_get_object( + &self, + path: &str, + range: BytesRange, + args: &OpRead, + ) -> Result> { + let mut req = self.obs_get_object_request(path, range, args)?; self.sign(&mut req).await?; self.send(req).await } - pub fn obs_get_object_request(&self, path: &str, args: &OpRead) -> Result> { + pub fn obs_get_object_request( + &self, + path: &str, + range: BytesRange, + args: &OpRead, + ) -> Result> { let p = build_abs_path(&self.root, path); let url = format!("{}/{}", self.endpoint, percent_encode_path(&p)); @@ -120,8 +130,7 @@ impl ObsCore { req = req.header(IF_MATCH, if_match); } - let range = args.range(); - if !range.is_full() { + if range.is_full() { req = req.header(http::header::RANGE, range.to_header()) } diff --git a/core/src/services/obs/mod.rs b/core/src/services/obs/mod.rs index f8a78290f0b5..93eb1328daca 100644 --- a/core/src/services/obs/mod.rs +++ b/core/src/services/obs/mod.rs @@ -21,4 +21,5 @@ pub use backend::ObsBuilder as Obs; mod core; mod error; mod lister; +mod reader; mod writer; diff --git a/core/src/services/obs/reader.rs b/core/src/services/obs/reader.rs new file mode 100644 index 000000000000..cb9c083a9cc7 --- /dev/null +++ b/core/src/services/obs/reader.rs @@ -0,0 +1,61 @@ +// 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 super::core::ObsCore; +use super::error::parse_error; +use crate::raw::{oio, OpRead}; +use http::StatusCode; +use std::future::Future; +use std::sync::Arc; + +pub struct ObsReader { + core: Arc, + + path: String, + op: OpRead, +} + +impl ObsReader { + pub fn new(core: Arc, path: &str, op: OpRead) -> Self { + ObsReader { + core, + path: path.to_string(), + op: op, + } + } +} + +impl oio::Read for ObsReader { + async fn read_at(&self, offset: u64, limit: usize) -> crate::Result { + let Some(range) = self.op.range().apply_on_offset(offset, limit) else { + return Ok(oio::Buffer::new()); + }; + + let resp = self + .core + .obs_get_object(&self.path, range, &self.op) + .await?; + + let status = resp.status(); + + match status { + StatusCode::OK | StatusCode::PARTIAL_CONTENT => Ok(resp.into_body()), + StatusCode::RANGE_NOT_SATISFIABLE => Ok(oio::Buffer::new()), + _ => Err(parse_error(resp).await?), + } + } +} diff --git a/core/src/services/onedrive/backend.rs b/core/src/services/onedrive/backend.rs index c3e41ebc99bf..91bd4c111947 100644 --- a/core/src/services/onedrive/backend.rs +++ b/core/src/services/onedrive/backend.rs @@ -18,7 +18,7 @@ use std::fmt::Debug; use async_trait::async_trait; -use bytes::Bytes; +use bytes::{Buf, Bytes}; use http::header; use http::Request; use http::Response; @@ -32,6 +32,7 @@ use super::graph_model::OnedriveGetItemBody; use super::lister::OnedriveLister; use super::writer::OneDriveWriter; use crate::raw::*; +use crate::services::onedrive::reader::OnedriveReader; use crate::*; #[derive(Clone)] @@ -62,7 +63,7 @@ impl Debug for OnedriveBackend { #[async_trait] impl Accessor for OnedriveBackend { - type Reader = oio::Buffer; + type Reader = OnedriveReader; type Writer = oio::OneShotWriter; type Lister = oio::PageLister; type BlockingReader = (); @@ -121,8 +122,8 @@ impl Accessor for OnedriveBackend { if status.is_success() { let bytes = resp.into_body(); - let decoded_response = serde_json::from_slice::(&bytes) - .map_err(new_json_deserialize_error)?; + let decoded_response: OnedriveGetItemBody = + serde_json::from_reader(bytes.reader()).map_err(new_json_deserialize_error)?; let entry_mode: EntryMode = match decoded_response.item_type { ItemType::Folder { .. } => EntryMode::DIR, @@ -149,22 +150,11 @@ impl Accessor for OnedriveBackend { } } - async fn read(&self, path: &str, _args: OpRead) -> Result<(RpRead, Self::Reader)> { - let resp = self.onedrive_get_content(path).await?; - - let status = resp.status(); - - match status { - StatusCode::OK | StatusCode::PARTIAL_CONTENT => { - let size = parse_content_length(resp.headers())?; - let range = parse_content_range(resp.headers())?; - Ok(( - RpRead::new().with_size(size).with_range(range), - resp.into_body(), - )) - } - _ => Err(parse_error(resp).await?), - } + async fn read(&self, path: &str, args: OpRead) -> Result<(RpRead, Self::Reader)> { + Ok(( + RpRead::default(), + OnedriveReader::new(self.clone(), path, args), + )) } async fn write(&self, path: &str, args: OpWrite) -> Result<(RpWrite, Self::Writer)> { @@ -235,7 +225,11 @@ impl OnedriveBackend { self.client.send(req).await } - async fn onedrive_get_content(&self, path: &str) -> Result> { + pub async fn onedrive_get_content( + &self, + path: &str, + range: BytesRange, + ) -> Result> { let path = build_rooted_abs_path(&self.root, path); let url: String = format!( "https://graph.microsoft.com/v1.0/me/drive/root:{}{}", @@ -243,7 +237,7 @@ impl OnedriveBackend { ":/content" ); - let mut req = Request::get(&url); + let mut req = Request::get(&url).header(header::RANGE, range.to_header()); let auth_header_content = format!("Bearer {}", self.access_token); req = req.header(header::AUTHORIZATION, auth_header_content); diff --git a/core/src/services/onedrive/error.rs b/core/src/services/onedrive/error.rs index 94d592fd5652..30a90f7a4f92 100644 --- a/core/src/services/onedrive/error.rs +++ b/core/src/services/onedrive/error.rs @@ -15,6 +15,7 @@ // specific language governing permissions and limitations // under the License. +use bytes::Buf; use http::Response; use http::StatusCode; diff --git a/core/src/services/onedrive/lister.rs b/core/src/services/onedrive/lister.rs index 64ca0b5cfc3c..f2bc24ddb0b8 100644 --- a/core/src/services/onedrive/lister.rs +++ b/core/src/services/onedrive/lister.rs @@ -22,6 +22,7 @@ use super::graph_model::ItemType; use crate::raw::oio; use crate::raw::*; use crate::*; +use bytes::Buf; pub struct OnedriveLister { root: String, @@ -77,8 +78,8 @@ impl oio::PageList for OnedriveLister { } let bytes = resp.into_body(); - let decoded_response = serde_json::from_slice::(&bytes) - .map_err(new_json_deserialize_error)?; + let decoded_response: GraphApiOnedriveListResponse = + serde_json::from_reader(bytes.reader()).map_err(new_json_deserialize_error)?; if let Some(next_link) = decoded_response.next_link { ctx.token = next_link; diff --git a/core/src/services/onedrive/mod.rs b/core/src/services/onedrive/mod.rs index 672fb6c7e1e7..b5211c55e4a0 100644 --- a/core/src/services/onedrive/mod.rs +++ b/core/src/services/onedrive/mod.rs @@ -24,4 +24,5 @@ pub use builder::OnedriveBuilder as Onedrive; pub use builder::OnedriveConfig; mod lister; +mod reader; mod writer; diff --git a/core/src/services/onedrive/reader.rs b/core/src/services/onedrive/reader.rs new file mode 100644 index 000000000000..72c0c9fe7572 --- /dev/null +++ b/core/src/services/onedrive/reader.rs @@ -0,0 +1,58 @@ +// 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 super::error::parse_error; +use crate::raw::{oio, OpRead}; +use crate::services::onedrive::backend::OnedriveBackend; +use http::StatusCode; +use std::future::Future; +use std::sync::Arc; + +pub struct OnedriveReader { + core: OnedriveBackend, + + path: String, + op: OpRead, +} + +impl OnedriveReader { + pub fn new(core: OnedriveBackend, path: &str, op: OpRead) -> Self { + OnedriveReader { + core, + path: path.to_string(), + op: op, + } + } +} + +impl oio::Read for OnedriveReader { + async fn read_at(&self, offset: u64, limit: usize) -> crate::Result { + let Some(range) = self.op.range().apply_on_offset(offset, limit) else { + return Ok(oio::Buffer::new()); + }; + + let resp = self.core.onedrive_get_content(&self.path, range).await?; + + let status = resp.status(); + + match status { + StatusCode::OK | StatusCode::PARTIAL_CONTENT => Ok(resp.into_body()), + StatusCode::RANGE_NOT_SATISFIABLE => Ok(oio::Buffer::new()), + _ => Err(parse_error(resp).await?), + } + } +} diff --git a/core/src/services/onedrive/writer.rs b/core/src/services/onedrive/writer.rs index b626a7bf4cd2..05d0250f042a 100644 --- a/core/src/services/onedrive/writer.rs +++ b/core/src/services/onedrive/writer.rs @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -use bytes::Bytes; +use bytes::{Buf, Bytes}; use http::StatusCode; use super::backend::OnedriveBackend; From 84a81dfb54001209998572e1a59c67a3ba6380dc Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Tue, 19 Mar 2024 20:58:55 +0800 Subject: [PATCH 033/111] Fix oss Signed-off-by: Xuanwo --- core/src/services/oss/backend.rs | 46 ++++++------------------ core/src/services/oss/core.rs | 23 ++++-------- core/src/services/oss/mod.rs | 1 + core/src/services/oss/reader.rs | 61 ++++++++++++++++++++++++++++++++ 4 files changed, 78 insertions(+), 53 deletions(-) create mode 100644 core/src/services/oss/reader.rs diff --git a/core/src/services/oss/backend.rs b/core/src/services/oss/backend.rs index a2965184e883..58cf0d600d59 100644 --- a/core/src/services/oss/backend.rs +++ b/core/src/services/oss/backend.rs @@ -35,6 +35,7 @@ use super::error::parse_error; use super::lister::OssLister; use super::writer::OssWriter; use crate::raw::*; +use crate::services::oss::reader::OssReader; use crate::services::oss::writer::OssWriters; use crate::*; @@ -376,7 +377,7 @@ pub struct OssBackend { #[async_trait] impl Accessor for OssBackend { - type Reader = oio::Buffer; + type Reader = OssReader; type Writer = OssWriters; type Lister = oio::PageLister; type BlockingReader = (); @@ -456,33 +457,10 @@ impl Accessor for OssBackend { } async fn read(&self, path: &str, args: OpRead) -> Result<(RpRead, Self::Reader)> { - let resp = self - .core - .oss_get_object( - path, - args.range(), - args.if_match(), - args.if_none_match(), - args.override_content_disposition(), - ) - .await?; - - let status = resp.status(); - - match status { - StatusCode::OK | StatusCode::PARTIAL_CONTENT => { - let size = parse_content_length(resp.headers())?; - let range = parse_content_range(resp.headers())?; - Ok(( - RpRead::new().with_size(size).with_range(range), - resp.into_body(), - )) - } - StatusCode::RANGE_NOT_SATISFIABLE => { - Ok((RpRead::new().with_size(Some(0)), oio::Buffer::empty())) - } - _ => Err(parse_error(resp).await?), - } + Ok(( + RpRead::default(), + OssReader::new(self.core.clone(), path, args), + )) } async fn write(&self, path: &str, args: OpWrite) -> Result<(RpWrite, Self::Writer)> { @@ -534,14 +512,10 @@ impl Accessor for OssBackend { self.core .oss_head_object_request(path, true, v.if_match(), v.if_none_match())? } - PresignOperation::Read(v) => self.core.oss_get_object_request( - path, - v.range(), - true, - v.if_match(), - v.if_none_match(), - v.override_content_disposition(), - )?, + PresignOperation::Read(v) => { + self.core + .oss_get_object_request(path, BytesRange::default(), true, v)? + } PresignOperation::Write(v) => { self.core .oss_put_object_request(path, None, v, AsyncBody::Empty, true)? diff --git a/core/src/services/oss/core.rs b/core/src/services/oss/core.rs index d70009cc2b60..6d3924ab68b7 100644 --- a/core/src/services/oss/core.rs +++ b/core/src/services/oss/core.rs @@ -238,9 +238,7 @@ impl OssCore { path: &str, range: BytesRange, is_presign: bool, - if_match: Option<&str>, - if_none_match: Option<&str>, - override_content_disposition: Option<&str>, + args: &OpRead, ) -> Result> { let p = build_abs_path(&self.root, path); let endpoint = self.get_endpoint(is_presign); @@ -248,7 +246,7 @@ impl OssCore { // Add query arguments to the URL based on response overrides let mut query_args = Vec::new(); - if let Some(override_content_disposition) = override_content_disposition { + if let Some(override_content_disposition) = args.override_content_disposition() { query_args.push(format!( "{}={}", constants::RESPONSE_CONTENT_DISPOSITION, @@ -270,10 +268,10 @@ impl OssCore { req = req.header("x-oss-range-behavior", "standard"); } - if let Some(if_match) = if_match { + if let Some(if_match) = args.if_match() { req = req.header(IF_MATCH, if_match) } - if let Some(if_none_match) = if_none_match { + if let Some(if_none_match) = args.if_none_match() { req = req.header(IF_NONE_MATCH, if_none_match); } @@ -370,18 +368,9 @@ impl OssCore { &self, path: &str, range: BytesRange, - if_match: Option<&str>, - if_none_match: Option<&str>, - override_content_disposition: Option<&str>, + args: &OpRead, ) -> Result> { - let mut req = self.oss_get_object_request( - path, - range, - false, - if_match, - if_none_match, - override_content_disposition, - )?; + let mut req = self.oss_get_object_request(path, range, false, args)?; self.sign(&mut req).await?; self.send(req).await } diff --git a/core/src/services/oss/mod.rs b/core/src/services/oss/mod.rs index 3954a9885c0d..03b2f9612a51 100644 --- a/core/src/services/oss/mod.rs +++ b/core/src/services/oss/mod.rs @@ -21,4 +21,5 @@ pub use backend::OssBuilder as Oss; mod core; mod error; mod lister; +mod reader; mod writer; diff --git a/core/src/services/oss/reader.rs b/core/src/services/oss/reader.rs new file mode 100644 index 000000000000..05197f78bb92 --- /dev/null +++ b/core/src/services/oss/reader.rs @@ -0,0 +1,61 @@ +// 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 super::core::OssCore; +use super::error::parse_error; +use crate::raw::{oio, OpRead}; +use http::StatusCode; +use std::future::Future; +use std::sync::Arc; + +pub struct OssReader { + core: Arc, + + path: String, + op: OpRead, +} + +impl OssReader { + pub fn new(core: Arc, path: &str, op: OpRead) -> Self { + OssReader { + core, + path: path.to_string(), + op: op, + } + } +} + +impl oio::Read for OssReader { + async fn read_at(&self, offset: u64, limit: usize) -> crate::Result { + let Some(range) = self.op.range().apply_on_offset(offset, limit) else { + return Ok(oio::Buffer::new()); + }; + + let resp = self + .core + .oss_get_object(&self.path, range, &self.op) + .await?; + + let status = resp.status(); + + match status { + StatusCode::OK | StatusCode::PARTIAL_CONTENT => Ok(resp.into_body()), + StatusCode::RANGE_NOT_SATISFIABLE => Ok(oio::Buffer::new()), + _ => Err(parse_error(resp).await?), + } + } +} From 50ab7436dc7f972516160d32fc82bd6e9fb1e6b3 Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Tue, 19 Mar 2024 21:02:09 +0800 Subject: [PATCH 034/111] Fix pcloud Signed-off-by: Xuanwo --- core/src/services/pcloud/backend.rs | 25 ++++--------- core/src/services/pcloud/core.rs | 7 ++-- core/src/services/pcloud/error.rs | 1 + core/src/services/pcloud/lister.rs | 7 ++-- core/src/services/pcloud/mod.rs | 1 + core/src/services/pcloud/reader.rs | 58 +++++++++++++++++++++++++++++ core/src/services/pcloud/writer.rs | 2 +- 7 files changed, 77 insertions(+), 24 deletions(-) create mode 100644 core/src/services/pcloud/reader.rs diff --git a/core/src/services/pcloud/backend.rs b/core/src/services/pcloud/backend.rs index 8b60bcf2a712..5d5dc6ccc8e3 100644 --- a/core/src/services/pcloud/backend.rs +++ b/core/src/services/pcloud/backend.rs @@ -21,6 +21,7 @@ use std::fmt::Formatter; use std::sync::Arc; use async_trait::async_trait; +use bytes::Buf; use http::StatusCode; use log::debug; use serde::Deserialize; @@ -32,6 +33,7 @@ use super::lister::PcloudLister; use super::writer::PcloudWriter; use super::writer::PcloudWriters; use crate::raw::*; +use crate::services::pcloud::reader::PcloudReader; use crate::*; /// Config for backblaze Pcloud services support. @@ -228,7 +230,7 @@ pub struct PcloudBackend { #[async_trait] impl Accessor for PcloudBackend { - type Reader = oio::Buffer; + type Reader = PcloudReader; type Writer = PcloudWriters; type Lister = oio::PageLister; type BlockingReader = (); @@ -295,24 +297,13 @@ impl Accessor for PcloudBackend { } } - async fn read(&self, path: &str, _args: OpRead) -> Result<(RpRead, Self::Reader)> { + async fn read(&self, path: &str, args: OpRead) -> Result<(RpRead, Self::Reader)> { let link = self.core.get_file_link(path).await?; - let resp = self.core.download(&link).await?; - - let status = resp.status(); - - match status { - StatusCode::OK => { - let size = parse_content_length(resp.headers())?; - let range = parse_content_range(resp.headers())?; - Ok(( - RpRead::new().with_size(size).with_range(range), - resp.into_body(), - )) - } - _ => Err(parse_error(resp).await?), - } + Ok(( + RpRead::default(), + PcloudReader::new(self.core.clone(), &link, args), + )) } async fn write(&self, path: &str, _args: OpWrite) -> Result<(RpWrite, Self::Writer)> { diff --git a/core/src/services/pcloud/core.rs b/core/src/services/pcloud/core.rs index 59c455312d23..4e84b3584372 100644 --- a/core/src/services/pcloud/core.rs +++ b/core/src/services/pcloud/core.rs @@ -18,10 +18,10 @@ use std::fmt::Debug; use std::fmt::Formatter; -use bytes::Bytes; -use http::Request; +use bytes::{Buf, Bytes}; use http::Response; use http::StatusCode; +use http::{header, Request}; use serde::Deserialize; use super::error::parse_error; @@ -108,11 +108,12 @@ impl PcloudCore { } } - pub async fn download(&self, url: &str) -> Result> { + pub async fn download(&self, url: &str, range: BytesRange) -> Result> { let req = Request::get(url); // set body let req = req + .header(header::RANGE, range.to_header()) .body(AsyncBody::Empty) .map_err(new_request_build_error)?; diff --git a/core/src/services/pcloud/error.rs b/core/src/services/pcloud/error.rs index b4900689e41a..3edc31f85306 100644 --- a/core/src/services/pcloud/error.rs +++ b/core/src/services/pcloud/error.rs @@ -15,6 +15,7 @@ // specific language governing permissions and limitations // under the License. +use bytes::Buf; use std::fmt::Debug; use std::fmt::Formatter; diff --git a/core/src/services/pcloud/lister.rs b/core/src/services/pcloud/lister.rs index aed358ff4ec3..8b3abd54620f 100644 --- a/core/src/services/pcloud/lister.rs +++ b/core/src/services/pcloud/lister.rs @@ -15,6 +15,7 @@ // specific language governing permissions and limitations // under the License. +use bytes::Buf; use std::sync::Arc; use http::StatusCode; @@ -50,8 +51,8 @@ impl oio::PageList for PcloudLister { StatusCode::OK => { let bs = resp.into_body(); - let resp: ListFolderResponse = - serde_json::from_reader(bs.reader()).map_err(new_json_deserialize_error)?; + let resp: ListFolderResponse = serde_json::from_reader(bs.clone().reader()) + .map_err(new_json_deserialize_error)?; let result = resp.result; if result == 2005 { @@ -85,7 +86,7 @@ impl oio::PageList for PcloudLister { return Err(Error::new( ErrorKind::Unexpected, - &String::from_utf8_lossy(&bs), + &String::from_utf8_lossy(&bs.to_bytes()), )); } _ => Err(parse_error(resp).await?), diff --git a/core/src/services/pcloud/mod.rs b/core/src/services/pcloud/mod.rs index 50fb5e5f53c3..5ec313a14f26 100644 --- a/core/src/services/pcloud/mod.rs +++ b/core/src/services/pcloud/mod.rs @@ -22,4 +22,5 @@ pub use backend::PcloudConfig; mod core; mod error; mod lister; +mod reader; mod writer; diff --git a/core/src/services/pcloud/reader.rs b/core/src/services/pcloud/reader.rs new file mode 100644 index 000000000000..f403d35c2260 --- /dev/null +++ b/core/src/services/pcloud/reader.rs @@ -0,0 +1,58 @@ +// 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 super::core::PcloudCore; +use super::error::parse_error; +use crate::raw::{oio, OpRead}; +use http::StatusCode; +use std::future::Future; +use std::sync::Arc; + +pub struct PcloudReader { + core: Arc, + + link: String, + op: OpRead, +} + +impl PcloudReader { + pub fn new(core: Arc, link: &str, op: OpRead) -> Self { + PcloudReader { + core, + link: link.to_string(), + op: op, + } + } +} + +impl oio::Read for PcloudReader { + async fn read_at(&self, offset: u64, limit: usize) -> crate::Result { + let Some(range) = self.op.range().apply_on_offset(offset, limit) else { + return Ok(oio::Buffer::new()); + }; + + let resp = self.core.download(&self.link, range).await?; + + let status = resp.status(); + + match status { + StatusCode::OK | StatusCode::PARTIAL_CONTENT => Ok(resp.into_body()), + StatusCode::RANGE_NOT_SATISFIABLE => Ok(oio::Buffer::new()), + _ => Err(parse_error(resp).await?), + } + } +} diff --git a/core/src/services/pcloud/writer.rs b/core/src/services/pcloud/writer.rs index 6ec18d05abe3..3ba4074bd045 100644 --- a/core/src/services/pcloud/writer.rs +++ b/core/src/services/pcloud/writer.rs @@ -17,7 +17,7 @@ use std::sync::Arc; -use bytes::Bytes; +use bytes::{Buf, Bytes}; use http::StatusCode; use super::core::PcloudCore; From afeecbf1448ccff3cb9f9eecf627928caa55c92a Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Tue, 19 Mar 2024 21:07:33 +0800 Subject: [PATCH 035/111] Fix seafile Signed-off-by: Xuanwo --- core/src/services/seafile/backend.rs | 24 ++++-------- core/src/services/seafile/core.rs | 38 ++++++++++-------- core/src/services/seafile/lister.rs | 5 ++- core/src/services/seafile/mod.rs | 1 + core/src/services/seafile/reader.rs | 58 ++++++++++++++++++++++++++++ 5 files changed, 91 insertions(+), 35 deletions(-) create mode 100644 core/src/services/seafile/reader.rs diff --git a/core/src/services/seafile/backend.rs b/core/src/services/seafile/backend.rs index eb0294c55b68..031c9ef97977 100644 --- a/core/src/services/seafile/backend.rs +++ b/core/src/services/seafile/backend.rs @@ -35,6 +35,7 @@ use super::writer::SeafileWriter; use super::writer::SeafileWriters; use crate::raw::*; use crate::services::seafile::core::SeafileSigner; +use crate::services::seafile::reader::SeafileReader; use crate::*; /// Config for backblaze seafile services support. @@ -256,7 +257,7 @@ pub struct SeafileBackend { #[async_trait] impl Accessor for SeafileBackend { - type Reader = oio::Buffer; + type Reader = SeafileReader; type Writer = SeafileWriters; type Lister = oio::PageLister; type BlockingReader = (); @@ -303,22 +304,11 @@ impl Accessor for SeafileBackend { metadata.map(RpStat::new) } - async fn read(&self, path: &str, _args: OpRead) -> Result<(RpRead, Self::Reader)> { - let resp = self.core.download_file(path).await?; - - let status = resp.status(); - - match status { - StatusCode::OK => { - let size = parse_content_length(resp.headers())?; - let range = parse_content_range(resp.headers())?; - Ok(( - RpRead::new().with_size(size).with_range(range), - resp.into_body(), - )) - } - _ => Err(parse_error(resp).await?), - } + async fn read(&self, path: &str, args: OpRead) -> Result<(RpRead, Self::Reader)> { + Ok(( + RpRead::default(), + SeafileReader::new(self.core.clone(), path, args), + )) } async fn write(&self, path: &str, args: OpWrite) -> Result<(RpWrite, Self::Writer)> { diff --git a/core/src/services/seafile/core.rs b/core/src/services/seafile/core.rs index 3c3f3737f86c..3b9396b2807e 100644 --- a/core/src/services/seafile/core.rs +++ b/core/src/services/seafile/core.rs @@ -19,7 +19,7 @@ use std::fmt::Debug; use std::fmt::Formatter; use std::sync::Arc; -use bytes::Bytes; +use bytes::{Buf, Bytes}; use http::header; use http::Request; use http::Response; @@ -96,9 +96,10 @@ impl SeafileCore { match status { StatusCode::OK => { - let resp_body = &resp.into_body(); - let auth_response = serde_json::from_slice::(resp_body) - .map_err(new_json_deserialize_error)?; + let resp_body = resp.into_body(); + let auth_response: AuthTokenResponse = + serde_json::from_reader(resp_body.reader()) + .map_err(new_json_deserialize_error)?; signer.auth_info = AuthInfo { token: auth_response.token, repo_id: "".to_string(), @@ -125,9 +126,9 @@ impl SeafileCore { match status { StatusCode::OK => { - let resp_body = &resp.into_body(); - let list_library_response = - serde_json::from_slice::>(resp_body) + let resp_body = resp.into_body(); + let list_library_response: Vec = + serde_json::from_reader(resp_body.reader()) .map_err(new_json_deserialize_error)?; for library in list_library_response { @@ -174,8 +175,8 @@ impl SeafileCore { match status { StatusCode::OK => { - let resp_body = &resp.into_body(); - let upload_url = serde_json::from_slice::(resp_body) + let resp_body = resp.into_body(); + let upload_url = serde_json::from_reader(resp_body.reader()) .map_err(new_json_deserialize_error)?; Ok(upload_url) } @@ -205,8 +206,8 @@ impl SeafileCore { match status { StatusCode::OK => { - let resp_body = &resp.into_body(); - let download_url = serde_json::from_slice::(resp_body) + let resp_body = resp.into_body(); + let download_url = serde_json::from_reader(resp_body.reader()) .map_err(new_json_deserialize_error)?; Ok(download_url) @@ -216,12 +217,17 @@ impl SeafileCore { } /// download file - pub async fn download_file(&self, path: &str) -> Result> { + pub async fn download_file( + &self, + path: &str, + range: BytesRange, + ) -> Result> { let download_url = self.get_download_url(path).await?; let req = Request::get(download_url); let req = req + .header(header::RANGE, range.to_header()) .body(AsyncBody::Empty) .map_err(new_request_build_error)?; @@ -256,8 +262,8 @@ impl SeafileCore { match status { StatusCode::OK => { - let resp_body = &resp.into_body(); - let file_detail = serde_json::from_slice::(resp_body) + let resp_body = resp.into_body(); + let file_detail: FileDetail = serde_json::from_reader(resp_body.reader()) .map_err(new_json_deserialize_error)?; Ok(file_detail) } @@ -287,8 +293,8 @@ impl SeafileCore { match status { StatusCode::OK => { - let resp_body = &resp.into_body(); - let dir_detail = serde_json::from_slice::(resp_body) + let resp_body = resp.into_body(); + let dir_detail: DirDetail = serde_json::from_reader(resp_body.reader()) .map_err(new_json_deserialize_error)?; Ok(dir_detail) } diff --git a/core/src/services/seafile/lister.rs b/core/src/services/seafile/lister.rs index 990973af1f46..25a0d2e837e9 100644 --- a/core/src/services/seafile/lister.rs +++ b/core/src/services/seafile/lister.rs @@ -15,6 +15,7 @@ // specific language governing permissions and limitations // under the License. +use bytes::Buf; use std::sync::Arc; use http::header; @@ -69,8 +70,8 @@ impl oio::PageList for SeafileLister { match status { StatusCode::OK => { - let resp_body = &resp.into_body(); - let infos = serde_json::from_slice::>(resp_body) + let resp_body = resp.into_body(); + let infos: Vec = serde_json::from_reader(resp_body.reader()) .map_err(new_json_deserialize_error)?; for info in infos { diff --git a/core/src/services/seafile/mod.rs b/core/src/services/seafile/mod.rs index dc9e1ccce078..63a112ee7b5c 100644 --- a/core/src/services/seafile/mod.rs +++ b/core/src/services/seafile/mod.rs @@ -22,4 +22,5 @@ pub use backend::SeafileConfig; mod core; mod error; mod lister; +mod reader; mod writer; diff --git a/core/src/services/seafile/reader.rs b/core/src/services/seafile/reader.rs new file mode 100644 index 000000000000..487795bc44a0 --- /dev/null +++ b/core/src/services/seafile/reader.rs @@ -0,0 +1,58 @@ +// 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 super::core::SeafileCore; +use super::error::parse_error; +use crate::raw::{oio, OpRead}; +use http::StatusCode; +use std::future::Future; +use std::sync::Arc; + +pub struct SeafileReader { + core: Arc, + + path: String, + op: OpRead, +} + +impl SeafileReader { + pub fn new(core: Arc, path: &str, op: OpRead) -> Self { + SeafileReader { + core, + path: path.to_string(), + op: op, + } + } +} + +impl oio::Read for SeafileReader { + async fn read_at(&self, offset: u64, limit: usize) -> crate::Result { + let Some(range) = self.op.range().apply_on_offset(offset, limit) else { + return Ok(oio::Buffer::new()); + }; + + let resp = self.core.download_file(&self.path, range).await?; + + let status = resp.status(); + + match status { + StatusCode::OK | StatusCode::PARTIAL_CONTENT => Ok(resp.into_body()), + StatusCode::RANGE_NOT_SATISFIABLE => Ok(oio::Buffer::new()), + _ => Err(parse_error(resp).await?), + } + } +} From 3f8895c65a3564461dfc50f5e4d10342ee2d47dc Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Tue, 19 Mar 2024 21:14:13 +0800 Subject: [PATCH 036/111] Fix services Signed-off-by: Xuanwo --- core/src/services/sftp/backend.rs | 15 ++----- core/src/services/supabase/backend.rs | 15 +++---- core/src/services/supabase/error.rs | 1 + core/src/services/supabase/mod.rs | 1 + core/src/services/supabase/reader.rs | 58 +++++++++++++++++++++++++++ core/src/services/swift/backend.rs | 23 +++-------- core/src/services/swift/core.rs | 11 +++-- core/src/services/swift/lister.rs | 3 +- core/src/services/swift/mod.rs | 1 + core/src/services/swift/reader.rs | 58 +++++++++++++++++++++++++++ core/src/services/upyun/backend.rs | 24 ++++------- core/src/services/upyun/core.rs | 7 +++- core/src/services/upyun/lister.rs | 5 ++- core/src/services/upyun/mod.rs | 1 + core/src/services/upyun/reader.rs | 58 +++++++++++++++++++++++++++ 15 files changed, 219 insertions(+), 62 deletions(-) create mode 100644 core/src/services/supabase/reader.rs create mode 100644 core/src/services/swift/reader.rs create mode 100644 core/src/services/upyun/reader.rs diff --git a/core/src/services/sftp/backend.rs b/core/src/services/sftp/backend.rs index beb843110be6..7e4c43c17f7a 100644 --- a/core/src/services/sftp/backend.rs +++ b/core/src/services/sftp/backend.rs @@ -23,6 +23,7 @@ use std::path::PathBuf; use std::pin::Pin; use async_trait::async_trait; +use bytes::Bytes; use futures::StreamExt; use log::debug; use openssh::KnownHosts; @@ -71,7 +72,7 @@ impl Debug for SftpConfig { /// SFTP services support. (only works on unix) /// -/// If you are interested in working on windows, please refer to [this](https://github.com/apache/opendal/issues/2963) issue. +/// If you are interested in working on windows, pl ease refer to [this](https://github.com/apache/opendal/issues/2963) issue. /// Welcome to leave your comments or make contributions. /// /// Warning: Maximum number of file holdings is depending on the remote system configuration. @@ -242,7 +243,7 @@ impl Debug for SftpBackend { #[async_trait] impl Accessor for SftpBackend { - type Reader = oio::TokioReader>>; + type Reader = Bytes; type Writer = SftpWriter; type Lister = Option; type BlockingReader = (); @@ -322,15 +323,7 @@ impl Accessor for SftpBackend { .await .map_err(parse_sftp_error)?; - // Sorry for the ugly code... - // - // - `f` is a openssh file. - // - `TokioCompatFile::new(f)` makes it implements tokio AsyncRead + AsyncSeek for openssh File. - // - `Box::pin(x)` to make sure this reader implements `Unpin`, since `TokioCompatFile` is not. - // - `oio::TokioReader::new(x)` makes it a `oio::TokioReader` which implements `oio::Read`. - let r = oio::TokioReader::new(Box::pin(TokioCompatFile::new(f))); - - Ok((RpRead::new(), r)) + todo!() } async fn write(&self, path: &str, op: OpWrite) -> Result<(RpWrite, Self::Writer)> { diff --git a/core/src/services/supabase/backend.rs b/core/src/services/supabase/backend.rs index 08b9245b69d5..9619e9824eb4 100644 --- a/core/src/services/supabase/backend.rs +++ b/core/src/services/supabase/backend.rs @@ -26,6 +26,7 @@ use super::core::*; use super::error::parse_error; use super::writer::*; use crate::raw::*; +use crate::services::supabase::reader::SupabaseReader; use crate::*; /// [Supabase](https://supabase.com/) service support @@ -156,7 +157,7 @@ pub struct SupabaseBackend { #[async_trait] impl Accessor for SupabaseBackend { - type Reader = oio::Buffer; + type Reader = SupabaseReader; type Writer = oio::OneShotWriter; // todo: implement Lister to support list and scan type Lister = (); @@ -204,14 +205,10 @@ impl Accessor for SupabaseBackend { } async fn read(&self, path: &str, args: OpRead) -> Result<(RpRead, Self::Reader)> { - let resp = self.core.supabase_get_object(path, args.range()).await?; - - let status = resp.status(); - - match status { - StatusCode::OK | StatusCode::PARTIAL_CONTENT => Ok((RpRead::new(), resp.into_body())), - _ => Err(parse_error(resp).await?), - } + Ok(( + RpRead::default(), + SupabaseReader::new(self.core.clone(), path, args), + )) } async fn write(&self, path: &str, args: OpWrite) -> Result<(RpWrite, Self::Writer)> { diff --git a/core/src/services/supabase/error.rs b/core/src/services/supabase/error.rs index bfc5038e4587..4e4602be1ebb 100644 --- a/core/src/services/supabase/error.rs +++ b/core/src/services/supabase/error.rs @@ -15,6 +15,7 @@ // specific language governing permissions and limitations // under the License. +use bytes::Buf; use http::Response; use http::StatusCode; use serde::Deserialize; diff --git a/core/src/services/supabase/mod.rs b/core/src/services/supabase/mod.rs index 89aeada88983..5abaac6b8575 100644 --- a/core/src/services/supabase/mod.rs +++ b/core/src/services/supabase/mod.rs @@ -19,4 +19,5 @@ mod backend; pub use backend::SupabaseBuilder as Supabase; mod core; mod error; +mod reader; mod writer; diff --git a/core/src/services/supabase/reader.rs b/core/src/services/supabase/reader.rs new file mode 100644 index 000000000000..5a63b8c2c62f --- /dev/null +++ b/core/src/services/supabase/reader.rs @@ -0,0 +1,58 @@ +// 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 super::core::SupabaseCore; +use super::error::parse_error; +use crate::raw::{oio, OpRead}; +use http::StatusCode; +use std::future::Future; +use std::sync::Arc; + +pub struct SupabaseReader { + core: Arc, + + path: String, + op: OpRead, +} + +impl SupabaseReader { + pub fn new(core: Arc, path: &str, op: OpRead) -> Self { + SupabaseReader { + core, + path: path.to_string(), + op: op, + } + } +} + +impl oio::Read for SupabaseReader { + async fn read_at(&self, offset: u64, limit: usize) -> crate::Result { + let Some(range) = self.op.range().apply_on_offset(offset, limit) else { + return Ok(oio::Buffer::new()); + }; + + let resp = self.core.supabase_get_object(&self.path, range).await?; + + let status = resp.status(); + + match status { + StatusCode::OK | StatusCode::PARTIAL_CONTENT => Ok(resp.into_body()), + StatusCode::RANGE_NOT_SATISFIABLE => Ok(oio::Buffer::new()), + _ => Err(parse_error(resp).await?), + } + } +} diff --git a/core/src/services/swift/backend.rs b/core/src/services/swift/backend.rs index e8ff32606329..0e1e5f27d20d 100644 --- a/core/src/services/swift/backend.rs +++ b/core/src/services/swift/backend.rs @@ -29,6 +29,7 @@ use super::error::parse_error; use super::lister::SwiftLister; use super::writer::SwiftWriter; use crate::raw::*; +use crate::services::swift::reader::SwiftReader; use crate::*; /// [OpenStack Swift](https://docs.openstack.org/api-ref/object-store/#)'s REST API support. @@ -190,7 +191,7 @@ pub struct SwiftBackend { #[async_trait] impl Accessor for SwiftBackend { - type Reader = oio::Buffer; + type Reader = SwiftReader; type Writer = oio::OneShotWriter; type Lister = oio::PageLister; type BlockingReader = (); @@ -235,22 +236,10 @@ impl Accessor for SwiftBackend { } async fn read(&self, path: &str, args: OpRead) -> Result<(RpRead, Self::Reader)> { - let resp = self.core.swift_read(path, args).await?; - - match resp.status() { - StatusCode::OK | StatusCode::PARTIAL_CONTENT => { - let size = parse_content_length(resp.headers())?; - let range = parse_content_range(resp.headers())?; - Ok(( - RpRead::new().with_size(size).with_range(range), - resp.into_body(), - )) - } - StatusCode::RANGE_NOT_SATISFIABLE => { - Ok((RpRead::new().with_size(Some(0)), oio::Buffer::empty())) - } - _ => Err(parse_error(resp).await?), - } + Ok(( + RpRead::default(), + SwiftReader::new(self.core.clone(), path, args), + )) } async fn write(&self, path: &str, args: OpWrite) -> Result<(RpWrite, Self::Writer)> { diff --git a/core/src/services/swift/core.rs b/core/src/services/swift/core.rs index bfc8b35ecc95..0ebcec1317eb 100644 --- a/core/src/services/swift/core.rs +++ b/core/src/services/swift/core.rs @@ -127,9 +127,12 @@ impl SwiftCore { self.client.send(req).await } - pub async fn swift_read(&self, path: &str, arg: OpRead) -> Result> { - let range = arg.range(); - + pub async fn swift_read( + &self, + path: &str, + range: BytesRange, + arg: &OpRead, + ) -> Result> { let p = build_abs_path(&self.root, path) .trim_end_matches('/') .to_string(); @@ -146,7 +149,7 @@ impl SwiftCore { req = req.header("X-Auth-Token", &self.token); if !range.is_full() { - req = req.header("Range", &range.to_header()); + req = req.header(header::RANGE, range.to_header()); } let req = req diff --git a/core/src/services/swift/lister.rs b/core/src/services/swift/lister.rs index 3dda024144ca..3c68c12817b2 100644 --- a/core/src/services/swift/lister.rs +++ b/core/src/services/swift/lister.rs @@ -15,6 +15,7 @@ // specific language governing permissions and limitations // under the License. +use bytes::Buf; use std::sync::Arc; use super::core::*; @@ -55,7 +56,7 @@ impl oio::PageList for SwiftLister { return Err(error); } - let bytes = response.into_body().bytes().await?; + let bytes = response.into_body(); let decoded_response: Vec = serde_json::from_reader(bytes.reader()).map_err(new_json_deserialize_error)?; diff --git a/core/src/services/swift/mod.rs b/core/src/services/swift/mod.rs index 9cfeb92e07a0..c4d08c8c44cb 100644 --- a/core/src/services/swift/mod.rs +++ b/core/src/services/swift/mod.rs @@ -21,4 +21,5 @@ pub use backend::SwiftBuilder as Swift; mod core; mod error; mod lister; +mod reader; mod writer; diff --git a/core/src/services/swift/reader.rs b/core/src/services/swift/reader.rs new file mode 100644 index 000000000000..00142c3c7275 --- /dev/null +++ b/core/src/services/swift/reader.rs @@ -0,0 +1,58 @@ +// 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 super::core::SwiftCore; +use super::error::parse_error; +use crate::raw::{oio, OpRead}; +use http::StatusCode; +use std::future::Future; +use std::sync::Arc; + +pub struct SwiftReader { + core: Arc, + + path: String, + op: OpRead, +} + +impl SwiftReader { + pub fn new(core: Arc, path: &str, op: OpRead) -> Self { + SwiftReader { + core, + path: path.to_string(), + op: op, + } + } +} + +impl oio::Read for SwiftReader { + async fn read_at(&self, offset: u64, limit: usize) -> crate::Result { + let Some(range) = self.op.range().apply_on_offset(offset, limit) else { + return Ok(oio::Buffer::new()); + }; + + let resp = self.core.swift_read(&self.path, range, &self.op).await?; + + let status = resp.status(); + + match status { + StatusCode::OK | StatusCode::PARTIAL_CONTENT => Ok(resp.into_body()), + StatusCode::RANGE_NOT_SATISFIABLE => Ok(oio::Buffer::new()), + _ => Err(parse_error(resp).await?), + } + } +} diff --git a/core/src/services/upyun/backend.rs b/core/src/services/upyun/backend.rs index 2b5fa84c1104..c5f974f721f2 100644 --- a/core/src/services/upyun/backend.rs +++ b/core/src/services/upyun/backend.rs @@ -33,6 +33,7 @@ use super::writer::UpyunWriter; use super::writer::UpyunWriters; use crate::raw::*; use crate::services::upyun::core::UpyunSigner; +use crate::services::upyun::reader::UpyunReader; use crate::*; /// Config for backblaze upyun services support. @@ -233,7 +234,7 @@ pub struct UpyunBackend { #[async_trait] impl Accessor for UpyunBackend { - type Reader = oio::Buffer; + type Reader = UpyunReader; type Writer = UpyunWriters; type Lister = oio::PageLister; type BlockingReader = (); @@ -297,22 +298,11 @@ impl Accessor for UpyunBackend { } } - async fn read(&self, path: &str, _args: OpRead) -> Result<(RpRead, Self::Reader)> { - let resp = self.core.download_file(path).await?; - - let status = resp.status(); - - match status { - StatusCode::OK => { - let size = parse_content_length(resp.headers())?; - let range = parse_content_range(resp.headers())?; - Ok(( - RpRead::new().with_size(size).with_range(range), - resp.into_body(), - )) - } - _ => Err(parse_error(resp).await?), - } + async fn read(&self, path: &str, args: OpRead) -> Result<(RpRead, Self::Reader)> { + Ok(( + RpRead::default(), + UpyunReader::new(self.core.clone(), path, args), + )) } async fn write(&self, path: &str, args: OpWrite) -> Result<(RpWrite, Self::Writer)> { diff --git a/core/src/services/upyun/core.rs b/core/src/services/upyun/core.rs index ea0b648d9817..1fcaf429db96 100644 --- a/core/src/services/upyun/core.rs +++ b/core/src/services/upyun/core.rs @@ -104,7 +104,11 @@ impl UpyunCore { } impl UpyunCore { - pub async fn download_file(&self, path: &str) -> Result> { + pub async fn download_file( + &self, + path: &str, + range: BytesRange, + ) -> Result> { let path = build_abs_path(&self.root, path); let url = format!( @@ -116,6 +120,7 @@ impl UpyunCore { let req = Request::get(url); let mut req = req + .header(header::RANGE, range.to_header()) .body(AsyncBody::Empty) .map_err(new_request_build_error)?; diff --git a/core/src/services/upyun/lister.rs b/core/src/services/upyun/lister.rs index e7a6e4ea16ad..b1dd13a4a1b2 100644 --- a/core/src/services/upyun/lister.rs +++ b/core/src/services/upyun/lister.rs @@ -15,6 +15,7 @@ // specific language governing permissions and limitations // under the License. +use bytes::Buf; use std::sync::Arc; use super::core::ListObjectsResponse; @@ -68,8 +69,8 @@ impl oio::PageList for UpyunLister { let bs = resp.into_body(); - let response = serde_json::from_slice::(&bs) - .map_err(new_json_deserialize_error)?; + let response: ListObjectsResponse = + serde_json::from_reader(bs.reader()).map_err(new_json_deserialize_error)?; // ref https://help.upyun.com/knowledge-base/rest_api/#e88eb7e58f96e79baee5bd95e69687e4bbb6e58897e8a1a8 // when iter is "g2gCZAAEbmV4dGQAA2VvZg", it means the list is done. diff --git a/core/src/services/upyun/mod.rs b/core/src/services/upyun/mod.rs index 039f2aa22944..a390359a7140 100644 --- a/core/src/services/upyun/mod.rs +++ b/core/src/services/upyun/mod.rs @@ -22,4 +22,5 @@ pub use backend::UpyunConfig; mod core; mod error; mod lister; +mod reader; mod writer; diff --git a/core/src/services/upyun/reader.rs b/core/src/services/upyun/reader.rs new file mode 100644 index 000000000000..1f47e313c540 --- /dev/null +++ b/core/src/services/upyun/reader.rs @@ -0,0 +1,58 @@ +// 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 super::core::UpyunCore; +use super::error::parse_error; +use crate::raw::{oio, OpRead}; +use http::StatusCode; +use std::future::Future; +use std::sync::Arc; + +pub struct UpyunReader { + core: Arc, + + path: String, + op: OpRead, +} + +impl UpyunReader { + pub fn new(core: Arc, path: &str, op: OpRead) -> Self { + UpyunReader { + core, + path: path.to_string(), + op: op, + } + } +} + +impl oio::Read for UpyunReader { + async fn read_at(&self, offset: u64, limit: usize) -> crate::Result { + let Some(range) = self.op.range().apply_on_offset(offset, limit) else { + return Ok(oio::Buffer::new()); + }; + + let resp = self.core.download_file(&self.path, range).await?; + + let status = resp.status(); + + match status { + StatusCode::OK | StatusCode::PARTIAL_CONTENT => Ok(resp.into_body()), + StatusCode::RANGE_NOT_SATISFIABLE => Ok(oio::Buffer::new()), + _ => Err(parse_error(resp).await?), + } + } +} From 2c6b760560d65d9b3b882a1a9811aeb4918c46e8 Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Tue, 19 Mar 2024 21:16:15 +0800 Subject: [PATCH 037/111] Fix vercel artifacts Signed-off-by: Xuanwo --- core/src/services/vercel_artifacts/backend.rs | 25 ++++---- core/src/services/vercel_artifacts/error.rs | 1 + core/src/services/vercel_artifacts/mod.rs | 1 + core/src/services/vercel_artifacts/reader.rs | 61 +++++++++++++++++++ 4 files changed, 74 insertions(+), 14 deletions(-) create mode 100644 core/src/services/vercel_artifacts/reader.rs diff --git a/core/src/services/vercel_artifacts/backend.rs b/core/src/services/vercel_artifacts/backend.rs index 177f234901b3..7bc9c22beddb 100644 --- a/core/src/services/vercel_artifacts/backend.rs +++ b/core/src/services/vercel_artifacts/backend.rs @@ -26,6 +26,7 @@ use http::StatusCode; use super::error::parse_error; use super::writer::VercelArtifactsWriter; use crate::raw::*; +use crate::services::vercel_artifacts::reader::VercelArtifactsReader; use crate::*; #[doc = include_str!("docs.md")] @@ -45,7 +46,7 @@ impl Debug for VercelArtifactsBackend { #[async_trait] impl Accessor for VercelArtifactsBackend { - type Reader = oio::Buffer; + type Reader = VercelArtifactsReader; type Writer = oio::OneShotWriter; type Lister = (); type BlockingReader = (); @@ -85,15 +86,10 @@ impl Accessor for VercelArtifactsBackend { } async fn read(&self, path: &str, args: OpRead) -> Result<(RpRead, Self::Reader)> { - let resp = self.vercel_artifacts_get(path, args).await?; - - let status = resp.status(); - - match status { - StatusCode::OK | StatusCode::PARTIAL_CONTENT => Ok((RpRead::new(), resp.into_body())), - - _ => Err(parse_error(resp).await?), - } + Ok(( + RpRead::default(), + VercelArtifactsReader::new(self.clone(), path, args), + )) } async fn write(&self, path: &str, args: OpWrite) -> Result<(RpWrite, Self::Writer)> { @@ -109,10 +105,11 @@ impl Accessor for VercelArtifactsBackend { } impl VercelArtifactsBackend { - async fn vercel_artifacts_get( + pub async fn vercel_artifacts_get( &self, hash: &str, - args: OpRead, + range: BytesRange, + _: &OpRead, ) -> Result> { let url: String = format!( "https://api.vercel.com/v8/artifacts/{}", @@ -121,8 +118,8 @@ impl VercelArtifactsBackend { let mut req = Request::get(&url); - if !args.range().is_full() { - req = req.header(header::RANGE, args.range().to_header()); + if !range.is_full() { + req = req.header(header::RANGE, range.to_header()); } let auth_header_content = format!("Bearer {}", self.access_token); diff --git a/core/src/services/vercel_artifacts/error.rs b/core/src/services/vercel_artifacts/error.rs index 94d592fd5652..30a90f7a4f92 100644 --- a/core/src/services/vercel_artifacts/error.rs +++ b/core/src/services/vercel_artifacts/error.rs @@ -15,6 +15,7 @@ // specific language governing permissions and limitations // under the License. +use bytes::Buf; use http::Response; use http::StatusCode; diff --git a/core/src/services/vercel_artifacts/mod.rs b/core/src/services/vercel_artifacts/mod.rs index 3bfddabca7af..656bbfdc5df3 100644 --- a/core/src/services/vercel_artifacts/mod.rs +++ b/core/src/services/vercel_artifacts/mod.rs @@ -18,6 +18,7 @@ mod backend; mod builder; mod error; +mod reader; mod writer; pub use builder::VercelArtifactsBuilder as VercelArtifacts; diff --git a/core/src/services/vercel_artifacts/reader.rs b/core/src/services/vercel_artifacts/reader.rs new file mode 100644 index 000000000000..29edfabeb825 --- /dev/null +++ b/core/src/services/vercel_artifacts/reader.rs @@ -0,0 +1,61 @@ +// 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 super::error::parse_error; +use crate::raw::{oio, OpRead}; +use crate::services::vercel_artifacts::backend::VercelArtifactsBackend; +use http::StatusCode; +use std::future::Future; +use std::sync::Arc; + +pub struct VercelArtifactsReader { + core: VercelArtifactsBackend, + + path: String, + op: OpRead, +} + +impl VercelArtifactsReader { + pub fn new(core: VercelArtifactsBackend, path: &str, op: OpRead) -> Self { + VercelArtifactsReader { + core, + path: path.to_string(), + op: op, + } + } +} + +impl oio::Read for VercelArtifactsReader { + async fn read_at(&self, offset: u64, limit: usize) -> crate::Result { + let Some(range) = self.op.range().apply_on_offset(offset, limit) else { + return Ok(oio::Buffer::new()); + }; + + let resp = self + .core + .vercel_artifacts_get(&self.path, range, &self.op) + .await?; + + let status = resp.status(); + + match status { + StatusCode::OK | StatusCode::PARTIAL_CONTENT => Ok(resp.into_body()), + StatusCode::RANGE_NOT_SATISFIABLE => Ok(oio::Buffer::new()), + _ => Err(parse_error(resp).await?), + } + } +} From 4eeff7d3fe6f1b80113eb28820107d577c673386 Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Tue, 19 Mar 2024 21:18:35 +0800 Subject: [PATCH 038/111] Fix vercel blob Signed-off-by: Xuanwo --- core/src/services/vercel_blob/backend.rs | 23 +++------- core/src/services/vercel_blob/core.rs | 12 +++-- core/src/services/vercel_blob/mod.rs | 1 + core/src/services/vercel_blob/reader.rs | 58 ++++++++++++++++++++++++ core/src/services/vercel_blob/writer.rs | 9 ++-- 5 files changed, 79 insertions(+), 24 deletions(-) create mode 100644 core/src/services/vercel_blob/reader.rs diff --git a/core/src/services/vercel_blob/backend.rs b/core/src/services/vercel_blob/backend.rs index d0c034142f3e..7e0c56dd9e49 100644 --- a/core/src/services/vercel_blob/backend.rs +++ b/core/src/services/vercel_blob/backend.rs @@ -21,6 +21,7 @@ use std::fmt::Formatter; use std::sync::Arc; use async_trait::async_trait; +use bytes::Buf; use http::StatusCode; use log::debug; use serde::Deserialize; @@ -33,6 +34,7 @@ use super::lister::VercelBlobLister; use super::writer::VercelBlobWriter; use super::writer::VercelBlobWriters; use crate::raw::*; +use crate::services::vercel_blob::reader::VercelBlobReader; use crate::*; /// Config for backblaze VercelBlob services support. @@ -178,7 +180,7 @@ pub struct VercelBlobBackend { #[async_trait] impl Accessor for VercelBlobBackend { - type Reader = oio::Buffer; + type Reader = VercelBlobReader; type Writer = VercelBlobWriters; type Lister = oio::PageLister; type BlockingReader = (); @@ -232,21 +234,10 @@ impl Accessor for VercelBlobBackend { } async fn read(&self, path: &str, args: OpRead) -> Result<(RpRead, Self::Reader)> { - let resp = self.core.download(path, args).await?; - - let status = resp.status(); - - match status { - StatusCode::OK | StatusCode::PARTIAL_CONTENT => { - let size = parse_content_length(resp.headers())?; - let range = parse_content_range(resp.headers())?; - Ok(( - RpRead::new().with_size(size).with_range(range), - resp.into_body(), - )) - } - _ => Err(parse_error(resp).await?), - } + Ok(( + RpRead::default(), + VercelBlobReader::new(self.core.clone(), path, args), + )) } async fn write(&self, path: &str, args: OpWrite) -> Result<(RpWrite, Self::Writer)> { diff --git a/core/src/services/vercel_blob/core.rs b/core/src/services/vercel_blob/core.rs index 2ee1ad5c42d0..45d97c5e9fbd 100644 --- a/core/src/services/vercel_blob/core.rs +++ b/core/src/services/vercel_blob/core.rs @@ -18,7 +18,7 @@ use std::fmt::Debug; use std::fmt::Formatter; -use bytes::Bytes; +use bytes::{Buf, Bytes}; use http::header; use http::request; use http::Request; @@ -83,7 +83,12 @@ impl VercelBlobCore { } impl VercelBlobCore { - pub async fn download(&self, path: &str, args: OpRead) -> Result> { + pub async fn download( + &self, + path: &str, + range: BytesRange, + _: &OpRead, + ) -> Result> { let p = build_abs_path(&self.root, path); // Vercel blob use an unguessable random id url to download the file // So we use list to get the url of the file and then use it to download the file @@ -98,9 +103,8 @@ impl VercelBlobCore { let mut req = Request::get(url); - let range = args.range(); if !range.is_full() { - req = req.header(http::header::RANGE, range.to_header()); + req = req.header(header::RANGE, range.to_header()); } // Set body diff --git a/core/src/services/vercel_blob/mod.rs b/core/src/services/vercel_blob/mod.rs index 2804d3fe7357..050f922abba5 100644 --- a/core/src/services/vercel_blob/mod.rs +++ b/core/src/services/vercel_blob/mod.rs @@ -22,4 +22,5 @@ pub use backend::VercelBlobConfig; mod core; mod error; mod lister; +mod reader; mod writer; diff --git a/core/src/services/vercel_blob/reader.rs b/core/src/services/vercel_blob/reader.rs new file mode 100644 index 000000000000..467272f6223c --- /dev/null +++ b/core/src/services/vercel_blob/reader.rs @@ -0,0 +1,58 @@ +// 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 super::core::VercelBlobCore; +use super::error::parse_error; +use crate::raw::{oio, OpRead}; +use http::StatusCode; +use std::future::Future; +use std::sync::Arc; + +pub struct VercelBlobReader { + core: Arc, + + path: String, + op: OpRead, +} + +impl VercelBlobReader { + pub fn new(core: Arc, path: &str, op: OpRead) -> Self { + VercelBlobReader { + core, + path: path.to_string(), + op: op, + } + } +} + +impl oio::Read for VercelBlobReader { + async fn read_at(&self, offset: u64, limit: usize) -> crate::Result { + let Some(range) = self.op.range().apply_on_offset(offset, limit) else { + return Ok(oio::Buffer::new()); + }; + + let resp = self.core.download(&self.path, range, &self.op).await?; + + let status = resp.status(); + + match status { + StatusCode::OK | StatusCode::PARTIAL_CONTENT => Ok(resp.into_body()), + StatusCode::RANGE_NOT_SATISFIABLE => Ok(oio::Buffer::new()), + _ => Err(parse_error(resp).await?), + } + } +} diff --git a/core/src/services/vercel_blob/writer.rs b/core/src/services/vercel_blob/writer.rs index 905d50a424bb..dbe9f5e57d7b 100644 --- a/core/src/services/vercel_blob/writer.rs +++ b/core/src/services/vercel_blob/writer.rs @@ -15,6 +15,7 @@ // specific language governing permissions and limitations // under the License. +use bytes::Buf; use std::sync::Arc; use http::StatusCode; @@ -70,8 +71,8 @@ impl oio::MultipartWrite for VercelBlobWriter { StatusCode::OK => { let bs = resp.into_body(); - let resp = serde_json::from_slice::(&bs) - .map_err(new_json_deserialize_error)?; + let resp: InitiateMultipartUploadResponse = + serde_json::from_reader(bs.reader()).map_err(new_json_deserialize_error)?; Ok(resp.upload_id) } @@ -99,8 +100,8 @@ impl oio::MultipartWrite for VercelBlobWriter { StatusCode::OK => { let bs = resp.into_body(); - let resp = serde_json::from_slice::(&bs) - .map_err(new_json_deserialize_error)?; + let resp: UploadPartResponse = + serde_json::from_reader(bs.reader()).map_err(new_json_deserialize_error)?; Ok(oio::MultipartPart { part_number, From a7a3522dc97d22dfec70f08b98e7d28a9da7d1a8 Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Tue, 19 Mar 2024 21:20:56 +0800 Subject: [PATCH 039/111] FIx webdav Signed-off-by: Xuanwo --- core/src/services/webdav/backend.rs | 23 +++--------- core/src/services/webdav/core.rs | 10 +++-- core/src/services/webdav/error.rs | 1 + core/src/services/webdav/lister.rs | 4 +- core/src/services/webdav/mod.rs | 1 + core/src/services/webdav/reader.rs | 58 +++++++++++++++++++++++++++++ 6 files changed, 75 insertions(+), 22 deletions(-) create mode 100644 core/src/services/webdav/reader.rs diff --git a/core/src/services/webdav/backend.rs b/core/src/services/webdav/backend.rs index 447c6c958317..329660e2bcab 100644 --- a/core/src/services/webdav/backend.rs +++ b/core/src/services/webdav/backend.rs @@ -31,6 +31,7 @@ use super::error::parse_error; use super::lister::WebdavLister; use super::writer::WebdavWriter; use crate::raw::*; +use crate::services::webdav::reader::WebdavReader; use crate::*; /// Config for [WebDAV](https://datatracker.ietf.org/doc/html/rfc4918) backend support. @@ -237,7 +238,7 @@ impl Debug for WebdavBackend { #[async_trait] impl Accessor for WebdavBackend { - type Reader = oio::Buffer; + type Reader = WebdavReader; type Writer = oio::OneShotWriter; type Lister = oio::PageLister; type BlockingReader = (); @@ -285,22 +286,10 @@ impl Accessor for WebdavBackend { } async fn read(&self, path: &str, args: OpRead) -> Result<(RpRead, Self::Reader)> { - let resp = self.core.webdav_get(path, args).await?; - let status = resp.status(); - match status { - StatusCode::OK | StatusCode::PARTIAL_CONTENT => { - let size = parse_content_length(resp.headers())?; - let range = parse_content_range(resp.headers())?; - Ok(( - RpRead::new().with_size(size).with_range(range), - resp.into_body(), - )) - } - StatusCode::RANGE_NOT_SATISFIABLE => { - Ok((RpRead::new().with_size(Some(0)), oio::Buffer::empty())) - } - _ => Err(parse_error(resp).await?), - } + Ok(( + RpRead::default(), + WebdavReader::new(self.core.clone(), path, args), + )) } async fn write(&self, path: &str, args: OpWrite) -> Result<(RpWrite, Self::Writer)> { diff --git a/core/src/services/webdav/core.rs b/core/src/services/webdav/core.rs index 5557525a6258..301f57202564 100644 --- a/core/src/services/webdav/core.rs +++ b/core/src/services/webdav/core.rs @@ -118,7 +118,7 @@ impl WebdavCore { let bs = resp.into_body(); - let result: Multistatus = deserialize_multistatus(&bs)?; + let result: Multistatus = deserialize_multistatus(&bs.to_bytes())?; let propfind_resp = result.response.first().ok_or_else(|| { Error::new( ErrorKind::NotFound, @@ -130,7 +130,12 @@ impl WebdavCore { Ok(metadata) } - pub async fn webdav_get(&self, path: &str, args: OpRead) -> Result> { + pub async fn webdav_get( + &self, + path: &str, + range: BytesRange, + _: &OpRead, + ) -> Result> { let path = build_rooted_abs_path(&self.root, path); let url: String = format!("{}{}", self.endpoint, percent_encode_path(&path)); @@ -140,7 +145,6 @@ impl WebdavCore { req = req.header(header::AUTHORIZATION, auth.clone()) } - let range = args.range(); if !range.is_full() { req = req.header(header::RANGE, range.to_header()); } diff --git a/core/src/services/webdav/error.rs b/core/src/services/webdav/error.rs index 08e463b8aa33..930c136a13c6 100644 --- a/core/src/services/webdav/error.rs +++ b/core/src/services/webdav/error.rs @@ -15,6 +15,7 @@ // specific language governing permissions and limitations // under the License. +use bytes::Buf; use http::Response; use http::StatusCode; diff --git a/core/src/services/webdav/lister.rs b/core/src/services/webdav/lister.rs index f1749d522dd2..d94914dfe181 100644 --- a/core/src/services/webdav/lister.rs +++ b/core/src/services/webdav/lister.rs @@ -56,7 +56,7 @@ impl oio::PageList for WebdavLister { }; let bs = if resp.status().is_success() { - resp.into_body().bytes().await? + resp.into_body() } else if resp.status() == StatusCode::NOT_FOUND && self.path.ends_with('/') { ctx.done = true; return Ok(()); @@ -64,7 +64,7 @@ impl oio::PageList for WebdavLister { return Err(parse_error(resp).await?); }; - let result: Multistatus = deserialize_multistatus(&bs)?; + let result: Multistatus = deserialize_multistatus(&bs.to_bytes())?; for res in result.response { let mut path = res diff --git a/core/src/services/webdav/mod.rs b/core/src/services/webdav/mod.rs index 1e18871a98c8..455fbe3d2ddd 100644 --- a/core/src/services/webdav/mod.rs +++ b/core/src/services/webdav/mod.rs @@ -22,4 +22,5 @@ pub use backend::WebdavConfig; mod core; mod error; mod lister; +mod reader; mod writer; diff --git a/core/src/services/webdav/reader.rs b/core/src/services/webdav/reader.rs new file mode 100644 index 000000000000..2817b8276d1d --- /dev/null +++ b/core/src/services/webdav/reader.rs @@ -0,0 +1,58 @@ +// 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 super::core::WebdavCore; +use super::error::parse_error; +use crate::raw::{oio, OpRead}; +use http::StatusCode; +use std::future::Future; +use std::sync::Arc; + +pub struct WebdavReader { + core: Arc, + + path: String, + op: OpRead, +} + +impl WebdavReader { + pub fn new(core: Arc, path: &str, op: OpRead) -> Self { + WebdavReader { + core, + path: path.to_string(), + op: op, + } + } +} + +impl oio::Read for WebdavReader { + async fn read_at(&self, offset: u64, limit: usize) -> crate::Result { + let Some(range) = self.op.range().apply_on_offset(offset, limit) else { + return Ok(oio::Buffer::new()); + }; + + let resp = self.core.webdav_get(&self.path, range, &self.op).await?; + + let status = resp.status(); + + match status { + StatusCode::OK | StatusCode::PARTIAL_CONTENT => Ok(resp.into_body()), + StatusCode::RANGE_NOT_SATISFIABLE => Ok(oio::Buffer::new()), + _ => Err(parse_error(resp).await?), + } + } +} From c78729564928b9795ed081953e9f7c52a8cefb89 Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Tue, 19 Mar 2024 21:25:31 +0800 Subject: [PATCH 040/111] Fix webhdfs Signed-off-by: Xuanwo --- core/src/services/webhdfs/backend.rs | 47 ++++++------------- core/src/services/webhdfs/error.rs | 1 + core/src/services/webhdfs/lister.rs | 9 ++-- core/src/services/webhdfs/mod.rs | 1 + core/src/services/webhdfs/reader.rs | 70 ++++++++++++++++++++++++++++ 5 files changed, 90 insertions(+), 38 deletions(-) create mode 100644 core/src/services/webhdfs/reader.rs diff --git a/core/src/services/webhdfs/backend.rs b/core/src/services/webhdfs/backend.rs index a0cdfd5de160..d63129c1ef1f 100644 --- a/core/src/services/webhdfs/backend.rs +++ b/core/src/services/webhdfs/backend.rs @@ -19,6 +19,7 @@ use core::fmt::Debug; use std::collections::HashMap; use async_trait::async_trait; +use bytes::Buf; use http::header::CONTENT_LENGTH; use http::header::CONTENT_TYPE; use http::Request; @@ -37,6 +38,7 @@ use super::message::FileStatusWrapper; use super::writer::WebhdfsWriter; use super::writer::WebhdfsWriters; use crate::raw::*; +use crate::services::webhdfs::reader::WebhdfsReader; use crate::*; const WEBHDFS_DEFAULT_ENDPOINT: &str = "http://127.0.0.1:9870"; @@ -269,8 +271,8 @@ impl WebhdfsBackend { let bs = resp.into_body(); - let resp = - serde_json::from_slice::(&bs).map_err(new_json_deserialize_error)?; + let resp: LocationResponse = + serde_json::from_reader(bs.reader()).map_err(new_json_deserialize_error)?; let mut req = Request::put(&resp.location); @@ -473,7 +475,7 @@ impl WebhdfsBackend { self.client.send(req).await } - async fn webhdfs_read_file( + pub async fn webhdfs_read_file( &self, path: &str, range: BytesRange, @@ -528,7 +530,7 @@ impl WebhdfsBackend { StatusCode::OK => { let bs = resp.into_body(); - let file_status = serde_json::from_slice::(&bs) + let file_status = serde_json::from_reader::<_, FileStatusWrapper>(bs.reader()) .map_err(new_json_deserialize_error)? .file_status; @@ -550,7 +552,7 @@ impl WebhdfsBackend { #[async_trait] impl Accessor for WebhdfsBackend { - type Reader = oio::Buffer; + type Reader = WebhdfsReader; type Writer = WebhdfsWriters; type Lister = oio::PageLister; type BlockingReader = (); @@ -598,7 +600,7 @@ impl Accessor for WebhdfsBackend { StatusCode::CREATED | StatusCode::OK => { let bs = resp.into_body(); - let resp = serde_json::from_slice::(&bs) + let resp = serde_json::from_reader::<_, BooleanResp>(bs.reader()) .map_err(new_json_deserialize_error)?; if resp.boolean { @@ -626,7 +628,7 @@ impl Accessor for WebhdfsBackend { StatusCode::OK => { let bs = resp.into_body(); - let file_status = serde_json::from_slice::(&bs) + let file_status = serde_json::from_reader::<_, FileStatusWrapper>(bs.reader()) .map_err(new_json_deserialize_error)? .file_status; @@ -647,33 +649,10 @@ impl Accessor for WebhdfsBackend { } async fn read(&self, path: &str, args: OpRead) -> Result<(RpRead, Self::Reader)> { - let range = args.range(); - let resp = self.webhdfs_read_file(path, range).await?; - match resp.status() { - StatusCode::OK | StatusCode::PARTIAL_CONTENT => { - let size = parse_content_length(resp.headers())?; - let range = parse_content_range(resp.headers())?; - Ok(( - RpRead::new().with_size(size).with_range(range), - resp.into_body(), - )) - } - // WebHDFS will returns 403 when range is outside of the end. - StatusCode::FORBIDDEN => { - let (parts, mut body) = resp.into_parts(); - let bs = body.copy_to_bytes(body.remaining()); - let s = String::from_utf8_lossy(&bs); - if s.contains("out of the range") { - Ok((RpRead::new(), oio::Buffer::empty())) - } else { - Err(parse_error_msg(parts, &s)?) - } - } - StatusCode::RANGE_NOT_SATISFIABLE => { - Ok((RpRead::new().with_size(Some(0)), oio::Buffer::empty())) - } - _ => Err(parse_error(resp).await?), - } + Ok(( + RpRead::default(), + WebhdfsReader::new(self.clone(), path, args), + )) } async fn write(&self, path: &str, args: OpWrite) -> Result<(RpWrite, Self::Writer)> { diff --git a/core/src/services/webhdfs/error.rs b/core/src/services/webhdfs/error.rs index bb0b6f846e0d..90d3aac74f3d 100644 --- a/core/src/services/webhdfs/error.rs +++ b/core/src/services/webhdfs/error.rs @@ -15,6 +15,7 @@ // specific language governing permissions and limitations // under the License. +use bytes::Buf; use http::response::Parts; use http::Response; use http::StatusCode; diff --git a/core/src/services/webhdfs/lister.rs b/core/src/services/webhdfs/lister.rs index c9e8036f14b6..fe758c4142a4 100644 --- a/core/src/services/webhdfs/lister.rs +++ b/core/src/services/webhdfs/lister.rs @@ -15,6 +15,7 @@ // specific language governing permissions and limitations // under the License. +use bytes::Buf; use http::StatusCode; use super::backend::WebhdfsBackend; @@ -46,7 +47,7 @@ impl oio::PageList for WebhdfsLister { ctx.done = true; let bs = resp.into_body(); - serde_json::from_slice::(&bs) + serde_json::from_reader::<_, FileStatusesWrapper>(bs.reader()) .map_err(new_json_deserialize_error)? .file_statuses .file_status @@ -65,9 +66,9 @@ impl oio::PageList for WebhdfsLister { match resp.status() { StatusCode::OK => { let bs = resp.into_body(); - let directory_listing = serde_json::from_slice::(&bs) - .map_err(new_json_deserialize_error)? - .directory_listing; + let res: DirectoryListingWrapper = + serde_json::from_reader(bs.reader()).map_err(new_json_deserialize_error)?; + let directory_listing = res.directory_listing; let file_statuses = directory_listing.partial_listing.file_statuses.file_status; if directory_listing.remaining_entries == 0 { diff --git a/core/src/services/webhdfs/mod.rs b/core/src/services/webhdfs/mod.rs index 3bfbbc77dd14..5118af268109 100644 --- a/core/src/services/webhdfs/mod.rs +++ b/core/src/services/webhdfs/mod.rs @@ -21,4 +21,5 @@ pub use backend::WebhdfsBuilder as Webhdfs; mod error; mod lister; mod message; +mod reader; mod writer; diff --git a/core/src/services/webhdfs/reader.rs b/core/src/services/webhdfs/reader.rs new file mode 100644 index 000000000000..415a1167c19d --- /dev/null +++ b/core/src/services/webhdfs/reader.rs @@ -0,0 +1,70 @@ +// 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 super::error::{parse_error, parse_error_msg}; +use crate::raw::{oio, OpRead, RpRead}; +use crate::services::webhdfs::backend::WebhdfsBackend; +use bytes::Buf; +use http::StatusCode; +use std::future::Future; +use std::sync::Arc; + +pub struct WebhdfsReader { + core: WebhdfsBackend, + + path: String, + op: OpRead, +} + +impl WebhdfsReader { + pub fn new(core: WebhdfsBackend, path: &str, op: OpRead) -> Self { + WebhdfsReader { + core, + path: path.to_string(), + op: op, + } + } +} + +impl oio::Read for WebhdfsReader { + async fn read_at(&self, offset: u64, limit: usize) -> crate::Result { + let Some(range) = self.op.range().apply_on_offset(offset, limit) else { + return Ok(oio::Buffer::new()); + }; + + let resp = self.core.webhdfs_read_file(&self.path, range).await?; + + let status = resp.status(); + + match status { + StatusCode::OK | StatusCode::PARTIAL_CONTENT => Ok(resp.into_body()), + // WebHDFS will returns 403 when range is outside of the end. + StatusCode::FORBIDDEN => { + let (parts, mut body) = resp.into_parts(); + let bs = body.copy_to_bytes(body.remaining()); + let s = String::from_utf8_lossy(&bs); + if s.contains("out of the range") { + Ok(oio::Buffer::new()) + } else { + Err(parse_error_msg(parts, &s)?) + } + } + StatusCode::RANGE_NOT_SATISFIABLE => Ok(oio::Buffer::new()), + _ => Err(parse_error(resp).await?), + } + } +} From 1afe1c656be56bad01151fbef4e4e4416c453c1f Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Tue, 19 Mar 2024 21:28:17 +0800 Subject: [PATCH 041/111] Fix yandex Signed-off-by: Xuanwo --- core/src/services/yandex_disk/backend.rs | 30 +++-------- core/src/services/yandex_disk/core.rs | 1 + core/src/services/yandex_disk/lister.rs | 1 + core/src/services/yandex_disk/mod.rs | 1 + core/src/services/yandex_disk/reader.rs | 65 ++++++++++++++++++++++++ 5 files changed, 76 insertions(+), 22 deletions(-) create mode 100644 core/src/services/yandex_disk/reader.rs diff --git a/core/src/services/yandex_disk/backend.rs b/core/src/services/yandex_disk/backend.rs index c515ca511375..09b4d2c34425 100644 --- a/core/src/services/yandex_disk/backend.rs +++ b/core/src/services/yandex_disk/backend.rs @@ -21,6 +21,7 @@ use std::fmt::Formatter; use std::sync::Arc; use async_trait::async_trait; +use bytes::Buf; use http::Request; use http::StatusCode; use log::debug; @@ -32,6 +33,7 @@ use super::lister::YandexDiskLister; use super::writer::YandexDiskWriter; use super::writer::YandexDiskWriters; use crate::raw::*; +use crate::services::yandex_disk::reader::YandexDiskReader; use crate::*; /// Config for backblaze YandexDisk services support. @@ -179,7 +181,7 @@ pub struct YandexDiskBackend { #[async_trait] impl Accessor for YandexDiskBackend { - type Reader = oio::Buffer; + type Reader = YandexDiskReader; type Writer = YandexDiskWriters; type Lister = oio::PageLister; type BlockingReader = (); @@ -245,27 +247,11 @@ impl Accessor for YandexDiskBackend { } } - async fn read(&self, path: &str, _args: OpRead) -> Result<(RpRead, Self::Reader)> { - let download_url = self.core.get_download_url(path).await?; - - let req = Request::get(download_url) - .body(AsyncBody::Empty) - .map_err(new_request_build_error)?; - let resp = self.core.send(req).await?; - - let status = resp.status(); - - match status { - StatusCode::OK => { - let size = parse_content_length(resp.headers())?; - let range = parse_content_range(resp.headers())?; - Ok(( - RpRead::new().with_size(size).with_range(range), - resp.into_body(), - )) - } - _ => Err(parse_error(resp).await?), - } + async fn read(&self, path: &str, args: OpRead) -> Result<(RpRead, Self::Reader)> { + Ok(( + RpRead::default(), + YandexDiskReader::new(self.core.clone(), path, args), + )) } async fn stat(&self, path: &str, _args: OpStat) -> Result { diff --git a/core/src/services/yandex_disk/core.rs b/core/src/services/yandex_disk/core.rs index ab658b3498bc..28fba494a3ec 100644 --- a/core/src/services/yandex_disk/core.rs +++ b/core/src/services/yandex_disk/core.rs @@ -15,6 +15,7 @@ // specific language governing permissions and limitations // under the License. +use bytes::Buf; use std::fmt::Debug; use std::fmt::Formatter; diff --git a/core/src/services/yandex_disk/lister.rs b/core/src/services/yandex_disk/lister.rs index 26addb5433fa..d7c947f1a352 100644 --- a/core/src/services/yandex_disk/lister.rs +++ b/core/src/services/yandex_disk/lister.rs @@ -15,6 +15,7 @@ // specific language governing permissions and limitations // under the License. +use bytes::Buf; use std::sync::Arc; use super::core::parse_info; diff --git a/core/src/services/yandex_disk/mod.rs b/core/src/services/yandex_disk/mod.rs index e2f2aff44888..606f8f816109 100644 --- a/core/src/services/yandex_disk/mod.rs +++ b/core/src/services/yandex_disk/mod.rs @@ -22,4 +22,5 @@ pub use backend::YandexDiskConfig; mod core; mod error; mod lister; +mod reader; mod writer; diff --git a/core/src/services/yandex_disk/reader.rs b/core/src/services/yandex_disk/reader.rs new file mode 100644 index 000000000000..dd417c5f3850 --- /dev/null +++ b/core/src/services/yandex_disk/reader.rs @@ -0,0 +1,65 @@ +// 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 super::core::YandexDiskCore; +use super::error::parse_error; +use crate::raw::{new_request_build_error, oio, AsyncBody, OpRead}; +use http::{header, Request, StatusCode}; +use std::future::Future; +use std::sync::Arc; + +pub struct YandexDiskReader { + core: Arc, + + path: String, + op: OpRead, +} + +impl YandexDiskReader { + pub fn new(core: Arc, path: &str, op: OpRead) -> Self { + YandexDiskReader { + core, + path: path.to_string(), + op: op, + } + } +} + +impl oio::Read for YandexDiskReader { + async fn read_at(&self, offset: u64, limit: usize) -> crate::Result { + let Some(range) = self.op.range().apply_on_offset(offset, limit) else { + return Ok(oio::Buffer::new()); + }; + + // TODO: move this out of reader. + let download_url = self.core.get_download_url(&self.path).await?; + + let req = Request::get(download_url) + .header(header::RANGE, range.to_header()) + .body(AsyncBody::Empty) + .map_err(new_request_build_error)?; + let resp = self.core.send(req).await?; + + let status = resp.status(); + + match status { + StatusCode::OK | StatusCode::PARTIAL_CONTENT => Ok(resp.into_body()), + StatusCode::RANGE_NOT_SATISFIABLE => Ok(oio::Buffer::new()), + _ => Err(parse_error(resp).await?), + } + } +} From f59472132d178c8d3749cf10def586d1e163c072 Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Tue, 19 Mar 2024 21:50:08 +0800 Subject: [PATCH 042/111] Save current work Signed-off-by: Xuanwo --- core/src/layers/blocking.rs | 8 +-- core/src/layers/chaos.rs | 33 +++-------- core/src/layers/concurrent_limit.rs | 8 +-- core/src/layers/dtrace.rs | 47 ++++----------- core/src/layers/error_context.rs | 14 +---- core/src/layers/logging.rs | 41 ++----------- core/src/layers/madsim.rs | 15 ++--- core/src/layers/metrics.rs | 59 ++++--------------- core/src/layers/minitrace.rs | 19 ++---- core/src/layers/oteltrace.rs | 16 ++--- core/src/layers/prometheus.rs | 31 +++------- core/src/layers/prometheus_client.rs | 51 ++++++++-------- core/src/layers/retry.rs | 46 ++------------- core/src/layers/throttle.rs | 16 ++--- core/src/layers/timeout.rs | 6 +- core/src/layers/tracing.rs | 24 ++------ core/src/raw/enum_utils.rs | 50 ++++------------ core/src/raw/oio/cursor.rs | 4 +- core/src/raw/oio/read/api.rs | 34 +++-------- core/src/raw/oio/read/buffer_reader.rs | 10 ++-- core/src/raw/oio/read/file_read.rs | 4 +- core/src/raw/oio/read/futures_read.rs | 2 +- .../src/raw/oio/read/into_read_from_stream.rs | 2 +- core/src/raw/oio/read/into_streamable_read.rs | 4 +- core/src/raw/oio/read/lazy_read.rs | 4 +- core/src/raw/oio/read/range_read.rs | 6 +- core/src/raw/oio/read/std_read.rs | 2 +- core/src/raw/oio/read/tokio_read.rs | 2 +- core/src/types/reader.rs | 4 +- 29 files changed, 143 insertions(+), 419 deletions(-) diff --git a/core/src/layers/blocking.rs b/core/src/layers/blocking.rs index bc6ff054820b..d48bda0c1893 100644 --- a/core/src/layers/blocking.rs +++ b/core/src/layers/blocking.rs @@ -288,12 +288,8 @@ impl BlockingWrapper { } impl oio::BlockingRead for BlockingWrapper { - fn read(&mut self, limit: usize) -> Result { - self.handle.block_on(self.inner.read(limit)) - } - - fn seek(&mut self, pos: std::io::SeekFrom) -> Result { - self.handle.block_on(self.inner.seek(pos)) + fn read_at(&mut self, offset: u64, limit: usize) -> Result { + self.handle.block_on(self.inner.read_at(offset, limit)) } } diff --git a/core/src/layers/chaos.rs b/core/src/layers/chaos.rs index 69c16e8adf70..ac646b9ae379 100644 --- a/core/src/layers/chaos.rs +++ b/core/src/layers/chaos.rs @@ -16,6 +16,7 @@ // under the License. use std::io; +use std::sync::{Arc, Mutex}; use async_trait::async_trait; use bytes::Bytes; @@ -145,7 +146,7 @@ impl LayeredAccessor for ChaosAccessor { /// ChaosReader will inject error into read operations. pub struct ChaosReader { inner: R, - rng: StdRng, + rng: Arc>, error_ratio: f64, } @@ -154,15 +155,15 @@ impl ChaosReader { fn new(inner: R, rng: StdRng, error_ratio: f64) -> Self { Self { inner, - rng, + rng: Arc::new(Mutex::new(rng)), error_ratio, } } /// If I feel lucky, we can return the correct response. Otherwise, /// we need to generate an error. - fn i_feel_lucky(&mut self) -> bool { - let point = self.rng.gen_range(0..=100); + fn i_feel_lucky(&self) -> bool { + let point = self.rng.lock().unwrap().gen_range(0..=100); point >= (self.error_ratio * 100.0) as i32 } @@ -174,17 +175,9 @@ impl ChaosReader { } impl oio::Read for ChaosReader { - async fn read(&mut self, limit: usize) -> Result { - if self.i_feel_lucky() { - self.inner.read(limit).await - } else { - Err(Self::unexpected_eof()) - } - } - - async fn seek(&mut self, pos: io::SeekFrom) -> Result { + async fn read_at(&self, offset: u64, limit: usize) -> Result { if self.i_feel_lucky() { - self.inner.seek(pos).await + self.inner.read_at(offset, limit).await } else { Err(Self::unexpected_eof()) } @@ -192,17 +185,9 @@ impl oio::Read for ChaosReader { } impl oio::BlockingRead for ChaosReader { - fn read(&mut self, limit: usize) -> Result { - if self.i_feel_lucky() { - self.inner.read(limit) - } else { - Err(Self::unexpected_eof()) - } - } - - fn seek(&mut self, pos: io::SeekFrom) -> Result { + fn read_at(&self, offset: u64, limit: usize) -> Result { if self.i_feel_lucky() { - self.inner.seek(pos) + self.inner.read_at(offset, limit) } else { Err(Self::unexpected_eof()) } diff --git a/core/src/layers/concurrent_limit.rs b/core/src/layers/concurrent_limit.rs index 63de98d9f900..c4ef48a1dcdd 100644 --- a/core/src/layers/concurrent_limit.rs +++ b/core/src/layers/concurrent_limit.rs @@ -264,12 +264,8 @@ impl oio::Read for ConcurrentLimitWrapper { } impl oio::BlockingRead for ConcurrentLimitWrapper { - fn read(&mut self, limit: usize) -> Result { - self.inner.read(limit) - } - - fn seek(&mut self, pos: SeekFrom) -> Result { - self.inner.seek(pos) + fn read_at(&self, offset: u64, limit: usize) -> Result { + self.inner.read_at(offset, limit) } } diff --git a/core/src/layers/dtrace.rs b/core/src/layers/dtrace.rs index 295443cd49b8..99203de8a612 100644 --- a/core/src/layers/dtrace.rs +++ b/core/src/layers/dtrace.rs @@ -344,12 +344,12 @@ impl DtraceLayerWrapper { } impl oio::Read for DtraceLayerWrapper { - async fn read(&mut self, limit: usize) -> Result { + async fn read_at(&self, offset: u64, limit: usize) -> Result { let c_path = CString::new(self.path.clone()).unwrap(); probe_lazy!(opendal, reader_read_start, c_path.as_ptr()); - match self.inner.read(limit).await { + match self.inner.read_at(offset, limit).await { Ok(bs) => { - probe_lazy!(opendal, reader_read_ok, c_path.as_ptr(), bs.len()); + probe_lazy!(opendal, reader_read_ok, c_path.as_ptr(), bs.remaining()); Ok(bs) } Err(e) => { @@ -358,31 +358,21 @@ impl oio::Read for DtraceLayerWrapper { } } } - - async fn seek(&mut self, pos: io::SeekFrom) -> Result { - let c_path = CString::new(self.path.clone()).unwrap(); - probe_lazy!(opendal, reader_seek_start, c_path.as_ptr()); - match self.inner.seek(pos).await { - Ok(n) => { - probe_lazy!(opendal, reader_seek_ok, c_path.as_ptr(), n); - Ok(n) - } - Err(e) => { - probe_lazy!(opendal, reader_seek_error, c_path.as_ptr()); - Err(e) - } - } - } } impl oio::BlockingRead for DtraceLayerWrapper { - fn read(&mut self, limit: usize) -> Result { + fn read_at(&self, offset: u64, limit: usize) -> Result { let c_path = CString::new(self.path.clone()).unwrap(); probe_lazy!(opendal, blocking_reader_read_start, c_path.as_ptr()); self.inner - .read(limit) + .read_at(offset, limit) .map(|bs| { - probe_lazy!(opendal, blocking_reader_read_ok, c_path.as_ptr(), bs.len()); + probe_lazy!( + opendal, + blocking_reader_read_ok, + c_path.as_ptr(), + bs.remaining() + ); bs }) .map_err(|e| { @@ -390,21 +380,6 @@ impl oio::BlockingRead for DtraceLayerWrapper { e }) } - - fn seek(&mut self, pos: io::SeekFrom) -> Result { - let c_path = CString::new(self.path.clone()).unwrap(); - probe_lazy!(opendal, blocking_reader_seek_start, c_path.as_ptr()); - self.inner - .seek(pos) - .map(|res| { - probe_lazy!(opendal, blocking_reader_seek_ok, c_path.as_ptr(), res); - res - }) - .map_err(|e| { - probe_lazy!(opendal, blocking_reader_seek_error, c_path.as_ptr()); - e - }) - } } impl oio::Write for DtraceLayerWrapper { diff --git a/core/src/layers/error_context.rs b/core/src/layers/error_context.rs index 28eee259ceee..8996ad0f03d2 100644 --- a/core/src/layers/error_context.rs +++ b/core/src/layers/error_context.rs @@ -360,23 +360,15 @@ impl oio::Read for ErrorContextWrapper { } impl oio::BlockingRead for ErrorContextWrapper { - fn read(&mut self, limit: usize) -> Result { - self.inner.read(limit).map_err(|err| { + fn read_at(&self, offset: u64, limit: usize) -> Result { + self.inner.read_at(offset, limit).map_err(|err| { err.with_operation(ReadOperation::BlockingRead) .with_context("service", self.scheme) .with_context("path", &self.path) + .with_context("offset", offset.to_string()) .with_context("limit", limit.to_string()) }) } - - fn seek(&mut self, pos: SeekFrom) -> Result { - self.inner.seek(pos).map_err(|err| { - err.with_operation(ReadOperation::BlockingSeek) - .with_context("service", self.scheme) - .with_context("path", &self.path) - .with_context("seek", format!("{pos:?}")) - }) - } } impl oio::Write for ErrorContextWrapper { diff --git a/core/src/layers/logging.rs b/core/src/layers/logging.rs index 778d0f68f544..96fb252aba01 100644 --- a/core/src/layers/logging.rs +++ b/core/src/layers/logging.rs @@ -1026,10 +1026,11 @@ impl oio::Read for LoggingReader { } impl oio::BlockingRead for LoggingReader { - fn read(&mut self, limit: usize) -> Result { - match self.inner.read(limit) { + fn read_at(&self, offset: u64, limit: usize) -> Result { + match self.inner.read_at(offset, limit) { Ok(bs) => { - self.read.fetch_add(bs.len() as u64, Ordering::Relaxed); + self.read + .fetch_add(bs.remaining() as u64, Ordering::Relaxed); trace!( target: LOGGING_TARGET, "service={} operation={} path={} read={} -> data read {}B", @@ -1037,7 +1038,7 @@ impl oio::BlockingRead for LoggingReader { ReadOperation::BlockingRead, self.path, self.read.load(Ordering::Relaxed), - bs.len() + bs.remaining() ); Ok(bs) } @@ -1058,38 +1059,6 @@ impl oio::BlockingRead for LoggingReader { } } } - - #[inline] - fn seek(&mut self, pos: io::SeekFrom) -> Result { - match self.inner.seek(pos) { - Ok(n) => { - trace!( - target: LOGGING_TARGET, - "service={} operation={} path={} read={} -> data seek to offset {n}", - self.ctx.scheme, - ReadOperation::BlockingSeek, - self.path, - self.read.load(Ordering::Relaxed), - ); - Ok(n) - } - Err(err) => { - if let Some(lvl) = self.ctx.error_level(&err) { - log!( - target: LOGGING_TARGET, - lvl, - "service={} operation={} path={} read={} -> data read failed: {}", - self.ctx.scheme, - ReadOperation::BlockingSeek, - self.path, - self.read.load(Ordering::Relaxed), - self.ctx.error_print(&err), - ); - } - Err(err) - } - } - } } pub struct LoggingWriter { diff --git a/core/src/layers/madsim.rs b/core/src/layers/madsim.rs index 4af7c1d84d1e..b48580a10bfa 100644 --- a/core/src/layers/madsim.rs +++ b/core/src/layers/madsim.rs @@ -264,21 +264,14 @@ pub struct MadsimReader { } impl oio::Read for MadsimReader { - async fn read(&mut self, size: usize) -> crate::Result { + async fn read_at(&self, offset: u64, limit: usize) -> crate::Result { if let Some(ref data) = self.data { - let size = min(size, data.len()); - Ok(data.clone().split_to(size)) + let size = min(limit, data.len()); + Ok(data.clone().split_to(size).into()) } else { - Ok(Bytes::new()) + Ok(oio::Buffer::new()) } } - - async fn seek(&mut self, _: SeekFrom) -> crate::Result { - Err(Error::new( - ErrorKind::Unsupported, - "will be supported in the future", - )) - } } pub struct MadsimWriter { diff --git a/core/src/layers/metrics.rs b/core/src/layers/metrics.rs index a4abc41bdfc5..d2fe64d70082 100644 --- a/core/src/layers/metrics.rs +++ b/core/src/layers/metrics.rs @@ -24,7 +24,7 @@ use std::sync::Arc; use std::time::Instant; use async_trait::async_trait; -use bytes::Bytes; +use bytes::{Buf, Bytes}; use futures::FutureExt; use futures::TryFutureExt; use metrics::increment_counter; @@ -470,7 +470,6 @@ impl LayeredAccessor for MetricsAccessor { self.handle.clone(), self.handle.bytes_total_read.clone(), self.handle.requests_duration_seconds_read.clone(), - Some(start), ), ) }) @@ -499,7 +498,6 @@ impl LayeredAccessor for MetricsAccessor { self.handle.clone(), self.handle.bytes_total_write.clone(), self.handle.requests_duration_seconds_write.clone(), - Some(start), ), ) }) @@ -630,7 +628,6 @@ impl LayeredAccessor for MetricsAccessor { self.handle.clone(), self.handle.bytes_total_blocking_read.clone(), self.handle.requests_duration_seconds_blocking_read.clone(), - Some(start), ), ) }); @@ -663,7 +660,6 @@ impl LayeredAccessor for MetricsAccessor { self.handle.clone(), self.handle.bytes_total_write.clone(), self.handle.requests_duration_seconds_write.clone(), - Some(start), ), ) }) @@ -736,9 +732,6 @@ pub struct MetricWrapper { bytes_counter: Counter, requests_duration_seconds: Histogram, handle: Arc, - - start: Option, - bytes: u64, } impl MetricWrapper { @@ -748,7 +741,6 @@ impl MetricWrapper { handle: Arc, bytes_counter: Counter, requests_duration_seconds: Histogram, - start: Option, ) -> Self { Self { inner, @@ -756,28 +748,16 @@ impl MetricWrapper { handle, bytes_counter, requests_duration_seconds, - start, - bytes: 0, - } - } -} - -impl Drop for MetricWrapper { - fn drop(&mut self) { - self.bytes_counter.increment(self.bytes); - if let Some(instant) = self.start { - let dur = instant.elapsed().as_secs_f64(); - self.requests_duration_seconds.record(dur); } } } impl oio::Read for MetricWrapper { - async fn read(&mut self, limit: usize) -> Result { - match self.inner.read(limit).await { - Ok(bytes) => { - self.bytes += bytes.len() as u64; - Ok(bytes) + async fn read_at(&self, offset: u64, limit: usize) -> Result { + match self.inner.read_at(offset, limit).await { + Ok(bs) => { + self.bytes_counter.increment(bs.remaining() as u64); + Ok(bs) } Err(e) => { self.handle.increment_errors_total(self.op, e.kind()); @@ -785,24 +765,14 @@ impl oio::Read for MetricWrapper { } } } - - async fn seek(&mut self, pos: io::SeekFrom) -> Result { - match self.inner.seek(pos).await { - Ok(n) => Ok(n), - Err(e) => { - self.handle.increment_errors_total(self.op, e.kind()); - Err(e) - } - } - } } impl oio::BlockingRead for MetricWrapper { - fn read(&mut self, limit: usize) -> Result { + fn read_at(&self, offset: u64, limit: usize) -> Result { self.inner - .read(limit) + .read_at(offset, limit) .map(|bs| { - self.bytes += bs.len() as u64; + self.bytes_counter.increment(bs.remaining() as u64); bs }) .map_err(|e| { @@ -810,13 +780,6 @@ impl oio::BlockingRead for MetricWrapper { e }) } - - fn seek(&mut self, pos: io::SeekFrom) -> Result { - self.inner.seek(pos).map_err(|err| { - self.handle.increment_errors_total(self.op, err.kind()); - err - }) - } } impl oio::Write for MetricWrapper { @@ -824,7 +787,7 @@ impl oio::Write for MetricWrapper { self.inner .write(bs) .map_ok(|n| { - self.bytes += n as u64; + self.bytes_counter.increment(bs.remaining() as u64); n }) .map_err(|err| { @@ -853,7 +816,7 @@ impl oio::BlockingWrite for MetricWrapper { self.inner .write(bs) .map(|n| { - self.bytes += n as u64; + self.bytes_counter.increment(bs.remaining() as u64); n }) .map_err(|err| { diff --git a/core/src/layers/minitrace.rs b/core/src/layers/minitrace.rs index 8dc720fe57ec..e256bcb4e453 100644 --- a/core/src/layers/minitrace.rs +++ b/core/src/layers/minitrace.rs @@ -298,27 +298,16 @@ impl MinitraceWrapper { impl oio::Read for MinitraceWrapper { #[trace(enter_on_poll = true)] - async fn read(&mut self, limit: usize) -> Result { - self.inner.read(limit).await - } - - #[trace(enter_on_poll = true)] - async fn seek(&mut self, pos: io::SeekFrom) -> Result { - self.inner.seek(pos).await + async fn read_at(&self, offset: u64, limit: usize) -> Result { + self.inner.read_at(offset, limit).await } } impl oio::BlockingRead for MinitraceWrapper { - fn read(&mut self, limit: usize) -> Result { + fn read_at(&self, offset: u64, limit: usize) -> Result { let _g = self.span.set_local_parent(); let _span = LocalSpan::enter_with_local_parent(ReadOperation::BlockingRead.into_static()); - self.inner.read(limit) - } - - fn seek(&mut self, pos: io::SeekFrom) -> Result { - let _g = self.span.set_local_parent(); - let _span = LocalSpan::enter_with_local_parent(ReadOperation::BlockingSeek.into_static()); - self.inner.seek(pos) + self.inner.read_at(offset, limit) } } diff --git a/core/src/layers/oteltrace.rs b/core/src/layers/oteltrace.rs index 138811bb6fcd..bd03f1ddd18e 100644 --- a/core/src/layers/oteltrace.rs +++ b/core/src/layers/oteltrace.rs @@ -277,22 +277,14 @@ impl OtelTraceWrapper { } impl oio::Read for OtelTraceWrapper { - async fn read(&mut self, limit: usize) -> Result { - self.inner.read(limit).await - } - - async fn seek(&mut self, pos: io::SeekFrom) -> Result { - self.inner.seek(pos).await + async fn read_at(&self, offset: u64, limit: usize) -> Result { + self.inner.read_at(offset, limit).await } } impl oio::BlockingRead for OtelTraceWrapper { - fn read(&mut self, limit: usize) -> Result { - self.inner.read(limit) - } - - fn seek(&mut self, pos: io::SeekFrom) -> Result { - self.inner.seek(pos) + fn read_at(&self, offset: u64, limit: usize) -> Result { + self.inner.read_at(offset, limit) } } diff --git a/core/src/layers/prometheus.rs b/core/src/layers/prometheus.rs index b04f8ddb6397..f99edd06d456 100644 --- a/core/src/layers/prometheus.rs +++ b/core/src/layers/prometheus.rs @@ -22,7 +22,7 @@ use std::io; use std::sync::Arc; use async_trait::async_trait; -use bytes::Bytes; +use bytes::{Buf, Bytes}; use futures::FutureExt; use futures::TryFutureExt; use log::debug; @@ -685,18 +685,18 @@ impl PrometheusMetricWrapper { } impl oio::Read for PrometheusMetricWrapper { - async fn read(&mut self, limit: usize) -> Result { + async fn read_at(&self, offset: u64, limit: usize) -> Result { let labels = self.stats.generate_metric_label( self.scheme.into_static(), Operation::Read.into_static(), &self.path, ); - match self.inner.read(limit).await { + match self.inner.read_at(offset, limit).await { Ok(bytes) => { self.stats .bytes_total .with_label_values(&labels) - .observe(bytes.len() as f64); + .observe(bytes.remaining() as f64); Ok(bytes) } Err(e) => { @@ -705,32 +705,22 @@ impl oio::Read for PrometheusMetricWrapper { } } } - - async fn seek(&mut self, pos: io::SeekFrom) -> Result { - match self.inner.seek(pos).await { - Ok(n) => Ok(n), - Err(e) => { - self.stats.increment_errors_total(self.op, e.kind()); - Err(e) - } - } - } } impl oio::BlockingRead for PrometheusMetricWrapper { - fn read(&mut self, limit: usize) -> Result { + fn read_at(&self, offset: u64, limit: usize) -> Result { let labels = self.stats.generate_metric_label( self.scheme.into_static(), Operation::BlockingRead.into_static(), &self.path, ); self.inner - .read(limit) + .read_at(offset, limit) .map(|bs| { self.stats .bytes_total .with_label_values(&labels) - .observe(bs.len() as f64); + .observe(bs.remaining() as f64); bs }) .map_err(|e| { @@ -738,13 +728,6 @@ impl oio::BlockingRead for PrometheusMetricWrapper { e }) } - - fn seek(&mut self, pos: io::SeekFrom) -> Result { - self.inner.seek(pos).map_err(|err| { - self.stats.increment_errors_total(self.op, err.kind()); - err - }) - } } impl oio::Write for PrometheusMetricWrapper { diff --git a/core/src/layers/prometheus_client.rs b/core/src/layers/prometheus_client.rs index e29050571c32..cd813f6cf146 100644 --- a/core/src/layers/prometheus_client.rs +++ b/core/src/layers/prometheus_client.rs @@ -25,7 +25,7 @@ use std::time::Duration; use std::time::Instant; use async_trait::async_trait; -use bytes::Bytes; +use bytes::{Buf, Bytes}; use futures::FutureExt; use futures::TryFutureExt; use prometheus_client::metrics::counter::Counter; @@ -539,23 +539,17 @@ impl PrometheusMetricWrapper { } impl oio::Read for PrometheusMetricWrapper { - async fn read(&mut self, limit: usize) -> Result { - match self.inner.read(limit).await { - Ok(bytes) => { - self.bytes_total += bytes.len(); - Ok(bytes) - } - Err(e) => { + async fn read_at(&self, offset: u64, limit: usize) -> Result { + let start = Instant::now(); + + match self.inner.read_at(offset, limit).await { + Ok(bs) => { self.metrics - .increment_errors_total(self.scheme, self.op, e.kind()); - Err(e) + .observe_bytes_total(self.scheme, self.op, bs.remaining()); + self.metrics + .observe_request_duration(self.scheme, self.op, start.elapsed()); + Ok(bs) } - } - } - - async fn seek(&mut self, pos: io::SeekFrom) -> Result { - match self.inner.seek(pos).await { - Ok(n) => Ok(n), Err(e) => { self.metrics .increment_errors_total(self.scheme, self.op, e.kind()); @@ -566,11 +560,15 @@ impl oio::Read for PrometheusMetricWrapper { } impl oio::BlockingRead for PrometheusMetricWrapper { - fn read(&mut self, limit: usize) -> Result { + fn read_at(&self, offset: u64, limit: usize) -> Result { + let start = Instant::now(); self.inner - .read(limit) + .read_at(offset, limit) .map(|bs| { - self.bytes_total += bs.len(); + self.metrics + .observe_bytes_total(self.scheme, self.op, bs.remaining()); + self.metrics + .observe_request_duration(self.scheme, self.op, start.elapsed()); bs }) .map_err(|e| { @@ -579,22 +577,19 @@ impl oio::BlockingRead for PrometheusMetricWrapper { e }) } - - fn seek(&mut self, pos: io::SeekFrom) -> Result { - self.inner.seek(pos).map_err(|err| { - self.metrics - .increment_errors_total(self.scheme, self.op, err.kind()); - err - }) - } } impl oio::Write for PrometheusMetricWrapper { fn write(&mut self, bs: Bytes) -> impl Future> + Send { + let start = Instant::now(); + self.inner .write(bs) .map_ok(|n| { - self.bytes_total += n; + self.metrics + .observe_bytes_total(self.scheme, self.op, bs.remaining()); + self.metrics + .observe_request_duration(self.scheme, self.op, start.elapsed()); n }) .map_err(|err| { diff --git a/core/src/layers/retry.rs b/core/src/layers/retry.rs index 9385842e5ec8..b266c525b20f 100644 --- a/core/src/layers/retry.rs +++ b/core/src/layers/retry.rs @@ -693,8 +693,8 @@ impl oio::Read for RetryWrapper { } impl oio::BlockingRead for RetryWrapper { - fn read(&mut self, limit: usize) -> Result { - { || self.inner.as_mut().unwrap().read(limit) } + fn read_at(&self, offset: u64, limit: usize) -> Result { + { || self.inner.as_ref().unwrap().read_at(offset, limit) } .retry(&self.builder) .when(|e| e.is_temporary()) .notify(|err, dur| { @@ -710,24 +710,6 @@ impl oio::BlockingRead for RetryWrapp .call() .map_err(|e| e.set_persistent()) } - - fn seek(&mut self, pos: io::SeekFrom) -> Result { - { || self.inner.as_mut().unwrap().seek(pos) } - .retry(&self.builder) - .when(|e| e.is_temporary()) - .notify(|err, dur| { - self.notify.intercept( - err, - dur, - &[ - ("operation", ReadOperation::BlockingSeek.into_static()), - ("path", &self.path), - ], - ); - }) - .call() - .map_err(|e| e.set_persistent()) - } } impl oio::Write for RetryWrapper { @@ -1061,17 +1043,7 @@ mod tests { } impl oio::Read for MockReader { - async fn seek(&mut self, pos: io::SeekFrom) -> Result { - self.pos = match pos { - io::SeekFrom::Current(n) => (self.pos as i64 + n) as u64, - io::SeekFrom::Start(n) => n, - io::SeekFrom::End(n) => (13 + n) as u64, - }; - - Ok(self.pos) - } - - async fn read(&mut self, _: usize) -> Result { + async fn read_at(&self, _: u64, _: usize) -> Result { let mut attempt = self.attempt.lock().unwrap(); *attempt += 1; @@ -1080,19 +1052,13 @@ mod tests { Error::new(ErrorKind::Unexpected, "retryable_error from reader") .set_temporary(), ), - 2 => { - self.pos += 7; - Ok(Bytes::copy_from_slice("Hello, ".as_bytes())) - } + 2 => Ok(Bytes::copy_from_slice("Hello, ".as_bytes()).into()), 3 => Err( Error::new(ErrorKind::Unexpected, "retryable_error from reader") .set_temporary(), ), - 4 => { - self.pos += 6; - Ok(Bytes::copy_from_slice("World!".as_bytes())) - } - 5 => Ok(Bytes::new()), + 4 => Ok(Bytes::copy_from_slice("World!".as_bytes()).into()), + 5 => Ok(Bytes::new().into()), _ => unreachable!(), } } diff --git a/core/src/layers/throttle.rs b/core/src/layers/throttle.rs index 11f7e9913fbd..ab2595b671e9 100644 --- a/core/src/layers/throttle.rs +++ b/core/src/layers/throttle.rs @@ -185,24 +185,16 @@ impl ThrottleWrapper { } impl oio::Read for ThrottleWrapper { - async fn read(&mut self, limit: usize) -> Result { + async fn read_at(&self, offset: u64, limit: usize) -> Result { // TODO: How can we handle buffer reads with a limiter? - self.inner.read(limit).await - } - - async fn seek(&mut self, pos: SeekFrom) -> Result { - self.inner.seek(pos).await + self.inner.read_at(offset, limit).await } } impl oio::BlockingRead for ThrottleWrapper { - fn read(&mut self, limit: usize) -> Result { + fn read_at(&self, offset: u64, limit: usize) -> Result { // TODO: How can we handle buffer reads with a limiter? - self.inner.read(limit) - } - - fn seek(&mut self, pos: SeekFrom) -> Result { - self.inner.seek(pos) + self.inner.read_at(offset, limit) } } diff --git a/core/src/layers/timeout.rs b/core/src/layers/timeout.rs index 57985440e940..98bea06bf582 100644 --- a/core/src/layers/timeout.rs +++ b/core/src/layers/timeout.rs @@ -388,11 +388,7 @@ mod tests { struct MockReader; impl oio::Read for MockReader { - fn seek(&mut self, _: SeekFrom) -> impl Future> { - pending() - } - - fn read(&mut self, _: usize) -> impl Future> { + fn read_at(&self, _: u64, _: usize) -> impl Future> { pending() } } diff --git a/core/src/layers/tracing.rs b/core/src/layers/tracing.rs index dcc0a79f4c91..7510c9720c79 100644 --- a/core/src/layers/tracing.rs +++ b/core/src/layers/tracing.rs @@ -272,16 +272,8 @@ impl oio::Read for TracingWrapper { parent = &self.span, level = "trace", skip_all)] - async fn read(&mut self, limit: usize) -> Result { - self.inner.read(limit).await - } - - #[tracing::instrument( - parent = &self.span, - level = "trace", - skip_all)] - async fn seek(&mut self, pos: io::SeekFrom) -> Result { - self.inner.seek(pos).await + async fn read_at(&self, offset: u64, limit: usize) -> Result { + self.inner.read_at(offset, limit).await } } @@ -290,16 +282,8 @@ impl oio::BlockingRead for TracingWrapper { parent = &self.span, level = "trace", skip_all)] - fn read(&mut self, limit: usize) -> Result { - self.inner.read(limit) - } - - #[tracing::instrument( - parent = &self.span, - level = "trace", - skip_all)] - fn seek(&mut self, pos: io::SeekFrom) -> Result { - self.inner.seek(pos) + fn read_at(&self, offset: u64, limit: usize) -> Result { + self.inner.read_at(offset, limit) } } diff --git a/core/src/raw/enum_utils.rs b/core/src/raw/enum_utils.rs index 96fe75a8438e..3fe83e82925a 100644 --- a/core/src/raw/enum_utils.rs +++ b/core/src/raw/enum_utils.rs @@ -67,17 +67,10 @@ impl oio::Read for TwoWays { } impl oio::BlockingRead for TwoWays { - fn read(&mut self, limit: usize) -> Result { + fn read_at(&self, offset: u64, limit: usize) -> Result { match self { - Self::One(v) => v.read(limit), - Self::Two(v) => v.read(limit), - } - } - - fn seek(&mut self, pos: SeekFrom) -> Result { - match self { - Self::One(v) => v.seek(pos), - Self::Two(v) => v.seek(pos), + Self::One(v) => v.read_at(offset, limit), + Self::Two(v) => v.read_at(offset, limit), } } } @@ -118,7 +111,7 @@ pub enum ThreeWays { } impl oio::Read for ThreeWays { - async fn read_at(&self, offset: u64, limit: usize) -> Result { + async fn read_at(&self, offset: u64, limit: usize) -> Result { match self { ThreeWays::One(v) => v.read_at(offset, limit).await, ThreeWays::Two(v) => v.read_at(offset, limit).await, @@ -130,19 +123,11 @@ impl oio::Read for ThreeWays oio::BlockingRead for ThreeWays { - fn read(&mut self, limit: usize) -> Result { + fn read_at(&self, offset: u64, limit: usize) -> Result { match self { - Self::One(v) => v.read(limit), - Self::Two(v) => v.read(limit), - Self::Three(v) => v.read(limit), - } - } - - fn seek(&mut self, pos: SeekFrom) -> Result { - match self { - Self::One(v) => v.seek(pos), - Self::Two(v) => v.seek(pos), - Self::Three(v) => v.seek(pos), + Self::One(v) => v.read_at(offset, limit), + Self::Two(v) => v.read_at(offset, limit), + Self::Three(v) => v.read_at(offset, limit), } } } @@ -213,21 +198,12 @@ where THREE: oio::BlockingRead, FOUR: oio::BlockingRead, { - fn read(&mut self, limit: usize) -> Result { - match self { - Self::One(v) => v.read(limit), - Self::Two(v) => v.read(limit), - Self::Three(v) => v.read(limit), - Self::Four(v) => v.read(limit), - } - } - - fn seek(&mut self, pos: SeekFrom) -> Result { + fn read_at(&mut self, offset: u64, limit: usize) -> Result { match self { - Self::One(v) => v.seek(pos), - Self::Two(v) => v.seek(pos), - Self::Three(v) => v.seek(pos), - Self::Four(v) => v.seek(pos), + Self::One(v) => v.read_at(offset, limit), + Self::Two(v) => v.read_at(offset, limit), + Self::Three(v) => v.read_at(offset, limit), + Self::Four(v) => v.read_at(offset, limit), } } } diff --git a/core/src/raw/oio/cursor.rs b/core/src/raw/oio/cursor.rs index 7320d764bc14..b875d7b83b9f 100644 --- a/core/src/raw/oio/cursor.rs +++ b/core/src/raw/oio/cursor.rs @@ -71,7 +71,7 @@ impl From> for Cursor { } // impl oio::Read for Cursor { -// async fn read(&mut self, limit: usize) -> Result { +// async fn read(&self,offset:u64, limit: usize) -> Result { // if self.is_empty() { // Ok(Bytes::new()) // } else { @@ -105,7 +105,7 @@ impl From> for Cursor { // } impl oio::BlockingRead for Cursor { - fn read(&mut self, limit: usize) -> Result { + fn read_at(&self, offset: u64, limit: usize) -> Result { if self.is_empty() { Ok(Bytes::new()) } else { diff --git a/core/src/raw/oio/read/api.rs b/core/src/raw/oio/read/api.rs index 371347107e88..4337fab5523a 100644 --- a/core/src/raw/oio/read/api.rs +++ b/core/src/raw/oio/read/api.rs @@ -17,13 +17,11 @@ use std::fmt::Display; use std::fmt::Formatter; -use std::io; use std::ops::Deref; use bytes::{Buf, Bytes}; use futures::Future; -use crate::raw::oio::Buffer; use crate::raw::*; use crate::*; @@ -111,13 +109,13 @@ impl Read for () { impl Read for Bytes { /// TODO: we can check if the offset is out of range. - async fn read_at(&self, offset: u64, limit: usize) -> Result { + async fn read_at(&self, offset: u64, limit: usize) -> Result { if offset >= self.len() as u64 { - return Ok(Buffer::new()); + return Ok(oio::Buffer::new()); } let offset = offset as usize; let limit = limit.min(self.len() - offset); - Ok(Buffer::from(self.slice(offset..offset + limit))) + Ok(oio::Buffer::from(self.slice(offset..offset + limit))) } } @@ -156,37 +154,21 @@ pub type BlockingReader = Box; /// is optional. We use `Read` to make users life easier. pub trait BlockingRead: Send + Sync { /// Read synchronously. - fn read(&mut self, limit: usize) -> Result; - - /// Seek synchronously. - fn seek(&mut self, pos: io::SeekFrom) -> Result; + fn read_at(&self, offset: u64, limit: usize) -> Result; } impl BlockingRead for () { - fn read(&mut self, limit: usize) -> Result { - let _ = limit; + fn read_at(&self, offset: u64, limit: usize) -> Result { + let _ = (offset, limit); unimplemented!("read is required to be implemented for oio::BlockingRead") } - - fn seek(&mut self, pos: io::SeekFrom) -> Result { - let _ = pos; - - Err(Error::new( - ErrorKind::Unsupported, - "output blocking reader doesn't support seeking", - )) - } } /// `Box` won't implement `BlockingRead` automatically. /// To make BlockingReader work as expected, we must add this impl. impl BlockingRead for Box { - fn read(&mut self, limit: usize) -> Result { - (**self).read(limit) - } - - fn seek(&mut self, pos: io::SeekFrom) -> Result { - (**self).seek(pos) + fn read_at(&self, offset: u64, limit: usize) -> Result { + (**self).read_at(offset, limit) } } diff --git a/core/src/raw/oio/read/buffer_reader.rs b/core/src/raw/oio/read/buffer_reader.rs index 733878d1cce0..2630937e5b40 100644 --- a/core/src/raw/oio/read/buffer_reader.rs +++ b/core/src/raw/oio/read/buffer_reader.rs @@ -135,7 +135,7 @@ impl oio::Read for BufferReader where R: oio::Read, { - async fn read(&mut self, limit: usize) -> Result { + async fn read_at(&self, offset: u64, limit: usize) -> Result { if limit == 0 { return Ok(Bytes::new()); } @@ -233,7 +233,7 @@ impl BlockingRead for BufferReader where R: BlockingRead, { - fn read(&mut self, limit: usize) -> Result { + fn read_at(&self, offset: u64, limit: usize) -> Result { if limit == 0 { return Ok(Bytes::new()); } @@ -379,14 +379,14 @@ mod tests { )) } - async fn read(&mut self, limit: usize) -> Result { + async fn read_at(&self, offset: u64, limit: usize) -> Result { oio::Read::read(&mut self.inner, limit).await } } impl BlockingRead for MockReader { - fn read(&mut self, limit: usize) -> Result { - self.inner.read(limit) + fn read_at(&self, offset: u64, limit: usize) -> Result { + self.inner.read_at(offset, limit) } fn seek(&mut self, _pos: SeekFrom) -> Result { diff --git a/core/src/raw/oio/read/file_read.rs b/core/src/raw/oio/read/file_read.rs index 64596c754a0b..b0b501edd9ce 100644 --- a/core/src/raw/oio/read/file_read.rs +++ b/core/src/raw/oio/read/file_read.rs @@ -210,7 +210,7 @@ where A: Accessor, R: oio::Read, { - async fn read(&mut self, limit: usize) -> Result { + async fn read_at(&self, offset: u64, limit: usize) -> Result { if self.reader.is_none() { // FileReader doesn't support range, we will always use full range to open a file. let op = self.op.clone().with_range(BytesRange::from(..)); @@ -267,7 +267,7 @@ where A: Accessor, R: oio::BlockingRead, { - fn read(&mut self, limit: usize) -> Result { + fn read_at(&self, offset: u64, limit: usize) -> Result { if self.reader.is_none() { // FileReader doesn't support range, we will always use full range to open a file. let op = self.op.clone().with_range(BytesRange::from(..)); diff --git a/core/src/raw/oio/read/futures_read.rs b/core/src/raw/oio/read/futures_read.rs index 7b1bd2e6aa8b..c964ad44b08c 100644 --- a/core/src/raw/oio/read/futures_read.rs +++ b/core/src/raw/oio/read/futures_read.rs @@ -55,7 +55,7 @@ where }) } - async fn read(&mut self, limit: usize) -> Result { + async fn read_at(&self, offset: u64, limit: usize) -> Result { // Make sure buf has enough space. if self.buf.capacity() < limit { self.buf.reserve(limit); diff --git a/core/src/raw/oio/read/into_read_from_stream.rs b/core/src/raw/oio/read/into_read_from_stream.rs index ca1c43e9b361..bab22d7ac958 100644 --- a/core/src/raw/oio/read/into_read_from_stream.rs +++ b/core/src/raw/oio/read/into_read_from_stream.rs @@ -50,7 +50,7 @@ where )) } - async fn read(&mut self, limit: usize) -> Result { + async fn read_at(&self, offset: u64, limit: usize) -> Result { if self.buf.is_empty() { self.buf = match self.inner.next().await.transpose()? { Some(v) => v.into(), diff --git a/core/src/raw/oio/read/into_streamable_read.rs b/core/src/raw/oio/read/into_streamable_read.rs index 6fe21a2c6077..e52627a358e4 100644 --- a/core/src/raw/oio/read/into_streamable_read.rs +++ b/core/src/raw/oio/read/into_streamable_read.rs @@ -43,7 +43,7 @@ impl oio::Read for StreamableReader { self.r.seek(pos).await } - async fn read(&mut self, limit: usize) -> Result { + async fn read_at(&self, offset: u64, limit: usize) -> Result { let size = min(self.buf.capacity(), limit); let dst = self.buf.spare_capacity_mut(); @@ -61,7 +61,7 @@ impl oio::Read for StreamableReader { } impl oio::BlockingRead for StreamableReader { - fn read(&mut self, limit: usize) -> Result { + fn read_at(&self, offset: u64, limit: usize) -> Result { self.r.read(limit) } diff --git a/core/src/raw/oio/read/lazy_read.rs b/core/src/raw/oio/read/lazy_read.rs index 1fd1c71b59ff..2a8cca3e9e87 100644 --- a/core/src/raw/oio/read/lazy_read.rs +++ b/core/src/raw/oio/read/lazy_read.rs @@ -73,7 +73,7 @@ where self.reader().await?.seek(pos).await } - async fn read(&mut self, limit: usize) -> Result { + async fn read_at(&self, offset: u64, limit: usize) -> Result { let r = self.reader().await?; r.read(limit).await } @@ -99,7 +99,7 @@ where A: Accessor, R: oio::BlockingRead, { - fn read(&mut self, limit: usize) -> Result { + fn read_at(&self, offset: u64, limit: usize) -> Result { self.blocking_reader()?.read(limit) } diff --git a/core/src/raw/oio/read/range_read.rs b/core/src/raw/oio/read/range_read.rs index 56289d534129..c69a3695afb9 100644 --- a/core/src/raw/oio/read/range_read.rs +++ b/core/src/raw/oio/read/range_read.rs @@ -252,7 +252,7 @@ where A: Accessor, R: oio::Read, { - async fn read(&mut self, limit: usize) -> Result { + async fn read_at(&self, offset: u64, limit: usize) -> Result { // Sanity check for normal cases. if self.cur >= self.size.unwrap_or(u64::MAX) { return Ok(Bytes::new()); @@ -336,7 +336,7 @@ where A: Accessor, R: oio::BlockingRead, { - fn read(&mut self, limit: usize) -> Result { + fn read_at(&self, offset: u64, limit: usize) -> Result { // Sanity check for normal cases. if self.cur >= self.size.unwrap_or(u64::MAX) { return Ok(Bytes::new()); @@ -496,7 +496,7 @@ mod tests { )) } - async fn read(&mut self, limit: usize) -> Result { + async fn read_at(&self, offset: u64, limit: usize) -> Result { let mut bs = vec![0; limit]; let n = self.inner.read(&mut bs).await.map_err(|err| { Error::new(ErrorKind::Unexpected, "read data from mock").set_source(err) diff --git a/core/src/raw/oio/read/std_read.rs b/core/src/raw/oio/read/std_read.rs index 8726ebf2499a..8d8655e44353 100644 --- a/core/src/raw/oio/read/std_read.rs +++ b/core/src/raw/oio/read/std_read.rs @@ -44,7 +44,7 @@ impl oio::BlockingRead for StdReader where R: Read + Seek + Send + Sync, { - fn read(&mut self, limit: usize) -> Result { + fn read_at(&self, offset: u64, limit: usize) -> Result { // Make sure buf has enough space. if self.buf.capacity() < limit { self.buf.reserve(limit); diff --git a/core/src/raw/oio/read/tokio_read.rs b/core/src/raw/oio/read/tokio_read.rs index f89e2ec5f35d..36ca350fbb51 100644 --- a/core/src/raw/oio/read/tokio_read.rs +++ b/core/src/raw/oio/read/tokio_read.rs @@ -47,7 +47,7 @@ impl oio::Read for TokioReader where R: AsyncRead + AsyncSeek + Unpin + Send + Sync, { - async fn read(&mut self, limit: usize) -> Result { + async fn read_at(&self, offset: u64, limit: usize) -> Result { // Make sure buf has enough space. if self.buf.capacity() < limit { self.buf.reserve(limit); diff --git a/core/src/types/reader.rs b/core/src/types/reader.rs index 4c5a1248af31..b677233043b2 100644 --- a/core/src/types/reader.rs +++ b/core/src/types/reader.rs @@ -414,8 +414,8 @@ impl BlockingReader { /// Read at most `size` bytes of data from reader. #[inline] - pub fn read(&mut self, limit: usize) -> Result { - self.inner.read(limit) + pub fn read_at(&self, offset: u64, limit: usize) -> Result { + self.inner.read_at(offset, limit) } /// Read exact `size` bytes of data from reader. From 708553d56972e8606519e18e3b33527724381a97 Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Tue, 19 Mar 2024 21:54:48 +0800 Subject: [PATCH 043/111] Fix build Signed-off-by: Xuanwo --- core/src/layers/dtrace.rs | 1 + core/src/raw/oio/read/mod.rs | 4 ++-- core/src/services/alluxio/error.rs | 5 +---- core/src/services/alluxio/mod.rs | 2 +- core/src/services/b2/error.rs | 5 +---- core/src/services/chainsafe/error.rs | 5 +---- core/src/services/github/error.rs | 5 +---- core/src/services/hdfs/mod.rs | 2 +- core/src/services/koofr/error.rs | 5 +---- core/src/services/pcloud/error.rs | 5 +---- core/src/services/seafile/error.rs | 5 +---- core/src/services/upyun/error.rs | 5 +---- core/src/services/vercel_blob/error.rs | 6 +----- core/src/services/yandex_disk/error.rs | 5 +---- 14 files changed, 15 insertions(+), 45 deletions(-) diff --git a/core/src/layers/dtrace.rs b/core/src/layers/dtrace.rs index 99203de8a612..d8ef35ca6375 100644 --- a/core/src/layers/dtrace.rs +++ b/core/src/layers/dtrace.rs @@ -22,6 +22,7 @@ use std::fmt::Formatter; use std::io; use async_trait::async_trait; +use bytes::Buf; use bytes::Bytes; use probe::probe_lazy; diff --git a/core/src/raw/oio/read/mod.rs b/core/src/raw/oio/read/mod.rs index f2673e251a4c..02ac30d8e57c 100644 --- a/core/src/raw/oio/read/mod.rs +++ b/core/src/raw/oio/read/mod.rs @@ -42,8 +42,8 @@ pub use api::Reader; // mod tokio_read; // pub use tokio_read::TokioReader; -mod std_read; -pub use std_read::StdReader; +// mod std_read; +// pub use std_read::StdReader; // mod lazy_read; // pub use lazy_read::LazyReader; diff --git a/core/src/services/alluxio/error.rs b/core/src/services/alluxio/error.rs index 1de357e80394..bce4de9a7624 100644 --- a/core/src/services/alluxio/error.rs +++ b/core/src/services/alluxio/error.rs @@ -91,10 +91,7 @@ mod tests { for res in err_res { let bs = bytes::Bytes::from(res.0); - let body = oio::Buffer::new( - Box::new(oio::into_stream(stream::iter(vec![Ok(bs.clone())]))), - None, - ); + let body = oio::Buffer::from(bs); let resp = Response::builder() .status(StatusCode::INTERNAL_SERVER_ERROR) .body(body) diff --git a/core/src/services/alluxio/mod.rs b/core/src/services/alluxio/mod.rs index 886a7807a3c2..226d0130353b 100644 --- a/core/src/services/alluxio/mod.rs +++ b/core/src/services/alluxio/mod.rs @@ -22,5 +22,5 @@ pub use backend::AlluxioConfig; mod core; mod error; mod lister; -mod writer; mod reader; +mod writer; diff --git a/core/src/services/b2/error.rs b/core/src/services/b2/error.rs index 209c6216c2aa..66bfe44e0907 100644 --- a/core/src/services/b2/error.rs +++ b/core/src/services/b2/error.rs @@ -124,10 +124,7 @@ mod test { for res in err_res { let bs = bytes::Bytes::from(res.0); - let body = oio::Buffer::new( - Box::new(oio::into_stream(stream::iter(vec![Ok(bs.clone())]))), - None, - ); + let body = oio::Buffer::from(bs); let resp = Response::builder().status(res.2).body(body).unwrap(); let err = parse_error(resp).await; diff --git a/core/src/services/chainsafe/error.rs b/core/src/services/chainsafe/error.rs index 1344212f1a85..dbc1ae61b44e 100644 --- a/core/src/services/chainsafe/error.rs +++ b/core/src/services/chainsafe/error.rs @@ -95,10 +95,7 @@ mod test { for res in err_res { let bs = bytes::Bytes::from(res.0); - let body = oio::Buffer::new( - Box::new(oio::into_stream(stream::iter(vec![Ok(bs.clone())]))), - None, - ); + let body = oio::Buffer::from(bs); let resp = Response::builder().status(res.2).body(body).unwrap(); let err = parse_error(resp).await; diff --git a/core/src/services/github/error.rs b/core/src/services/github/error.rs index e2dd87909655..df295c7d5ef8 100644 --- a/core/src/services/github/error.rs +++ b/core/src/services/github/error.rs @@ -93,10 +93,7 @@ mod test { for res in err_res { let bs = bytes::Bytes::from(res.0); - let body = oio::Buffer::new( - Box::new(oio::into_stream(stream::iter(vec![Ok(bs.clone())]))), - None, - ); + let body = oio::Buffer::from(bs); let resp = Response::builder().status(res.2).body(body).unwrap(); let err = parse_error(resp).await; diff --git a/core/src/services/hdfs/mod.rs b/core/src/services/hdfs/mod.rs index df55c9c3e01a..afa3c200a7c3 100644 --- a/core/src/services/hdfs/mod.rs +++ b/core/src/services/hdfs/mod.rs @@ -19,5 +19,5 @@ mod backend; pub use backend::HdfsBuilder as Hdfs; pub use backend::HdfsConfig; mod lister; +mod reader; mod writer; -mod r#reader diff --git a/core/src/services/koofr/error.rs b/core/src/services/koofr/error.rs index ec09b7640cee..22543e2518e8 100644 --- a/core/src/services/koofr/error.rs +++ b/core/src/services/koofr/error.rs @@ -65,10 +65,7 @@ mod test { for res in err_res { let bs = bytes::Bytes::from(res.0); - let body = oio::Buffer::new( - Box::new(oio::into_stream(stream::iter(vec![Ok(bs.clone())]))), - None, - ); + let body = oio::Buffer::from(bs); let resp = Response::builder().status(res.2).body(body).unwrap(); let err = parse_error(resp).await; diff --git a/core/src/services/pcloud/error.rs b/core/src/services/pcloud/error.rs index 3edc31f85306..03a1bf919cf9 100644 --- a/core/src/services/pcloud/error.rs +++ b/core/src/services/pcloud/error.rs @@ -81,10 +81,7 @@ mod test { for res in err_res { let bs = bytes::Bytes::from(res.0); - let body = oio::Buffer::new( - Box::new(oio::into_stream(stream::iter(vec![Ok(bs.clone())]))), - None, - ); + let body = oio::Buffer::from(bs); let resp = Response::builder().status(res.2).body(body).unwrap(); let err = parse_error(resp).await; diff --git a/core/src/services/seafile/error.rs b/core/src/services/seafile/error.rs index 949d083fdad7..f7982e841cee 100644 --- a/core/src/services/seafile/error.rs +++ b/core/src/services/seafile/error.rs @@ -79,10 +79,7 @@ mod test { for res in err_res { let bs = bytes::Bytes::from(res.0); - let body = oio::Buffer::new( - Box::new(oio::into_stream(stream::iter(vec![Ok(bs.clone())]))), - None, - ); + let body = oio::Buffer::from(bs); let resp = Response::builder().status(res.2).body(body).unwrap(); let err = parse_error(resp).await; diff --git a/core/src/services/upyun/error.rs b/core/src/services/upyun/error.rs index 2ecaadc2a4e1..30678db4e2f7 100644 --- a/core/src/services/upyun/error.rs +++ b/core/src/services/upyun/error.rs @@ -89,10 +89,7 @@ mod test { for res in err_res { let bs = bytes::Bytes::from(res.0); - let body = oio::Buffer::new( - Box::new(oio::into_stream(stream::iter(vec![Ok(bs.clone())]))), - None, - ); + let body = oio::Buffer::from(bs); let resp = Response::builder().status(res.2).body(body).unwrap(); let err = parse_error(resp).await; diff --git a/core/src/services/vercel_blob/error.rs b/core/src/services/vercel_blob/error.rs index d733071246f7..badc56714ef3 100644 --- a/core/src/services/vercel_blob/error.rs +++ b/core/src/services/vercel_blob/error.rs @@ -87,11 +87,7 @@ mod test { )]; for res in err_res { - let bs = bytes::Bytes::from(res.0); - let body = oio::Buffer::new( - Box::new(oio::into_stream(stream::iter(vec![Ok(bs.clone())]))), - None, - ); + let body = oio::Buffer::from(res.0.as_bytes().to_vec()); let resp = Response::builder().status(res.2).body(body).unwrap(); let err = parse_error(resp).await; diff --git a/core/src/services/yandex_disk/error.rs b/core/src/services/yandex_disk/error.rs index bd82eaed5e9f..492e81131645 100644 --- a/core/src/services/yandex_disk/error.rs +++ b/core/src/services/yandex_disk/error.rs @@ -97,10 +97,7 @@ mod test { for res in err_res { let bs = bytes::Bytes::from(res.0); - let body = oio::Buffer::new( - Box::new(oio::into_stream(stream::iter(vec![Ok(bs.clone())]))), - None, - ); + let body = oio::Buffer::from(bs); let resp = Response::builder().status(res.2).body(body).unwrap(); let err = parse_error(resp).await; From c6566f28d3873fe811526bb973b708c660d5864b Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Tue, 19 Mar 2024 21:55:54 +0800 Subject: [PATCH 044/111] Fix build Signed-off-by: Xuanwo --- core/src/services/webhdfs/error.rs | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/core/src/services/webhdfs/error.rs b/core/src/services/webhdfs/error.rs index 90d3aac74f3d..bf98093ab72c 100644 --- a/core/src/services/webhdfs/error.rs +++ b/core/src/services/webhdfs/error.rs @@ -100,10 +100,7 @@ mod tests { } "#, ); - let body = oio::Buffer::new( - Box::new(oio::into_stream(stream::iter(vec![Ok(ill_args.clone())]))), - None, - ); + let body = oio::Buffer::from(ill_args.clone()); let resp = Response::builder() .status(StatusCode::BAD_REQUEST) .body(body) From 4ca19ef08f803aaa5658c55786bbdb5ddbd1ba7a Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Wed, 20 Mar 2024 16:58:33 +0800 Subject: [PATCH 045/111] Fix build Signed-off-by: Xuanwo --- core/src/layers/blocking.rs | 2 +- core/src/layers/complete.rs | 2 +- core/src/layers/metrics.rs | 4 +- core/src/layers/prometheus_client.rs | 16 +- core/src/layers/retry.rs | 2 +- core/src/layers/timeout.rs | 2 +- core/src/raw/adapters/kv/backend.rs | 4 +- core/src/raw/adapters/typed_kv/backend.rs | 4 +- core/src/raw/enum_utils.rs | 2 +- core/src/raw/http_util/multipart.rs | 13 +- core/src/raw/oio/buf/buffer.rs | 26 +- core/src/raw/oio/cursor.rs | 151 ---------- core/src/raw/oio/mod.rs | 3 - core/src/raw/oio/read/api.rs | 12 + core/src/raw/oio/stream/api.rs | 18 +- core/src/raw/tests/read.rs | 104 ++----- core/src/services/fs/backend.rs | 7 +- core/src/services/hdfs/backend.rs | 7 +- core/src/services/webhdfs/error.rs | 2 +- core/src/types/operator/blocking_operator.rs | 2 +- core/src/types/reader.rs | 280 ++++++++----------- core/tests/behavior/async_fuzz.rs | 97 +------ core/tests/behavior/async_read.rs | 23 -- 23 files changed, 227 insertions(+), 556 deletions(-) delete mode 100644 core/src/raw/oio/cursor.rs diff --git a/core/src/layers/blocking.rs b/core/src/layers/blocking.rs index d48bda0c1893..93c8d0ed4882 100644 --- a/core/src/layers/blocking.rs +++ b/core/src/layers/blocking.rs @@ -288,7 +288,7 @@ impl BlockingWrapper { } impl oio::BlockingRead for BlockingWrapper { - fn read_at(&mut self, offset: u64, limit: usize) -> Result { + fn read_at(&self, offset: u64, limit: usize) -> Result { self.handle.block_on(self.inner.read_at(offset, limit)) } } diff --git a/core/src/layers/complete.rs b/core/src/layers/complete.rs index 6bc8fdfeb963..2271e91e78c1 100644 --- a/core/src/layers/complete.rs +++ b/core/src/layers/complete.rs @@ -803,7 +803,7 @@ mod tests { } async fn read(&self, _: &str, _: OpRead) -> Result<(RpRead, Self::Reader)> { - Ok((RpRead::new(), Box::new(oio::Cursor::new()))) + Ok((RpRead::new(), Box::new(Bytes::new()))) } async fn write(&self, _: &str, _: OpWrite) -> Result<(RpWrite, Self::Writer)> { diff --git a/core/src/layers/metrics.rs b/core/src/layers/metrics.rs index d2fe64d70082..6b296e671560 100644 --- a/core/src/layers/metrics.rs +++ b/core/src/layers/metrics.rs @@ -787,7 +787,7 @@ impl oio::Write for MetricWrapper { self.inner .write(bs) .map_ok(|n| { - self.bytes_counter.increment(bs.remaining() as u64); + self.bytes_counter.increment(n as u64); n }) .map_err(|err| { @@ -816,7 +816,7 @@ impl oio::BlockingWrite for MetricWrapper { self.inner .write(bs) .map(|n| { - self.bytes_counter.increment(bs.remaining() as u64); + self.bytes_counter.increment(n as u64); n }) .map_err(|err| { diff --git a/core/src/layers/prometheus_client.rs b/core/src/layers/prometheus_client.rs index cd813f6cf146..693d386f4ac8 100644 --- a/core/src/layers/prometheus_client.rs +++ b/core/src/layers/prometheus_client.rs @@ -580,14 +580,14 @@ impl oio::BlockingRead for PrometheusMetricWrapper { } impl oio::Write for PrometheusMetricWrapper { - fn write(&mut self, bs: Bytes) -> impl Future> + Send { + async fn write(&mut self, bs: Bytes) -> Result { let start = Instant::now(); self.inner .write(bs) - .map_ok(|n| { - self.metrics - .observe_bytes_total(self.scheme, self.op, bs.remaining()); + .await + .map(|n| { + self.metrics.observe_bytes_total(self.scheme, self.op, n); self.metrics .observe_request_duration(self.scheme, self.op, start.elapsed()); n @@ -599,16 +599,16 @@ impl oio::Write for PrometheusMetricWrapper { }) } - fn abort(&mut self) -> impl Future> + Send { - self.inner.abort().map_err(|err| { + async fn abort(&mut self) -> Result<()> { + self.inner.abort().await.map_err(|err| { self.metrics .increment_errors_total(self.scheme, self.op, err.kind()); err }) } - fn close(&mut self) -> impl Future> + Send { - self.inner.close().map_err(|err| { + async fn close(&mut self) -> Result<()> { + self.inner.close().await.map_err(|err| { self.metrics .increment_errors_total(self.scheme, self.op, err.kind()); err diff --git a/core/src/layers/retry.rs b/core/src/layers/retry.rs index b266c525b20f..9ceea0bec931 100644 --- a/core/src/layers/retry.rs +++ b/core/src/layers/retry.rs @@ -1119,7 +1119,7 @@ mod tests { let mut r = op.reader("retryable_error").await.unwrap(); let mut content = Vec::new(); let size = r - .read_to_end(&mut content) + .read_to_end_at(&mut content, 0) .await .expect("read must succeed"); assert_eq!(size, 13); diff --git a/core/src/layers/timeout.rs b/core/src/layers/timeout.rs index 98bea06bf582..5b3de5aef53d 100644 --- a/core/src/layers/timeout.rs +++ b/core/src/layers/timeout.rs @@ -429,7 +429,7 @@ mod tests { let mut reader = op.reader("test").await.unwrap(); - let res = reader.read(4).await; + let res = reader.read_at(&mut Vec::with_capacity(4), 0).await; assert!(res.is_err()); let err = res.unwrap_err(); assert_eq!(err.kind(), ErrorKind::Unexpected); diff --git a/core/src/raw/adapters/kv/backend.rs b/core/src/raw/adapters/kv/backend.rs index cfec881521f1..1741e102698a 100644 --- a/core/src/raw/adapters/kv/backend.rs +++ b/core/src/raw/adapters/kv/backend.rs @@ -64,7 +64,7 @@ where #[cfg_attr(target_arch = "wasm32", async_trait(?Send))] impl Accessor for Backend { type Reader = Bytes; - type BlockingReader = oio::Cursor; + type BlockingReader = Bytes; type Writer = KvWriter; type BlockingWriter = KvWriter; type Lister = HierarchyLister; @@ -118,7 +118,7 @@ impl Accessor for Backend { }; let bs = self.apply_range(bs, args.range()); - Ok((RpRead::new(), oio::Cursor::from(bs))) + Ok((RpRead::new(), Bytes::from(bs))) } async fn write(&self, path: &str, _: OpWrite) -> Result<(RpWrite, Self::Writer)> { diff --git a/core/src/raw/adapters/typed_kv/backend.rs b/core/src/raw/adapters/typed_kv/backend.rs index 504a33493688..03c79cea9136 100644 --- a/core/src/raw/adapters/typed_kv/backend.rs +++ b/core/src/raw/adapters/typed_kv/backend.rs @@ -58,7 +58,7 @@ where #[cfg_attr(target_arch = "wasm32", async_trait(?Send))] impl Accessor for Backend { type Reader = Bytes; - type BlockingReader = oio::Cursor; + type BlockingReader = Bytes; type Writer = KvWriter; type BlockingWriter = KvWriter; type Lister = HierarchyLister; @@ -126,7 +126,7 @@ impl Accessor for Backend { }; let bs = self.apply_range(bs, args.range()); - Ok((RpRead::new(), oio::Cursor::from(bs))) + Ok((RpRead::new(), bs)) } async fn write(&self, path: &str, args: OpWrite) -> Result<(RpWrite, Self::Writer)> { diff --git a/core/src/raw/enum_utils.rs b/core/src/raw/enum_utils.rs index 3fe83e82925a..99bedb1ac516 100644 --- a/core/src/raw/enum_utils.rs +++ b/core/src/raw/enum_utils.rs @@ -198,7 +198,7 @@ where THREE: oio::BlockingRead, FOUR: oio::BlockingRead, { - fn read_at(&mut self, offset: u64, limit: usize) -> Result { + fn read_at(&self, offset: u64, limit: usize) -> Result { match self { Self::One(v) => v.read_at(offset, limit), Self::Two(v) => v.read_at(offset, limit), diff --git a/core/src/raw/http_util/multipart.rs b/core/src/raw/http_util/multipart.rs index 30f2fc8f74d8..27f80f2c65dd 100644 --- a/core/src/raw/http_util/multipart.rs +++ b/core/src/raw/http_util/multipart.rs @@ -245,7 +245,7 @@ impl FormDataPart { Self { headers, content_length: 0, - content: Box::new(oio::Cursor::new()), + content: Box::new(Bytes::new()), } } @@ -260,7 +260,7 @@ impl FormDataPart { let content = content.into(); self.content_length = content.len() as u64; - self.content = Box::new(oio::Cursor::from(content)); + self.content = Box::new(content); self } @@ -399,10 +399,7 @@ impl MixedPart { let (content_length, content) = match body { AsyncBody::Empty => (0, None), - AsyncBody::Bytes(bs) => ( - bs.len() as u64, - Some(Box::new(oio::Cursor::from(bs)) as Streamer), - ), + AsyncBody::Bytes(bs) => (bs.len() as u64, Some(Box::new(bs) as Streamer)), AsyncBody::Stream(stream) => { let len = parts .headers @@ -486,7 +483,7 @@ impl MixedPart { let content = content.into(); self.content_length = content.len() as u64; - self.content = Some(Box::new(oio::Cursor::from(content))); + self.content = Some(Box::new(content)); self } @@ -651,7 +648,7 @@ impl Part for MixedPart { version: Version::HTTP_11, headers, content_length: body_bytes.len() as u64, - content: Some(Box::new(oio::Cursor::from(body_bytes))), + content: Some(Box::new(body_bytes)), method: None, uri: None, diff --git a/core/src/raw/oio/buf/buffer.rs b/core/src/raw/oio/buf/buffer.rs index 2fa7d960504d..40986766f8a0 100644 --- a/core/src/raw/oio/buf/buffer.rs +++ b/core/src/raw/oio/buf/buffer.rs @@ -15,6 +15,7 @@ // specific language governing permissions and limitations // under the License. +use crate::raw::oio; use bytes::{Buf, BufMut, Bytes, BytesMut}; use std::collections::VecDeque; @@ -30,7 +31,7 @@ enum Inner { impl Buffer { #[inline] pub const fn new() -> Self { - Self(Inner::Contiguous(Bytes::new())) + Self(Inner::NonContiguous(VecDeque::new())) } #[inline] @@ -38,6 +39,29 @@ impl Buffer { let mut bs = self.clone(); bs.copy_to_bytes(bs.remaining()) } + + /// Merge two buffer together without copying internal bytes. + pub fn merge(mut self, buf: Buffer) -> Self { + let mut vec = match self.0 { + Inner::Contiguous(b) => { + // NOTE: we will have at least two bytes in the vec. + let mut vec = VecDeque::with_capacity(2); + vec.push_back(b); + vec + } + Inner::NonContiguous(v) => v, + }; + + match buf.0 { + Inner::Contiguous(b) => vec.push_back(b), + Inner::NonContiguous(bs) => { + vec.reserve(bs.len()); + vec.extend(bs) + } + } + + Self(Inner::NonContiguous(vec)) + } } impl From> for Buffer { diff --git a/core/src/raw/oio/cursor.rs b/core/src/raw/oio/cursor.rs deleted file mode 100644 index b875d7b83b9f..000000000000 --- a/core/src/raw/oio/cursor.rs +++ /dev/null @@ -1,151 +0,0 @@ -// 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::cmp::min; -use std::io::SeekFrom; -use std::task::Context; -use std::task::Poll; - -use bytes::Bytes; - -use crate::raw::*; -use crate::*; - -/// Cursor is the cursor for [`Bytes`] that implements [`oio::Read`] -#[derive(Default)] -pub struct Cursor { - inner: Bytes, - pos: u64, -} - -impl Cursor { - /// Create a new empty cursor. - pub fn new() -> Self { - Self::default() - } - - /// Returns `true` if the remaining slice is empty. - pub fn is_empty(&self) -> bool { - self.pos as usize >= self.inner.len() - } - - /// Returns the remaining slice. - pub fn remaining_slice(&self) -> &[u8] { - let len = self.pos.min(self.inner.len() as u64) as usize; - &self.inner.as_ref()[len..] - } - - /// Return the length of remaining slice. - pub fn len(&self) -> usize { - self.inner.len() - self.pos as usize - } -} - -impl From for Cursor { - fn from(v: Bytes) -> Self { - Cursor { inner: v, pos: 0 } - } -} - -impl From> for Cursor { - fn from(v: Vec) -> Self { - Cursor { - inner: Bytes::from(v), - pos: 0, - } - } -} - -// impl oio::Read for Cursor { -// async fn read(&self,offset:u64, limit: usize) -> Result { -// if self.is_empty() { -// Ok(Bytes::new()) -// } else { -// // The clone here is required as we don't want to change it. -// let mut bs = self.inner.clone().split_off(self.pos as usize); -// let bs = bs.split_to(min(bs.len(), limit)); -// self.pos += bs.len() as u64; -// Ok(bs) -// } -// } -// -// async fn seek(&mut self, pos: SeekFrom) -> Result { -// let (base, amt) = match pos { -// SeekFrom::Start(n) => (0, n as i64), -// SeekFrom::End(n) => (self.inner.len() as i64, n), -// SeekFrom::Current(n) => (self.pos as i64, n), -// }; -// -// let n = match base.checked_add(amt) { -// Some(n) if n >= 0 => n as u64, -// _ => { -// return Err(Error::new( -// ErrorKind::InvalidInput, -// "invalid seek to a negative or overflowing position", -// )) -// } -// }; -// self.pos = n; -// Ok(n) -// } -// } - -impl oio::BlockingRead for Cursor { - fn read_at(&self, offset: u64, limit: usize) -> Result { - if self.is_empty() { - Ok(Bytes::new()) - } else { - // The clone here is required as we don't want to change it. - let mut bs = self.inner.clone().split_off(self.pos as usize); - let bs = bs.split_to(min(bs.len(), limit)); - self.pos += bs.len() as u64; - Ok(bs) - } - } - - fn seek(&mut self, pos: SeekFrom) -> Result { - let (base, amt) = match pos { - SeekFrom::Start(n) => (0, n as i64), - SeekFrom::End(n) => (self.inner.len() as i64, n), - SeekFrom::Current(n) => (self.pos as i64, n), - }; - - let n = match base.checked_add(amt) { - Some(n) if n >= 0 => n as u64, - _ => { - return Err(Error::new( - ErrorKind::InvalidInput, - "invalid seek to a negative or overflowing position", - )) - } - }; - self.pos = n; - Ok(n) - } -} - -impl oio::Stream for Cursor { - fn poll_next(&mut self, _: &mut Context<'_>) -> Poll>> { - if self.is_empty() { - return Poll::Ready(None); - } - - let bs = self.inner.clone(); - self.pos += bs.len() as u64; - Poll::Ready(Some(Ok(bs))) - } -} diff --git a/core/src/raw/oio/mod.rs b/core/src/raw/oio/mod.rs index 7a10a8331c33..11e363f80895 100644 --- a/core/src/raw/oio/mod.rs +++ b/core/src/raw/oio/mod.rs @@ -34,9 +34,6 @@ pub use stream::*; mod list; pub use list::*; -mod cursor; -pub use cursor::Cursor; - mod entry; pub use entry::Entry; diff --git a/core/src/raw/oio/read/api.rs b/core/src/raw/oio/read/api.rs index 4337fab5523a..df31650a0b7e 100644 --- a/core/src/raw/oio/read/api.rs +++ b/core/src/raw/oio/read/api.rs @@ -165,6 +165,18 @@ impl BlockingRead for () { } } +impl BlockingRead for Bytes { + /// TODO: we can check if the offset is out of range. + fn read_at(&self, offset: u64, limit: usize) -> Result { + if offset >= self.len() as u64 { + return Ok(oio::Buffer::new()); + } + let offset = offset as usize; + let limit = limit.min(self.len() - offset); + Ok(oio::Buffer::from(self.slice(offset..offset + limit))) + } +} + /// `Box` won't implement `BlockingRead` automatically. /// To make BlockingReader work as expected, we must add this impl. impl BlockingRead for Box { diff --git a/core/src/raw/oio/stream/api.rs b/core/src/raw/oio/stream/api.rs index 7a672d2ff2a0..dc53682fd370 100644 --- a/core/src/raw/oio/stream/api.rs +++ b/core/src/raw/oio/stream/api.rs @@ -21,8 +21,8 @@ use std::task::ready; use std::task::Context; use std::task::Poll; -use bytes::Bytes; use bytes::BytesMut; +use bytes::{Buf, Bytes}; use crate::*; @@ -46,6 +46,16 @@ impl Stream for () { } } +impl Stream for Bytes { + fn poll_next(&mut self, cx: &mut Context<'_>) -> Poll>> { + if self.has_remaining() { + Poll::Ready(Some(Ok(self.copy_to_bytes(self.remaining())))) + } else { + Poll::Ready(None) + } + } +} + /// `Box` won't implement `Stream` automatically. /// To make Streamer work as expected, we must add this impl. impl Stream for Box { @@ -54,12 +64,6 @@ impl Stream for Box { } } -// impl Stream for T { -// fn poll_next(&mut self, cx: &mut Context<'_>) -> Poll>> { -// raw::oio::Read::poll_next(self, cx) -// } -// } - impl futures::Stream for dyn Stream { type Item = Result; diff --git a/core/src/raw/tests/read.rs b/core/src/raw/tests/read.rs index b2503901a32b..4a2743905ab5 100644 --- a/core/src/raw/tests/read.rs +++ b/core/src/raw/tests/read.rs @@ -15,9 +15,7 @@ // specific language governing permissions and limitations // under the License. -use std::io::SeekFrom; - -use bytes::Bytes; +use bytes::{Bytes, BytesMut}; use rand::thread_rng; use rand::RngCore; use sha2::Digest; @@ -27,20 +25,14 @@ use crate::raw::*; use crate::*; /// ReadAction represents a read action. -#[derive(Debug, Clone, Eq, PartialEq)] +#[derive(Debug, Clone, Copy, Eq, PartialEq)] pub enum ReadAction { /// Read represents a read action with given input buf size. /// /// # NOTE /// /// The size is the input buf size, it's possible that the actual read size is smaller. - Read(usize), - /// Seek represents a seek action with given seek position. - /// - /// # NOTE - /// - /// It's valid that seek outside of the file's end. - Seek(SeekFrom), + Read(usize, usize), } /// ReadChecker is used to check the correctness of the read process. @@ -49,8 +41,6 @@ pub struct ReadChecker { raw_data: Bytes, /// Ranged Data is the data that we read from the storage. ranged_data: Bytes, - /// Cur is the current position of the read process. - cur: usize, } impl ReadChecker { @@ -69,8 +59,6 @@ impl ReadChecker { Self { raw_data, ranged_data, - - cur: 0, } } @@ -83,8 +71,8 @@ impl ReadChecker { /// /// - buf_size is the read action's buf size. /// - output is the output of this read action. - fn check_read(&mut self, input: usize, output: &[u8]) { - if input == 0 { + fn check_read(&self, offset: usize, size: usize, output: &[u8]) { + if size == 0 { assert_eq!( output.len(), 0, @@ -93,20 +81,20 @@ impl ReadChecker { return; } - if input > 0 && output.is_empty() { + if size > 0 && output.is_empty() { assert!( - self.cur >= self.ranged_data.len(), + offset >= self.ranged_data.len(), "check read failed: no data read means cur must outsides of ranged_data", ); return; } assert!( - self.cur + output.len() <= self.ranged_data.len(), - "check read failed: cur + output length must be less than ranged_data length, cur: {}, output: {}, ranged_data: {}", self.cur, output.len(), self.ranged_data.len(), + offset + output.len() <= self.ranged_data.len(), + "check read failed: cur + output length must be less than ranged_data length, offset: {}, output: {}, ranged_data: {}", offset, output.len(), self.ranged_data.len(), ); - let expected = &self.ranged_data[self.cur..self.cur + output.len()]; + let expected = &self.ranged_data[offset..offset + output.len()]; // Check the read result assert_eq!( @@ -114,59 +102,21 @@ impl ReadChecker { format!("{:x}", Sha256::digest(expected)), "check read failed: output bs is different with expected bs", ); - - // Update the current position - self.cur += output.len(); - } - - /// check_seek checks the correctness of the read process after a seek action. - /// - /// - input is the `SeekFrom` passed by SeekAction. - /// - output ts the result after the seek operation. - fn check_seek(&mut self, input: SeekFrom, output: Result) { - let expected = match input { - SeekFrom::Start(offset) => offset as i64, - SeekFrom::End(offset) => self.ranged_data.len() as i64 + offset, - SeekFrom::Current(offset) => self.cur as i64 + offset, - }; - - if expected < 0 { - let Err(err) = output else { - panic!("check seek failed: seek should fail with negative offset"); - }; - - assert_eq!( - err.kind(), - ErrorKind::InvalidInput, - "check seek failed: seek should fail with error InvalidInput with negative offset" - ); - return; - } - - assert_eq!( - output.unwrap(), - expected as u64, - "check seek failed: seek result is different with expected result", - ); - - // only update the current position when seek succeed - self.cur = expected as usize; } /// Check will check the correctness of the read process via given actions. /// /// Check will panic if any check failed. - pub async fn check(&mut self, mut r: Reader, actions: &[ReadAction]) { + pub async fn check(&mut self, r: Reader, actions: &[ReadAction]) { for action in actions { - match action { - ReadAction::Read(size) => { - let bs = r.read(*size).await.expect("read must success"); - self.check_read(*size, &bs); - } - - ReadAction::Seek(pos) => { - let res = r.seek(*pos).await; - self.check_seek(*pos, res); + match *action { + ReadAction::Read(offset, size) => { + let mut bs = Vec::with_capacity(size); + let n = r + .read_at(&mut bs, offset as u64) + .await + .expect("read must success"); + self.check_read(offset, size, &bs[..n]); } } } @@ -177,15 +127,13 @@ impl ReadChecker { /// Check will panic if any check failed. pub fn blocking_check(&mut self, mut r: BlockingReader, actions: &[ReadAction]) { for action in actions { - match action { - ReadAction::Read(size) => { - let bs = r.read(*size).expect("read must success"); - self.check_read(*size, &bs); - } - - ReadAction::Seek(pos) => { - let res = r.seek(*pos); - self.check_seek(*pos, res); + match *action { + ReadAction::Read(offset, size) => { + let mut bs = Vec::with_capacity(size); + let n = r + .read_at(&mut bs, offset as u64) + .expect("read must success"); + self.check_read(offset, size, &bs[..n]); } } } diff --git a/core/src/services/fs/backend.rs b/core/src/services/fs/backend.rs index 96dd7b666329..9f78976b58ac 100644 --- a/core/src/services/fs/backend.rs +++ b/core/src/services/fs/backend.rs @@ -20,6 +20,7 @@ use std::path::Path; use std::path::PathBuf; use async_trait::async_trait; +use bytes::Bytes; use chrono::DateTime; use log::debug; use uuid::Uuid; @@ -244,7 +245,7 @@ impl Accessor for FsBackend { type Reader = FsReader; type Writer = FsWriter; type Lister = Option>; - type BlockingReader = oio::StdReader; + type BlockingReader = Bytes; type BlockingWriter = FsWriter; type BlockingLister = Option>; @@ -476,9 +477,7 @@ impl Accessor for FsBackend { .open(p) .map_err(new_std_io_error)?; - let r = oio::StdReader::new(f); - - Ok((RpRead::new(), r)) + todo!() } fn blocking_write(&self, path: &str, op: OpWrite) -> Result<(RpWrite, Self::BlockingWriter)> { diff --git a/core/src/services/hdfs/backend.rs b/core/src/services/hdfs/backend.rs index 9d94f048bfef..31edfb299496 100644 --- a/core/src/services/hdfs/backend.rs +++ b/core/src/services/hdfs/backend.rs @@ -23,6 +23,7 @@ use std::path::PathBuf; use std::sync::Arc; use async_trait::async_trait; +use bytes::Bytes; use futures::AsyncWriteExt; use log::debug; use serde::Deserialize; @@ -249,7 +250,7 @@ impl Accessor for HdfsBackend { type Reader = HdfsReader; type Writer = HdfsWriter; type Lister = Option; - type BlockingReader = oio::StdReader; + type BlockingReader = Bytes; type BlockingWriter = HdfsWriter; type BlockingLister = Option; @@ -507,9 +508,7 @@ impl Accessor for HdfsBackend { .open(&p) .map_err(new_std_io_error)?; - let r = oio::StdReader::new(f); - - Ok((RpRead::new(), r)) + todo!() } fn blocking_write(&self, path: &str, op: OpWrite) -> Result<(RpWrite, Self::BlockingWriter)> { diff --git a/core/src/services/webhdfs/error.rs b/core/src/services/webhdfs/error.rs index bf98093ab72c..a2bbca026562 100644 --- a/core/src/services/webhdfs/error.rs +++ b/core/src/services/webhdfs/error.rs @@ -96,7 +96,7 @@ mod tests { "exception" : "IllegalArgumentException", "javaClassName": "java.lang.IllegalArgumentException", "message" : "Invalid value for webhdfs parameter \"permission\": ..." - } + } } "#, ); diff --git a/core/src/types/operator/blocking_operator.rs b/core/src/types/operator/blocking_operator.rs index 06576986d3ee..1afbd3a11ba8 100644 --- a/core/src/types/operator/blocking_operator.rs +++ b/core/src/types/operator/blocking_operator.rs @@ -410,7 +410,7 @@ impl BlockingOperator { let (_, r) = inner.blocking_read(&path, args.with_range(range))?; let mut r = BlockingReader::new(r); let mut buf = Vec::with_capacity(size_hint as usize); - r.read_to_end(&mut buf)?; + r.read_to_end_at(&mut buf, 0)?; Ok(buf) }, diff --git a/core/src/types/reader.rs b/core/src/types/reader.rs index b677233043b2..8b94af8f581b 100644 --- a/core/src/types/reader.rs +++ b/core/src/types/reader.rs @@ -120,67 +120,63 @@ impl Reader { // todo!() // } - /// Read at most `size` bytes of data from reader. - #[inline] - pub async fn read_at(&self, offset: u64, limit: usize) -> Result { - self.inner.read_at_dyn(offset, limit).await + pub async fn read_at(&self, buf: &mut impl BufMut, offset: u64) -> Result { + let bs = self.inner.read_at_dyn(offset, buf.remaining_mut()).await?; + let n = bs.remaining(); + buf.put(bs); + Ok(n) } - /// Read exact `size` bytes of data from reader. - pub async fn read_exact_at(&self, offset: u64, size: usize) -> Result { - let mut buf = BytesMut::with_capacity(size); - let mut remaining = size; - let mut cur = offset; + /// Read given range bytes of data from reader. + pub async fn read_range(&self, buf: &mut impl BufMut, range: Range) -> Result { + if range.is_empty() { + return Ok(0); + } + let (mut offset, mut size) = (range.start, range.end - range.start); + + let mut read = 0; loop { - let bs = self.inner.read_at_dyn(cur, remaining).await?; + let bs = self.inner.read_at_dyn(offset, size as usize).await?; let n = bs.remaining(); + read += n; buf.put(bs); - cur += n as u64; - remaining -= n; - if remaining == 0 { - break; + if n == 0 { + return Ok(read); + } + + offset += n as u64; + + debug_assert!( + size >= n as u64, + "read should not return more bytes than expected" + ); + size -= n as u64; + if size == 0 { + return Ok(read); } } - Ok(buf.freeze()) } - pub async fn read_range(&self, range: Range) -> Result { - todo!() + pub async fn read_to_end(&self, buf: &mut impl BufMut) -> Result { + self.read_to_end_at(buf, 0).await } - /// Reads all bytes until EOF in this source, placing them into buf. - pub async fn read_to_end_at(&self, offset: u64, buf: &mut Vec) -> Result { - todo!() - // let start_len = buf.len(); - // - // loop { - // if buf.len() == buf.capacity() { - // buf.reserve(32); // buf is full, need more space - // } - // - // let spare = buf.spare_capacity_mut(); - // let mut read_buf: ReadBuf = ReadBuf::uninit(spare); - // - // // SAFETY: These bytes were initialized but not filled in the previous loop - // unsafe { - // read_buf.assume_init(read_buf.capacity()); - // } - // - // match self.read(read_buf.initialize_unfilled().len()).await { - // Ok(bs) if bs.is_empty() => { - // return Ok(buf.len() - start_len); - // } - // Ok(bs) => { - // read_buf.initialize_unfilled()[..bs.len()].copy_from_slice(&bs); - // // SAFETY: Read API makes sure that returning `n` is correct. - // unsafe { - // buf.set_len(buf.len() + bs.len()); - // } - // } - // Err(e) => return Err(e), - // } - // } + pub async fn read_to_end_at(&self, buf: &mut impl BufMut, mut offset: u64) -> Result { + let mut size = 0; + loop { + // TODO: io size should be tuned based on storage + let bs = self.inner.read_at_dyn(offset, 4 * 1024 * 1024).await?; + let n = bs.remaining(); + size += n; + + buf.put(bs); + if n == 0 { + return Ok(size); + } + + offset += n as u64; + } } } @@ -224,7 +220,7 @@ unsafe impl Sync for State {} // } // State::Seeking(_) => Poll::Ready(Err(io::Error::new( // io::ErrorKind::Interrupted, -// "another io operation is in progress", +// "another io operation is in progress", // ))), // } // } @@ -406,129 +402,99 @@ impl BlockingReader { BlockingReader { inner: r } } - /// Seek to the position of `pos` of reader. - #[inline] - pub fn seek(&mut self, pos: SeekFrom) -> Result { - self.inner.seek(pos) - } - - /// Read at most `size` bytes of data from reader. - #[inline] - pub fn read_at(&self, offset: u64, limit: usize) -> Result { - self.inner.read_at(offset, limit) + /// Read given range bytes of data from reader. + pub fn read_at(&self, buf: &mut impl BufMut, offset: u64) -> Result { + let bs = self.inner.read_at(offset, buf.remaining_mut())?; + let n = bs.remaining(); + buf.put(bs); + Ok(n) } - /// Read exact `size` bytes of data from reader. - pub fn read_exact(&mut self, size: usize) -> Result { - // Lucky path. - let bs1 = self.inner.read(size)?; - debug_assert!( - bs1.len() <= size, - "read should not return more bytes than expected" - ); - if bs1.len() == size { - return Ok(bs1); - } - if bs1.is_empty() { - return Err( - Error::new(ErrorKind::ContentIncomplete, "reader got too little data") - .with_context("expect", size.to_string()), - ); + /// Read given range bytes of data from reader. + pub fn read_range(&self, buf: &mut impl BufMut, range: Range) -> Result { + if range.is_empty() { + return Ok(0); } + let (mut offset, mut size) = (range.start, range.end - range.start); - let mut bs = BytesMut::with_capacity(size); - bs.put_slice(&bs1); - - let mut remaining = size - bs.len(); + let mut read = 0; loop { - let tmp = self.inner.read(remaining)?; - if tmp.is_empty() { - return Err( - Error::new(ErrorKind::ContentIncomplete, "reader got too little data") - .with_context("expect", size.to_string()) - .with_context("actual", bs.len().to_string()), - ); + let bs = self.inner.read_at(offset, size as usize)?; + let n = bs.remaining(); + read += n; + buf.put(bs); + if n == 0 { + return Ok(read); } - bs.put_slice(&tmp); + + offset += n as u64; + debug_assert!( - tmp.len() <= remaining, + size >= n as u64, "read should not return more bytes than expected" ); - - remaining -= tmp.len(); - if remaining == 0 { - break; + size -= n as u64; + if size == 0 { + return Ok(read); } } + } - Ok(bs.freeze()) + pub fn read_to_end(&self, buf: &mut impl BufMut) -> Result { + self.read_to_end_at(buf, 0) } - /// Reads all bytes until EOF in this source, placing them into buf. - pub fn read_to_end(&mut self, buf: &mut Vec) -> Result { - let start_len = buf.len(); + pub fn read_to_end_at(&self, buf: &mut impl BufMut, mut offset: u64) -> Result { + let mut size = 0; loop { - if buf.len() == buf.capacity() { - buf.reserve(32); // buf is full, need more space - } - - let spare = buf.spare_capacity_mut(); - let mut read_buf: ReadBuf = ReadBuf::uninit(spare); + // TODO: io size should be tuned based on storage + let bs = self.inner.read_at(offset, 4 * 1024 * 1024)?; + let n = bs.remaining(); + size += n; - // SAFETY: These bytes were initialized but not filled in the previous loop - unsafe { - read_buf.assume_init(read_buf.capacity()); + buf.put(bs); + if n == 0 { + return Ok(size); } - match self.read(read_buf.initialized_mut().len()) { - Ok(bs) if bs.is_empty() => return Ok(buf.len() - start_len), - Ok(bs) => { - read_buf.initialized_mut()[..bs.len()].copy_from_slice(&bs); - - // SAFETY: Read API makes sure that returning `n` is correct. - unsafe { - buf.set_len(buf.len() + bs.len()); - } - } - Err(e) => return Err(e), - } + offset += n as u64; } } } -impl io::Read for BlockingReader { - #[inline] - fn read(&mut self, buf: &mut [u8]) -> io::Result { - let bs = self.inner.read(buf.len()).map_err(format_std_io_error)?; - buf[..bs.len()].copy_from_slice(&bs); - Ok(bs.len()) - } -} - -impl io::Seek for BlockingReader { - #[inline] - fn seek(&mut self, pos: io::SeekFrom) -> io::Result { - self.inner.seek(pos).map_err(format_std_io_error) - } -} - -impl Iterator for BlockingReader { - type Item = io::Result; - - #[inline] - fn next(&mut self) -> Option { - match self - .inner - .read(4 * 1024 * 1024) - .map_err(format_std_io_error) - { - Ok(bs) if bs.is_empty() => None, - Ok(bs) => Some(Ok(bs)), - Err(err) => Some(Err(err)), - } - } -} +// impl io::Read for BlockingReader { +// #[inline] +// fn read(&mut self, buf: &mut [u8]) -> io::Result { +// let bs = self.inner.read(buf.len()).map_err(format_std_io_error)?; +// buf[..bs.len()].copy_from_slice(&bs); +// Ok(bs.len()) +// } +// } +// +// impl io::Seek for BlockingReader { +// #[inline] +// fn seek(&mut self, pos: io::SeekFrom) -> io::Result { +// self.inner.seek(pos).map_err(format_std_io_error) +// } +// } +// +// impl Iterator for BlockingReader { +// type Item = io::Result; +// +// #[inline] +// fn next(&mut self) -> Option { +// match self +// .inner +// .read(4 * 1024 * 1024) +// .map_err(format_std_io_error) +// { +// Ok(bs) if bs.is_empty() => None, +// Ok(bs) => Some(Ok(bs)), +// Err(err) => Some(Err(err)), +// } +// } +// } #[cfg(test)] mod tests { @@ -561,7 +527,7 @@ mod tests { let mut reader = op.reader(path).await.unwrap(); let mut buf = Vec::new(); reader - .read_to_end(&mut buf) + .read_to_end_at(&mut buf, 0) .await .expect("read to end must succeed"); @@ -581,17 +547,7 @@ mod tests { let mut reader = op.reader(path).await.unwrap(); let mut buf = Vec::new(); reader - .read_to_end(&mut buf) - .await - .expect("read to end must succeed"); - assert_eq!(buf, content); - - let n = reader.seek(tokio::io::SeekFrom::Start(0)).await.unwrap(); - assert_eq!(n, 0, "seek position must be 0"); - - let mut buf = Vec::new(); - reader - .read_to_end(&mut buf) + .read_to_end_at(&mut buf, 0) .await .expect("read to end must succeed"); assert_eq!(buf, content); diff --git a/core/tests/behavior/async_fuzz.rs b/core/tests/behavior/async_fuzz.rs index 6113f0ee72d0..9f27db993f96 100644 --- a/core/tests/behavior/async_fuzz.rs +++ b/core/tests/behavior/async_fuzz.rs @@ -25,14 +25,9 @@ use opendal::raw::BytesRange; use crate::*; -pub fn tests(op: &Operator, tests: &mut Vec) { - tests.extend(async_trials!( - op, - test_fuzz_issue_2717, - test_fuzz_pr_3395_case_1, - test_fuzz_pr_3395_case_2 - )) -} +// pub fn tests(op: &Operator, tests: &mut Vec) { +// tests.extend(async_trials!(op)) +// } async fn test_fuzz_read( op: Operator, @@ -59,89 +54,3 @@ async fn test_fuzz_read( checker.check(r, actions).await; Ok(()) } - -/// This fuzz test is to reproduce . -/// -/// The simplified cases could be seen as: -/// -/// ``` -/// FuzzInput { -/// actions: [ -/// Seek( -/// End( -/// -2, -/// ), -/// ), -/// Read { -/// size: 0, -/// }, -/// ], -/// data: [ -/// 0, -/// 0, -/// ], -/// range: ( -/// 1, -/// 2, -/// ) -/// ] -/// } -/// ``` -/// -/// Which means: -/// -/// - A file with 2 bytes of content. -/// - Open as an range reader of `1..2`. -/// - Seek to `End(-2)` first -/// -/// The expected result is seek returns InvalidInput error because the seek position -/// is invalid for given range `1..2`. However, the actual behavior is we seek to `0` -/// and results in a panic. -pub async fn test_fuzz_issue_2717(op: Operator) -> Result<()> { - let actions = [ReadAction::Seek(SeekFrom::End(-2))]; - - test_fuzz_read(op, 2, .., &actions).await -} - -/// This fuzz test is to reproduce bug inside . -/// -/// The simplified cases could be seen as: -/// -/// ``` -/// FuzzInput { -/// path: "06ae5d93-c0e9-43f2-ae5a-225cfaaa40a0", -/// size: 1, -/// range: BytesRange(Some(0), None), -/// actions: [Seek(Current(1)), Next, Seek(End(-1))], -/// } -/// ``` -pub async fn test_fuzz_pr_3395_case_1(op: Operator) -> Result<()> { - let actions = [ - ReadAction::Seek(SeekFrom::Current(1)), - ReadAction::Read(1024), - ReadAction::Seek(SeekFrom::End(-1)), - ]; - test_fuzz_read(op, 1, 0.., &actions).await -} - -/// This fuzz test is to reproduce bug inside . -/// -/// The simplified cases could be seen as: -/// -/// ``` -/// FuzzInput { -/// path: "e6056989-7c7c-4075-b975-5ae380884333", -/// size: 1, -/// range: BytesRange(Some(0), None), -/// actions: [Next, Seek(Current(1)), Next, Seek(End(0))], -/// } -/// ``` -pub async fn test_fuzz_pr_3395_case_2(op: Operator) -> Result<()> { - let actions = [ - ReadAction::Read(1024), - ReadAction::Seek(SeekFrom::Current(1)), - ReadAction::Read(1024), - ReadAction::Seek(SeekFrom::End(0)), - ]; - test_fuzz_read(op, 1, 0.., &actions).await -} diff --git a/core/tests/behavior/async_read.rs b/core/tests/behavior/async_read.rs index adbbeafa5897..226d287d7a28 100644 --- a/core/tests/behavior/async_read.rs +++ b/core/tests/behavior/async_read.rs @@ -49,7 +49,6 @@ pub fn tests(op: &Operator, tests: &mut Vec) { test_read_with_override_cache_control, test_read_with_override_content_disposition, test_read_with_override_content_type, - test_read_with_invalid_seek )) } @@ -596,28 +595,6 @@ pub async fn test_read_with_override_content_type(op: Operator) -> anyhow::Resul Ok(()) } -/// seeking a negative position should return a InvalidInput error -pub async fn test_read_with_invalid_seek(op: Operator) -> anyhow::Result<()> { - let (path, content, _) = TEST_FIXTURE.new_file(op.clone()); - - op.write(&path, content.clone()) - .await - .expect("write must succeed"); - - let mut r = op.reader(&path).await?; - let res = r.seek(std::io::SeekFrom::Current(-1024)).await; - - assert!(res.is_err()); - - assert_eq!( - res.unwrap_err().kind(), - ErrorKind::InvalidInput, - "seeking a negative position should return a InvalidInput error" - ); - - Ok(()) -} - /// Read full content should match. pub async fn test_read_only_read_full(op: Operator) -> anyhow::Result<()> { let bs = op.read("normal_file.txt").await?; From e4127d4ac9065d34d65067d5e1f84d67a8585ce3 Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Wed, 20 Mar 2024 16:59:39 +0800 Subject: [PATCH 046/111] Remove not needed files Signed-off-by: Xuanwo --- core/src/raw/oio/read/file_read.rs | 319 --------- core/src/raw/oio/read/futures_read.rs | 84 --- .../src/raw/oio/read/into_read_from_stream.rs | 64 -- core/src/raw/oio/read/into_streamable_read.rs | 131 ---- core/src/raw/oio/read/lazy_read.rs | 109 ---- core/src/raw/oio/read/mod.rs | 29 - core/src/raw/oio/read/range_read.rs | 603 ------------------ core/src/raw/oio/read/std_read.rs | 77 --- core/src/raw/oio/read/tokio_read.rs | 84 --- 9 files changed, 1500 deletions(-) delete mode 100644 core/src/raw/oio/read/file_read.rs delete mode 100644 core/src/raw/oio/read/futures_read.rs delete mode 100644 core/src/raw/oio/read/into_read_from_stream.rs delete mode 100644 core/src/raw/oio/read/into_streamable_read.rs delete mode 100644 core/src/raw/oio/read/lazy_read.rs delete mode 100644 core/src/raw/oio/read/range_read.rs delete mode 100644 core/src/raw/oio/read/std_read.rs delete mode 100644 core/src/raw/oio/read/tokio_read.rs diff --git a/core/src/raw/oio/read/file_read.rs b/core/src/raw/oio/read/file_read.rs deleted file mode 100644 index b0b501edd9ce..000000000000 --- a/core/src/raw/oio/read/file_read.rs +++ /dev/null @@ -1,319 +0,0 @@ -// 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::SeekFrom; -use std::sync::Arc; - -use bytes::Bytes; - -use crate::raw::*; -use crate::*; - -/// FileReader that implement range read and streamable read on seekable reader. -/// -/// `oio::Reader` requires the underlying reader to handle range correctly and have streamable support. -/// But some services like `fs`, `hdfs` only have seek support. FileReader implements range and stream -/// support based on `seek`. We will maintain the correct range for give file and implement streamable -/// operations based on [`oio::AdaptiveBuf`]. -pub struct FileReader { - acc: Arc, - path: Arc, - op: OpRead, - - offset: Option, - size: Option, - cur: u64, - - reader: Option, - /// Do we need to reset our cursor? - seek_dirty: bool, -} - -impl FileReader -where - A: Accessor, -{ - /// Create a new FileReader. - /// - /// # Notes - /// - /// It's required that input reader's cursor is at the input `start` of the file. - pub fn new(acc: Arc, path: &str, op: OpRead) -> FileReader { - FileReader { - acc, - path: Arc::new(path.to_string()), - op, - - offset: None, - size: None, - cur: 0, - reader: None, - seek_dirty: false, - } - } -} - -impl FileReader -where - A: Accessor, - R: oio::Read, -{ - /// calculate_offset will make sure that the offset has been set. - async fn offset(r: &mut R, range: BytesRange) -> Result<(Option, Option)> { - let (offset, size) = match (range.offset(), range.size()) { - (None, None) => (0, None), - (None, Some(size)) => { - let start = r.seek(SeekFrom::End(-(size as i64))).await?; - (start, Some(size)) - } - (Some(offset), None) => { - let start = r.seek(SeekFrom::Start(offset)).await?; - (start, None) - } - (Some(offset), Some(size)) => { - let start = r.seek(SeekFrom::Start(offset)).await?; - (start, Some(size)) - } - }; - - Ok((Some(offset), size)) - } - - async fn seek_inner( - r: &mut R, - offset: Option, - size: Option, - cur: u64, - pos: SeekFrom, - ) -> Result { - let offset = offset.expect("offset should be set for calculate_position"); - - match pos { - SeekFrom::Start(n) => { - // It's valid for user to seek outsides end of the file. - r.seek(SeekFrom::Start(offset + n)).await - } - SeekFrom::End(n) => { - let size = - size.expect("size should be set for calculate_position when seek with end"); - if size as i64 + n < 0 { - return Err(Error::new( - ErrorKind::InvalidInput, - "seek to a negative position is invalid", - ) - .with_context("position", format!("{pos:?}"))); - } - // size is known, we can convert SeekFrom::End into SeekFrom::Start. - let pos = SeekFrom::Start(offset + (size as i64 + n) as u64); - r.seek(pos).await - } - SeekFrom::Current(n) => { - if cur as i64 + n < 0 { - return Err(Error::new( - ErrorKind::InvalidInput, - "seek to a negative position is invalid", - ) - .with_context("position", format!("{pos:?}"))); - } - let pos = SeekFrom::Start(offset + (cur as i64 + n) as u64); - r.seek(pos).await - } - } - } -} - -impl FileReader -where - A: Accessor, - R: oio::BlockingRead, -{ - /// calculate_offset will make sure that the offset has been set. - fn calculate_offset(r: &mut R, range: BytesRange) -> Result<(Option, Option)> { - let (offset, size) = match (range.offset(), range.size()) { - (None, None) => (0, None), - (None, Some(size)) => { - let start = r.seek(SeekFrom::End(-(size as i64)))?; - (start, Some(size)) - } - (Some(offset), None) => { - let start = r.seek(SeekFrom::Start(offset))?; - (start, None) - } - (Some(offset), Some(size)) => { - let start = r.seek(SeekFrom::Start(offset))?; - (start, Some(size)) - } - }; - - Ok((Some(offset), size)) - } - - fn blocking_seek_inner( - r: &mut R, - offset: Option, - size: Option, - cur: u64, - pos: SeekFrom, - ) -> Result { - let offset = offset.expect("offset should be set for calculate_position"); - - match pos { - SeekFrom::Start(n) => { - // It's valid for user to seek outsides end of the file. - r.seek(SeekFrom::Start(offset + n)) - } - SeekFrom::End(n) => { - let size = - size.expect("size should be set for calculate_position when seek with end"); - if size as i64 + n < 0 { - return Err(Error::new( - ErrorKind::InvalidInput, - "seek to a negative position is invalid", - ) - .with_context("position", format!("{pos:?}"))); - } - // size is known, we can convert SeekFrom::End into SeekFrom::Start. - let pos = SeekFrom::Start(offset + (size as i64 + n) as u64); - r.seek(pos) - } - SeekFrom::Current(n) => { - if cur as i64 + n < 0 { - return Err(Error::new( - ErrorKind::InvalidInput, - "seek to a negative position is invalid", - ) - .with_context("position", format!("{pos:?}"))); - } - let pos = SeekFrom::Start(offset + (cur as i64 + n) as u64); - r.seek(pos) - } - } - } -} - -impl oio::Read for FileReader -where - A: Accessor, - R: oio::Read, -{ - async fn read_at(&self, offset: u64, limit: usize) -> Result { - if self.reader.is_none() { - // FileReader doesn't support range, we will always use full range to open a file. - let op = self.op.clone().with_range(BytesRange::from(..)); - let (_, r) = self.acc.read(&self.path, op).await?; - self.reader = Some(r); - } - - let r = self.reader.as_mut().expect("reader must be valid"); - - // We should know where to start read the data. - if self.offset.is_none() { - (self.offset, self.size) = Self::offset(r, self.op.range()).await?; - } - - r.read(limit).await - } - - async fn seek(&mut self, pos: SeekFrom) -> Result { - if self.reader.is_none() { - // FileReader doesn't support range, we will always use full range to open a file. - let op = self.op.clone().with_range(BytesRange::from(..)); - let (_, r) = self.acc.read(&self.path, op).await?; - self.reader = Some(r); - } - - let r = self.reader.as_mut().expect("reader must be valid"); - - // We should know where to start read the data. - if self.offset.is_none() { - (self.offset, self.size) = Self::offset(r, self.op.range()).await?; - } - - // Fetch size when seek end. - let current_offset = self.offset.unwrap() + self.cur; - if matches!(pos, SeekFrom::End(_)) && self.size.is_none() { - let size = r.seek(SeekFrom::End(0)).await?; - self.size = Some(size - self.offset.unwrap()); - self.seek_dirty = true; - } - if self.seek_dirty { - // Reset cursor. - r.seek(SeekFrom::Start(current_offset)).await?; - self.seek_dirty = false; - } - - let pos = Self::seek_inner(r, self.offset, self.size, self.cur, pos).await?; - self.cur = pos - self.offset.unwrap(); - Ok(self.cur) - } -} - -impl oio::BlockingRead for FileReader -where - A: Accessor, - R: oio::BlockingRead, -{ - fn read_at(&self, offset: u64, limit: usize) -> Result { - if self.reader.is_none() { - // FileReader doesn't support range, we will always use full range to open a file. - let op = self.op.clone().with_range(BytesRange::from(..)); - let (_, r) = self.acc.blocking_read(&self.path, op)?; - self.reader = Some(r); - } - - let r = self.reader.as_mut().expect("reader must be valid"); - - // We should know where to start read the data. - if self.offset.is_none() { - (self.offset, self.size) = Self::calculate_offset(r, self.op.range())?; - } - - r.read(limit) - } - - fn seek(&mut self, pos: SeekFrom) -> Result { - if self.reader.is_none() { - // FileReader doesn't support range, we will always use full range to open a file. - let op = self.op.clone().with_range(BytesRange::from(..)); - let (_, r) = self.acc.blocking_read(&self.path, op)?; - self.reader = Some(r); - } - - let r = self.reader.as_mut().expect("reader must be valid"); - - // We should know where to start read the data. - if self.offset.is_none() { - (self.offset, self.size) = Self::calculate_offset(r, self.op.range())?; - } - // Fetch size when seek end. - let current_offset = self.offset.unwrap() + self.cur; - if matches!(pos, SeekFrom::End(_)) && self.size.is_none() { - let size = r.seek(SeekFrom::End(0))?; - self.size = Some(size - self.offset.unwrap()); - self.seek_dirty = true; - } - if self.seek_dirty { - // Reset cursor. - r.seek(SeekFrom::Start(current_offset))?; - self.seek_dirty = false; - } - - let pos = Self::blocking_seek_inner(r, self.offset, self.size, self.cur, pos)?; - self.cur = pos - self.offset.unwrap(); - Ok(self.cur) - } -} diff --git a/core/src/raw/oio/read/futures_read.rs b/core/src/raw/oio/read/futures_read.rs deleted file mode 100644 index c964ad44b08c..000000000000 --- a/core/src/raw/oio/read/futures_read.rs +++ /dev/null @@ -1,84 +0,0 @@ -// 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::SeekFrom; - -use bytes::Bytes; -use futures::AsyncRead; -use futures::AsyncReadExt; -use futures::AsyncSeek; -use futures::AsyncSeekExt; -use tokio::io::ReadBuf; - -use crate::raw::*; -use crate::*; - -/// FuturesReader implements [`oio::Read`] via [`AsyncRead`] + [`AsyncSeek`]. -pub struct FuturesReader { - inner: R, - buf: Vec, -} - -impl FuturesReader { - /// Create a new futures reader. - pub fn new(inner: R) -> Self { - Self { - inner, - buf: Vec::with_capacity(64 * 1024), - } - } -} - -impl oio::Read for FuturesReader -where - R: AsyncRead + AsyncSeek + Unpin + Send + Sync, -{ - async fn seek(&mut self, pos: SeekFrom) -> Result { - self.inner.seek(pos).await.map_err(|err| { - new_std_io_error(err) - .with_operation(oio::ReadOperation::Seek) - .with_context("source", "FuturesReader") - }) - } - - async fn read_at(&self, offset: u64, limit: usize) -> Result { - // Make sure buf has enough space. - if self.buf.capacity() < limit { - self.buf.reserve(limit); - } - let buf = self.buf.spare_capacity_mut(); - let mut read_buf: ReadBuf = ReadBuf::uninit(buf); - - // SAFETY: Read at most `size` bytes into `read_buf`. - unsafe { - read_buf.assume_init(limit); - } - - let n = self - .inner - .read(read_buf.initialized_mut()) - .await - .map_err(|err| { - new_std_io_error(err) - .with_operation(oio::ReadOperation::Read) - .with_context("source", "FuturesReader") - })?; - read_buf.set_filled(n); - - Ok(Bytes::copy_from_slice(read_buf.filled())) - } -} diff --git a/core/src/raw/oio/read/into_read_from_stream.rs b/core/src/raw/oio/read/into_read_from_stream.rs deleted file mode 100644 index bab22d7ac958..000000000000 --- a/core/src/raw/oio/read/into_read_from_stream.rs +++ /dev/null @@ -1,64 +0,0 @@ -// 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::cmp::min; -use std::io::SeekFrom; - -use bytes::Bytes; -use futures::StreamExt; - -use crate::raw::*; -use crate::*; - -/// Convert given stream `futures::Stream>` into [`oio::Reader`]. -pub fn into_read_from_stream(stream: S) -> FromStreamReader { - FromStreamReader { - inner: stream, - buf: Bytes::new(), - } -} - -/// FromStreamReader will convert a `futures::Stream>` into `oio::Read` -pub struct FromStreamReader { - inner: S, - buf: Bytes, -} - -impl oio::Read for FromStreamReader -where - S: futures::Stream> + Send + Sync + Unpin + 'static, - T: Into, -{ - async fn seek(&mut self, _: SeekFrom) -> Result { - Err(Error::new( - ErrorKind::Unsupported, - "FromStreamReader can't support operation", - )) - } - - async fn read_at(&self, offset: u64, limit: usize) -> Result { - if self.buf.is_empty() { - self.buf = match self.inner.next().await.transpose()? { - Some(v) => v.into(), - None => return Ok(Bytes::new()), - }; - } - - let bs = self.buf.split_to(min(limit, self.buf.len())); - Ok(bs) - } -} diff --git a/core/src/raw/oio/read/into_streamable_read.rs b/core/src/raw/oio/read/into_streamable_read.rs deleted file mode 100644 index e52627a358e4..000000000000 --- a/core/src/raw/oio/read/into_streamable_read.rs +++ /dev/null @@ -1,131 +0,0 @@ -// 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::cmp::min; -use std::io::SeekFrom; - -use bytes::Bytes; -use tokio::io::ReadBuf; - -use crate::raw::*; -use crate::*; - -/// into_streamable is used to make [`oio::Read`] or [`oio::BlockingRead`] streamable. -pub fn into_streamable_read(r: R, capacity: usize) -> StreamableReader { - StreamableReader { - r, - buf: Vec::with_capacity(capacity), - } -} - -/// Make given read streamable. -pub struct StreamableReader { - r: R, - buf: Vec, -} - -impl oio::Read for StreamableReader { - async fn seek(&mut self, pos: SeekFrom) -> Result { - self.r.seek(pos).await - } - - async fn read_at(&self, offset: u64, limit: usize) -> Result { - let size = min(self.buf.capacity(), limit); - - let dst = self.buf.spare_capacity_mut(); - let mut buf = ReadBuf::uninit(dst); - - // SAFETY: Read at most `size` bytes into `read_buf`. - unsafe { buf.assume_init(size) }; - - let bs = self.r.read(size).await?; - buf.put_slice(&bs); - buf.set_filled(bs.len()); - - Ok(Bytes::from(buf.filled().to_vec())) - } -} - -impl oio::BlockingRead for StreamableReader { - fn read_at(&self, offset: u64, limit: usize) -> Result { - self.r.read(limit) - } - - fn seek(&mut self, pos: SeekFrom) -> Result { - self.r.seek(pos) - } -} - -#[cfg(test)] -mod tests { - use crate::raw::oio::Read; - use bytes::BufMut; - use bytes::BytesMut; - use rand::prelude::*; - - use super::*; - - #[tokio::test] - async fn test_into_stream() { - let mut rng = ThreadRng::default(); - // Generate size between 1B..16MB. - let size = rng.gen_range(1..16 * 1024 * 1024); - let mut content = vec![0; size]; - rng.fill_bytes(&mut content); - // Generate cap between 1B..1MB; - let cap = rng.gen_range(1..1024 * 1024); - - let r = oio::Cursor::from(content.clone()); - let mut s = into_streamable_read(Box::new(r) as oio::Reader, cap); - - let mut bs = BytesMut::new(); - loop { - let b = s.read(4 * 1024 * 1024).await.expect("read must success"); - if b.is_empty() { - break; - } - bs.put_slice(&b); - } - assert_eq!(bs.freeze().to_vec(), content) - } - - #[test] - fn test_into_stream_blocking() { - use oio::BlockingRead; - - let mut rng = ThreadRng::default(); - // Generate size between 1B..16MB. - let size = rng.gen_range(1..16 * 1024 * 1024); - let mut content = vec![0; size]; - rng.fill_bytes(&mut content); - // Generate cap between 1B..1MB; - let cap = rng.gen_range(1..1024 * 1024); - - let r = oio::Cursor::from(content.clone()); - let mut s = into_streamable_read(Box::new(r) as oio::BlockingReader, cap); - - let mut bs = BytesMut::with_capacity(size); - loop { - let buf = s.read(size).expect("read must success"); - if buf.is_empty() { - break; - } - bs.put_slice(&buf) - } - assert_eq!(bs.freeze().to_vec(), content) - } -} diff --git a/core/src/raw/oio/read/lazy_read.rs b/core/src/raw/oio/read/lazy_read.rs deleted file mode 100644 index 2a8cca3e9e87..000000000000 --- a/core/src/raw/oio/read/lazy_read.rs +++ /dev/null @@ -1,109 +0,0 @@ -// 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::SeekFrom; -use std::sync::Arc; - -use bytes::Bytes; - -use crate::raw::*; -use crate::*; - -/// LazyReader implements [`oio::Read`] in a lazy way. -/// -/// The real requests are send when users calling read or seek. -pub struct LazyReader { - acc: Arc, - path: Arc, - op: OpRead, - reader: Option, -} - -impl LazyReader -where - A: Accessor, -{ - /// Create a new [`oio::Reader`] with lazy support. - pub fn new(acc: Arc, path: &str, op: OpRead) -> LazyReader { - LazyReader { - acc, - path: Arc::new(path.to_string()), - op, - - reader: None, - } - } -} - -impl LazyReader -where - A: Accessor, - R: oio::Read, -{ - async fn reader(&mut self) -> Result<&mut R> { - if self.reader.is_none() { - let (_, r) = self.acc.read(&self.path, self.op.clone()).await?; - self.reader = Some(r); - } - - Ok(self.reader.as_mut().expect("reader must be valid")) - } -} - -impl oio::Read for LazyReader -where - A: Accessor, - R: oio::Read, -{ - async fn seek(&mut self, pos: SeekFrom) -> Result { - self.reader().await?.seek(pos).await - } - - async fn read_at(&self, offset: u64, limit: usize) -> Result { - let r = self.reader().await?; - r.read(limit).await - } -} - -impl LazyReader -where - A: Accessor, - R: oio::BlockingRead, -{ - fn blocking_reader(&mut self) -> Result<&mut R> { - if self.reader.is_none() { - let (_, r) = self.acc.blocking_read(&self.path, self.op.clone())?; - self.reader = Some(r); - } - - Ok(self.reader.as_mut().expect("reader must be valid")) - } -} - -impl oio::BlockingRead for LazyReader -where - A: Accessor, - R: oio::BlockingRead, -{ - fn read_at(&self, offset: u64, limit: usize) -> Result { - self.blocking_reader()?.read(limit) - } - - fn seek(&mut self, pos: SeekFrom) -> Result { - self.blocking_reader()?.seek(pos) - } -} diff --git a/core/src/raw/oio/read/mod.rs b/core/src/raw/oio/read/mod.rs index 02ac30d8e57c..3144ce874b3a 100644 --- a/core/src/raw/oio/read/mod.rs +++ b/core/src/raw/oio/read/mod.rs @@ -21,32 +21,3 @@ pub use api::BlockingReader; pub use api::Read; pub use api::ReadOperation; pub use api::Reader; - -// mod into_streamable_read; -// pub use into_streamable_read::into_streamable_read; -// pub use into_streamable_read::StreamableReader; - -// mod range_read; -// pub use range_read::RangeReader; - -// mod file_read; -// pub use file_read::FileReader; - -// mod into_read_from_stream; -// pub use into_read_from_stream::into_read_from_stream; -// pub use into_read_from_stream::FromStreamReader; - -// mod futures_read; -// pub use futures_read::FuturesReader; - -// mod tokio_read; -// pub use tokio_read::TokioReader; - -// mod std_read; -// pub use std_read::StdReader; - -// mod lazy_read; -// pub use lazy_read::LazyReader; - -// mod buffer_reader; -// pub use buffer_reader::BufferReader; diff --git a/core/src/raw/oio/read/range_read.rs b/core/src/raw/oio/read/range_read.rs deleted file mode 100644 index c69a3695afb9..000000000000 --- a/core/src/raw/oio/read/range_read.rs +++ /dev/null @@ -1,603 +0,0 @@ -// 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::SeekFrom; -use std::sync::Arc; - -use bytes::Bytes; - -use crate::raw::*; -use crate::*; - -/// RangeReader that can do seek on non-seekable reader. -/// -/// `oio::Reader` requires the underlying reader to be seekable, but some services like s3, gcs -/// doesn't support seek natively. RangeReader implement seek by read_with_range. We will start -/// a new read request with the correct range when seek is called. -/// -/// The `seek` operation on `RangeReader` is zero cost and purely in-memory. But calling `seek` -/// while there is a pending read request will cancel the request and start a new one. This could -/// add extra cost to the read operation. -pub struct RangeReader { - acc: Arc, - path: Arc, - op: OpRead, - - offset: Option, - size: Option, - cur: u64, - reader: Option, -} - -impl RangeReader -where - A: Accessor, -{ - /// Create a new [`oio::Reader`] by range support. - /// - /// # Input - /// - /// The input is an Accessor will may return a non-seekable reader. - /// - /// # Output - /// - /// The output is a reader that can be seek by range. - /// - /// # Notes - /// - /// This operation is not zero cost. If the accessor already returns a - /// seekable reader, please don't use this. - pub fn new(acc: Arc, path: &str, op: OpRead) -> RangeReader { - // Normalize range like `..` into `0..` to make sure offset is valid. - let (offset, size) = match (op.range().offset(), op.range().size()) { - (None, None) => (Some(0), None), - v => v, - }; - - RangeReader { - acc, - path: Arc::new(path.to_string()), - op, - - offset, - size, - cur: 0, - reader: None, - } - } - - /// Ensure current reader's offset is valid via total_size. - fn ensure_offset(&mut self, total_size: u64) -> Result<()> { - (self.offset, self.size) = match (self.offset, self.size) { - (None, Some(size)) => { - if size > total_size { - // If returns an error, we should reset - // state to Idle so that we can retry it. - self.reader = None; - return Err(Error::new( - ErrorKind::InvalidInput, - "read to a negative or overflowing position is invalid", - )); - } - - (Some(total_size - size), Some(size)) - } - (Some(offset), None) => { - // It's valid for reader to seek to a position that out of the content length. - // We should return `Ok(0)` instead of an error at this case to align fs behavior. - let size = total_size.checked_sub(offset).unwrap_or_default(); - - (Some(offset), Some(size)) - } - (Some(offset), Some(size)) => (Some(offset), Some(size)), - (None, None) => { - unreachable!("fill_range should not reach this case after normalization") - } - }; - - Ok(()) - } - - /// Ensure size will use the information returned by RpRead to calculate the correct size for reader. - /// - /// - If `RpRead` returns `range`, we can calculate the correct size by `range.size()`. - /// - If `RpRead` returns `size`, we can use it's as the returning body's size. - fn ensure_size(&mut self, total_size: Option, content_size: Option) { - if let Some(total_size) = total_size { - // It's valid for reader to seek to a position that out of the content length. - // We should return `Ok(0)` instead of an error at this case to align fs behavior. - let size = total_size - .checked_sub(self.offset.expect("reader offset must be valid")) - .unwrap_or_default(); - - // Ensure size when: - // - // - reader's size is unknown. - // - reader's size is larger than file's size. - if self.size.is_none() || Some(size) < self.size { - self.size = Some(size); - return; - } - } - - if let Some(content_size) = content_size { - if content_size == 0 { - // Skip size set if content size is 0 since it could be invalid. - // - // For example, users seek to `u64::MAX` and calling read. - return; - } - - let calculated_size = content_size + self.cur; - - // Ensure size when: - // - // - reader's size is unknown. - // - reader's size is larger than file's size. - if self.size.is_none() || Some(calculated_size) < self.size { - self.size = Some(calculated_size); - } - } - } - - /// Calculate the current range, maybe sent as next read request. - /// - /// # Panics - /// - /// Offset must be normalized before calling this function. - /// - /// - `..` should be transformed into `0..` - /// - `..size` should be transformed into `(total-size)..total`. - fn calculate_range(&self) -> BytesRange { - let offset = self - .offset - .expect("offset must be set before calculating range"); - - BytesRange::new(Some(offset + self.cur), self.size.map(|v| v - self.cur)) - } -} - -impl RangeReader -where - A: Accessor, - R: oio::Read, -{ - async fn read_future(&self) -> Result<(RpRead, R)> { - let mut op = self.op.clone(); - // cur != 0 means we have read some data out, we should convert - // the op into deterministic to avoid ETag changes. - if self.cur != 0 { - op = op.into_deterministic(); - } - // Alter OpRead with correct calculated range. - op = op.with_range(self.calculate_range()); - - self.acc.read(&self.path, op).await - } - - async fn stat_future(&self) -> Result { - // Handle if-match and if-none-match correctly. - let mut args = OpStat::default(); - // TODO: stat should support range to check if ETag matches. - if self.op.range().is_full() { - if let Some(v) = self.op.if_match() { - args = args.with_if_match(v); - } - if let Some(v) = self.op.if_none_match() { - args = args.with_if_none_match(v); - } - } - - self.acc.stat(&self.path, args).await - } -} - -impl RangeReader -where - A: Accessor, - R: oio::BlockingRead, -{ - fn read_action(&self) -> Result<(RpRead, R)> { - let acc = self.acc.clone(); - let path = self.path.clone(); - - let mut op = self.op.clone(); - // cur != 0 means we have read some data out, we should convert - // the op into deterministic to avoid ETag changes. - if self.cur != 0 { - op = op.into_deterministic(); - } - // Alter OpRead with correct calculated range. - op = op.with_range(self.calculate_range()); - - acc.blocking_read(&path, op) - } - - fn stat_action(&self) -> Result { - let acc = self.acc.clone(); - let path = self.path.clone(); - - // Handle if-match and if-none-match correctly. - let mut args = OpStat::default(); - // TODO: stat should support range to check if ETag matches. - if self.op.range().is_full() { - if let Some(v) = self.op.if_match() { - args = args.with_if_match(v); - } - if let Some(v) = self.op.if_none_match() { - args = args.with_if_none_match(v); - } - } - - acc.blocking_stat(&path, args) - } -} - -impl oio::Read for RangeReader -where - A: Accessor, - R: oio::Read, -{ - async fn read_at(&self, offset: u64, limit: usize) -> Result { - // Sanity check for normal cases. - if self.cur >= self.size.unwrap_or(u64::MAX) { - return Ok(Bytes::new()); - } - - if self.offset.is_none() { - let rp = match self.stat_future().await { - Ok(v) => v, - Err(err) => return Err(err), - }; - let length = rp.into_metadata().content_length(); - self.ensure_offset(length)? - } - if self.reader.is_none() { - let (rp, r) = match self.read_future().await { - Ok((rp, r)) => (rp, r), - Err(err) => return Err(err), - }; - - self.ensure_size(rp.range().unwrap_or_default().size(), rp.size()); - self.reader = Some(r); - } - - let r = self.reader.as_mut().expect("reader must be valid"); - match r.read(limit).await { - Ok(bs) => { - self.cur += bs.len() as u64; - Ok(bs) - } - Err(err) => { - self.reader = None; - Err(err) - } - } - } - - async fn seek(&mut self, pos: SeekFrom) -> Result { - // There is an optimization here that we can calculate if users trying to seek - // the same position, for example, `reader.seek(SeekFrom::Current(0))`. - // In this case, we can just return current position without dropping reader. - if pos == SeekFrom::Current(0) || pos == SeekFrom::Start(self.cur) { - return Ok(self.cur); - } - - // We are seeking to other places, let's drop existing reader. - self.reader = None; - - let (base, amt) = match pos { - SeekFrom::Start(n) => (0, n as i64), - SeekFrom::Current(n) => (self.cur as i64, n), - SeekFrom::End(n) => { - if let Some(size) = self.size { - (size as i64, n) - } else { - let rp = self.stat_future().await?; - let length = rp.into_metadata().content_length(); - self.ensure_offset(length)?; - - (length as i64, n) - } - } - }; - - let seek_pos = match base.checked_add(amt) { - Some(n) if n >= 0 => n as u64, - _ => { - return Err(Error::new( - ErrorKind::InvalidInput, - "invalid seek to a negative or overflowing position", - )) - } - }; - - self.cur = seek_pos; - Ok(self.cur) - } -} - -impl oio::BlockingRead for RangeReader -where - A: Accessor, - R: oio::BlockingRead, -{ - fn read_at(&self, offset: u64, limit: usize) -> Result { - // Sanity check for normal cases. - if self.cur >= self.size.unwrap_or(u64::MAX) { - return Ok(Bytes::new()); - } - - if self.offset.is_none() { - let rp = match self.stat_action() { - Ok(v) => v, - Err(err) => return Err(err), - }; - let length = rp.into_metadata().content_length(); - self.ensure_offset(length)? - } - if self.reader.is_none() { - let (rp, r) = match self.read_action() { - Ok((rp, r)) => (rp, r), - Err(err) => return Err(err), - }; - - self.ensure_size(rp.range().unwrap_or_default().size(), rp.size()); - self.reader = Some(r); - } - - let r = self.reader.as_mut().expect("reader must be valid"); - match r.read(limit) { - Ok(bs) => { - self.cur += bs.len() as u64; - Ok(bs) - } - Err(err) => { - self.reader = None; - Err(err) - } - } - } - - fn seek(&mut self, pos: SeekFrom) -> Result { - // There is an optimization here that we can calculate if users trying to seek - // the same position, for example, `reader.seek(SeekFrom::Current(0))`. - // In this case, we can just return current position without dropping reader. - if pos == SeekFrom::Current(0) || pos == SeekFrom::Start(self.cur) { - return Ok(self.cur); - } - - // We are seeking to other places, let's drop existing reader. - self.reader = None; - - let (base, amt) = match pos { - SeekFrom::Start(n) => (0, n as i64), - SeekFrom::Current(n) => (self.cur as i64, n), - SeekFrom::End(n) => { - if let Some(size) = self.size { - (size as i64, n) - } else { - let rp = self.stat_action()?; - let length = rp.into_metadata().content_length(); - self.ensure_offset(length)?; - - (length as i64, n) - } - } - }; - - let seek_pos = match base.checked_add(amt) { - Some(n) if n >= 0 => n as u64, - _ => { - return Err(Error::new( - ErrorKind::InvalidInput, - "invalid seek to a negative or overflowing position", - )) - } - }; - - self.cur = seek_pos; - Ok(self.cur) - } -} - -#[cfg(test)] -mod tests { - use std::io::SeekFrom; - - use async_trait::async_trait; - use bytes::Bytes; - use futures::AsyncReadExt; - use rand::prelude::*; - use sha2::Digest; - use sha2::Sha256; - - use super::*; - - // Generate bytes between [4MiB, 16MiB) - fn gen_bytes() -> (Bytes, usize) { - let mut rng = thread_rng(); - - let size = rng.gen_range(4 * 1024 * 1024..16 * 1024 * 1024); - let mut content = vec![0; size]; - rng.fill_bytes(&mut content); - - (Bytes::from(content), size) - } - - #[derive(Debug, Clone, Default)] - struct MockReadService { - data: Bytes, - } - - impl MockReadService { - fn new(data: Bytes) -> Self { - Self { data } - } - } - - #[async_trait] - impl Accessor for MockReadService { - type Reader = MockReader; - type Writer = (); - type Lister = (); - type BlockingReader = (); - type BlockingWriter = (); - type BlockingLister = (); - - fn info(&self) -> AccessorInfo { - let mut am = AccessorInfo::default(); - am.set_native_capability(Capability { - read: true, - ..Default::default() - }); - - am - } - - async fn read(&self, _: &str, args: OpRead) -> Result<(RpRead, Self::Reader)> { - let bs = args.range().apply_on_bytes(self.data.clone()); - - Ok(( - RpRead::new(), - MockReader { - inner: futures::io::Cursor::new(bs.into()), - }, - )) - } - } - - #[derive(Debug, Clone, Default)] - struct MockReader { - inner: futures::io::Cursor>, - } - - impl oio::Read for MockReader { - async fn seek(&mut self, pos: SeekFrom) -> Result { - let _ = pos; - - Err(Error::new( - ErrorKind::Unsupported, - "output reader doesn't support seeking", - )) - } - - async fn read_at(&self, offset: u64, limit: usize) -> Result { - let mut bs = vec![0; limit]; - let n = self.inner.read(&mut bs).await.map_err(|err| { - Error::new(ErrorKind::Unexpected, "read data from mock").set_source(err) - })?; - if n == 0 { - Ok(Bytes::new()) - } else { - Ok(Bytes::from(bs[..n].to_vec())) - } - } - } - - #[tokio::test] - async fn test_read_all() -> anyhow::Result<()> { - let (bs, _) = gen_bytes(); - let acc = Arc::new(MockReadService::new(bs.clone())); - - let r = Box::new(RangeReader::new( - acc, - "x", - OpRead::default().with_range(BytesRange::from(..)), - )) as oio::Reader; - let mut r = Reader::new(r); - - let mut buf = Vec::new(); - r.read_to_end(&mut buf).await?; - assert_eq!(bs.len(), buf.len(), "read size"); - assert_eq!( - format!("{:x}", Sha256::digest(&bs)), - format!("{:x}", Sha256::digest(&buf)), - "read content" - ); - - let n = r.seek(SeekFrom::Start(0)).await?; - assert_eq!(n, 0, "seek position must be 0"); - - let mut buf = Vec::new(); - r.read_to_end(&mut buf).await?; - assert_eq!(bs.len(), buf.len(), "read twice size"); - assert_eq!( - format!("{:x}", Sha256::digest(&bs)), - format!("{:x}", Sha256::digest(&buf)), - "read twice content" - ); - - Ok(()) - } - - #[tokio::test] - async fn test_read_part() -> anyhow::Result<()> { - let (bs, _) = gen_bytes(); - let acc = Arc::new(MockReadService::new(bs.clone())); - - let r = Box::new(RangeReader::new( - acc, - "x", - OpRead::default().with_range(BytesRange::from(4096..4096 + 4096)), - )) as oio::Reader; - let mut r = Reader::new(r); - - let mut buf = Vec::new(); - r.read_to_end(&mut buf).await?; - assert_eq!(4096, buf.len(), "read size"); - assert_eq!( - format!("{:x}", Sha256::digest(&bs[4096..4096 + 4096])), - format!("{:x}", Sha256::digest(&buf)), - "read content" - ); - - let n = r.seek(SeekFrom::Start(0)).await?; - assert_eq!(n, 0, "seek position must be 0"); - - let mut buf = Vec::new(); - r.read_to_end(&mut buf).await?; - assert_eq!(4096, buf.len(), "read twice size"); - assert_eq!( - format!("{:x}", Sha256::digest(&bs[4096..4096 + 4096])), - format!("{:x}", Sha256::digest(&buf)), - "read twice content" - ); - - let n = r.seek(SeekFrom::Start(1024)).await?; - assert_eq!(1024, n, "seek to 1024"); - - let buf = r.read_exact(1024).await?; - assert_eq!( - format!("{:x}", Sha256::digest(&bs[4096 + 1024..4096 + 2048])), - format!("{:x}", Sha256::digest(&buf)), - "read after seek 1024" - ); - - let n = r.seek(SeekFrom::Current(1024)).await?; - assert_eq!(3072, n, "seek to 3072"); - - let buf = r.read_exact(1024).await?; - assert_eq!( - format!("{:x}", Sha256::digest(&bs[4096 + 3072..4096 + 3072 + 1024])), - format!("{:x}", Sha256::digest(&buf)), - "read after seek to 3072" - ); - - Ok(()) - } -} diff --git a/core/src/raw/oio/read/std_read.rs b/core/src/raw/oio/read/std_read.rs deleted file mode 100644 index 8d8655e44353..000000000000 --- a/core/src/raw/oio/read/std_read.rs +++ /dev/null @@ -1,77 +0,0 @@ -// 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 bytes::Bytes; -use std::io::Read; -use std::io::Seek; -use std::io::SeekFrom; -use tokio::io::ReadBuf; - -use crate::raw::*; -use crate::*; - -/// FuturesReader implements [`oio::BlockingRead`] via [`Read`] + [`Seek`]. -pub struct StdReader { - inner: R, - buf: Vec, -} - -impl StdReader { - /// Create a new std reader. - pub fn new(inner: R) -> Self { - Self { - inner, - buf: Vec::with_capacity(64 * 1024), - } - } -} - -impl oio::BlockingRead for StdReader -where - R: Read + Seek + Send + Sync, -{ - fn read_at(&self, offset: u64, limit: usize) -> Result { - // Make sure buf has enough space. - if self.buf.capacity() < limit { - self.buf.reserve(limit); - } - let buf = self.buf.spare_capacity_mut(); - let mut read_buf: ReadBuf = ReadBuf::uninit(buf); - - // SAFETY: Read at most `size` bytes into `read_buf`. - unsafe { - read_buf.assume_init(limit); - } - - let n = self.inner.read(read_buf.initialized_mut()).map_err(|err| { - new_std_io_error(err) - .with_operation(oio::ReadOperation::Read) - .with_context("source", "TokioReader") - })?; - read_buf.set_filled(n); - - Ok(Bytes::copy_from_slice(read_buf.filled())) - } - - fn seek(&mut self, pos: SeekFrom) -> Result { - self.inner.seek(pos).map_err(|err| { - new_std_io_error(err) - .with_operation(oio::ReadOperation::BlockingSeek) - .with_context("source", "StdReader") - }) - } -} diff --git a/core/src/raw/oio/read/tokio_read.rs b/core/src/raw/oio/read/tokio_read.rs deleted file mode 100644 index 36ca350fbb51..000000000000 --- a/core/src/raw/oio/read/tokio_read.rs +++ /dev/null @@ -1,84 +0,0 @@ -// 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::SeekFrom; - -use bytes::Bytes; -use tokio::io::AsyncRead; -use tokio::io::AsyncReadExt; -use tokio::io::AsyncSeek; -use tokio::io::AsyncSeekExt; -use tokio::io::ReadBuf; - -use crate::raw::*; -use crate::*; - -/// FuturesReader implements [`oio::Read`] via [`AsyncRead`] + [`AsyncSeek`]. -pub struct TokioReader { - inner: R, - buf: Vec, -} - -impl TokioReader { - /// Create a new tokio reader. - pub fn new(inner: R) -> Self { - Self { - inner, - buf: Vec::with_capacity(64 * 1024), - } - } -} - -impl oio::Read for TokioReader -where - R: AsyncRead + AsyncSeek + Unpin + Send + Sync, -{ - async fn read_at(&self, offset: u64, limit: usize) -> Result { - // Make sure buf has enough space. - if self.buf.capacity() < limit { - self.buf.reserve(limit); - } - let buf = self.buf.spare_capacity_mut(); - let mut read_buf: ReadBuf = ReadBuf::uninit(buf); - - // SAFETY: Read at most `size` bytes into `read_buf`. - unsafe { - read_buf.assume_init(limit); - } - - let n = self - .inner - .read(read_buf.initialized_mut()) - .await - .map_err(|err| { - new_std_io_error(err) - .with_operation(oio::ReadOperation::Read) - .with_context("source", "TokioReader") - })?; - read_buf.set_filled(n); - - Ok(Bytes::copy_from_slice(read_buf.filled())) - } - - async fn seek(&mut self, pos: SeekFrom) -> Result { - self.inner.seek(pos).await.map_err(|err| { - new_std_io_error(err) - .with_operation(oio::ReadOperation::Seek) - .with_context("source", "TokioReader") - }) - } -} From 5e2666a3a226d689cdbebf0553f612eb8a74f4d3 Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Wed, 20 Mar 2024 17:03:25 +0800 Subject: [PATCH 047/111] Save work Signed-off-by: Xuanwo --- core/src/raw/oio/read/buffer_reader.rs | 802 ------------------------- core/src/types/reader.rs | 2 +- 2 files changed, 1 insertion(+), 803 deletions(-) delete mode 100644 core/src/raw/oio/read/buffer_reader.rs diff --git a/core/src/raw/oio/read/buffer_reader.rs b/core/src/raw/oio/read/buffer_reader.rs deleted file mode 100644 index 2630937e5b40..000000000000 --- a/core/src/raw/oio/read/buffer_reader.rs +++ /dev/null @@ -1,802 +0,0 @@ -// 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::cmp::min; -use std::io::SeekFrom; - -use bytes::Bytes; -use tokio::io::ReadBuf; - -use super::BlockingRead; -use crate::raw::*; -use crate::*; - -/// [BufferReader] allows the underlying reader to fetch data at the buffer's size -/// and is used to amortize the IO's overhead. -pub struct BufferReader { - r: R, - cur: u64, - - /// TODO: maybe we can use chunked bytes here? - buf: Vec, - filled: usize, - pos: usize, -} - -impl BufferReader { - /// Create a new [`oio::Reader`] with a buffer. - pub fn new(r: R, cap: usize) -> BufferReader { - BufferReader { - r, - cur: 0, - - buf: Vec::with_capacity(cap), - filled: 0, - pos: 0, - } - } - - /// Invalidates all data in the internal buffer. - #[inline] - fn discard_buffer(&mut self) { - self.buf.clear(); - self.pos = 0; - self.filled = 0; - } - - /// Returns the capacity of the internal buffer. - fn capacity(&self) -> usize { - self.buf.capacity() - } - - fn consume(&mut self, amt: usize) { - let new_pos = min(self.pos + amt, self.filled); - let amt = new_pos - self.pos; - - self.pos = new_pos; - self.cur += amt as u64; - } - - fn seek_relative(&mut self, offset: i64) -> Option { - let pos = self.pos as u64; - - if let (Some(new_pos), Some(new_cur)) = ( - pos.checked_add_signed(offset), - self.cur.checked_add_signed(offset), - ) { - if new_pos <= self.filled as u64 { - self.cur = new_cur; - self.pos = new_pos as usize; - return Some(self.cur); - } - } - - None - } - - fn unconsumed_buffer_len(&self) -> i64 { - (self.filled as i64) - (self.pos as i64) - } -} - -impl BufferReader -where - R: oio::Read, -{ - async fn fill_buf(&mut self) -> Result<&[u8]> { - // If we've reached the end of our internal buffer then we need to fetch - // some more data from the underlying reader. - // Branch using `>=` instead of the more correct `==` - // to tell the compiler that the pos..cap slice is always valid. - if self.pos >= self.filled { - debug_assert!(self.pos == self.filled); - - let cap = self.capacity(); - self.buf.clear(); - let dst = self.buf.spare_capacity_mut(); - let mut buf = ReadBuf::uninit(dst); - unsafe { buf.assume_init(cap) }; - - let bs = self.r.read(cap).await?; - buf.put_slice(&bs); - unsafe { self.buf.set_len(bs.len()) } - - self.pos = 0; - self.filled = bs.len(); - } - - Ok(&self.buf[self.pos..self.filled]) - } - - async fn inner_seek(&mut self, pos: SeekFrom) -> Result { - let cur = self.r.seek(pos).await?; - self.discard_buffer(); - self.cur = cur; - - Ok(cur) - } -} - -impl oio::Read for BufferReader -where - R: oio::Read, -{ - async fn read_at(&self, offset: u64, limit: usize) -> Result { - if limit == 0 { - return Ok(Bytes::new()); - } - - // If we don't have any buffered data and we're doing a massive read - // (larger than our internal buffer), bypass our internal buffer - // entirely. - if self.pos == self.filled && limit >= self.capacity() { - let res = self.r.read(limit).await; - self.discard_buffer(); - return match res { - Ok(bs) => { - self.cur += bs.len() as u64; - Ok(bs) - } - Err(err) => Err(err), - }; - } - - let bytes = self.fill_buf().await?; - - if bytes.is_empty() { - return Ok(Bytes::new()); - } - let size = min(bytes.len(), limit); - let bytes = Bytes::copy_from_slice(&bytes[..size]); - self.consume(bytes.len()); - Ok(bytes) - } - - async fn seek(&mut self, pos: SeekFrom) -> Result { - match pos { - SeekFrom::Start(new_pos) => { - // TODO(weny): Check the overflowing. - let Some(offset) = (new_pos as i64).checked_sub(self.cur as i64) else { - return self.inner_seek(pos).await; - }; - - match self.seek_relative(offset) { - Some(cur) => Ok(cur), - None => self.inner_seek(pos).await, - } - } - SeekFrom::Current(offset) => match self.seek_relative(offset) { - Some(cur) => Ok(cur), - None => { - self.inner_seek(SeekFrom::Current(offset - self.unconsumed_buffer_len())) - .await - } - }, - SeekFrom::End(_) => self.inner_seek(pos).await, - } - } -} - -impl BufferReader -where - R: BlockingRead, -{ - fn blocking_fill_buf(&mut self) -> Result<&[u8]> { - // If we've reached the end of our internal buffer then we need to fetch - // some more data from the underlying reader. - // Branch using `>=` instead of the more correct `==` - // to tell the compiler that the pos..cap slice is always valid. - if self.pos >= self.filled { - debug_assert!(self.pos == self.filled); - - let cap = self.capacity(); - self.buf.clear(); - let dst = self.buf.spare_capacity_mut(); - let mut buf = ReadBuf::uninit(dst); - unsafe { buf.assume_init(cap) }; - - let bs = self.r.read(cap)?; - buf.put_slice(&bs); - unsafe { self.buf.set_len(bs.len()) } - - self.pos = 0; - self.filled = bs.len(); - } - - Ok(&self.buf[self.pos..self.filled]) - } - - fn blocking_inner_seek(&mut self, pos: SeekFrom) -> Result { - let cur = self.r.seek(pos)?; - self.discard_buffer(); - self.cur = cur; - - Ok(cur) - } -} - -impl BlockingRead for BufferReader -where - R: BlockingRead, -{ - fn read_at(&self, offset: u64, limit: usize) -> Result { - if limit == 0 { - return Ok(Bytes::new()); - } - - // If we don't have any buffered data and we're doing a massive read - // (larger than our internal buffer), bypass our internal buffer - // entirely. - if self.pos == self.filled && limit >= self.capacity() { - let res = self.r.read(limit); - self.discard_buffer(); - return match res { - Ok(bs) => { - self.cur += bs.len() as u64; - Ok(bs) - } - Err(err) => Err(err), - }; - } - - let bytes = self.blocking_fill_buf()?; - - if bytes.is_empty() { - return Ok(Bytes::new()); - } - let size = min(bytes.len(), limit); - let bytes = Bytes::copy_from_slice(&bytes[..size]); - self.consume(bytes.len()); - Ok(bytes) - } - - fn seek(&mut self, pos: SeekFrom) -> Result { - match pos { - SeekFrom::Start(new_pos) => { - // TODO(weny): Check the overflowing. - let Some(offset) = (new_pos as i64).checked_sub(self.cur as i64) else { - return self.blocking_inner_seek(pos); - }; - - match self.seek_relative(offset) { - Some(cur) => Ok(cur), - None => self.blocking_inner_seek(pos), - } - } - SeekFrom::Current(offset) => match self.seek_relative(offset) { - Some(cur) => Ok(cur), - None => self - .blocking_inner_seek(SeekFrom::Current(offset - self.unconsumed_buffer_len())), - }, - SeekFrom::End(_) => self.blocking_inner_seek(pos), - } - } -} - -#[cfg(test)] -mod tests { - use std::io::SeekFrom; - use std::sync::Arc; - - use async_trait::async_trait; - use bytes::Bytes; - use rand::prelude::*; - use sha2::Digest; - use sha2::Sha256; - - use super::*; - use crate::raw::oio::RangeReader; - - // Generate bytes between [4MiB, 16MiB) - fn gen_bytes() -> (Bytes, usize) { - let mut rng = thread_rng(); - - let size = rng.gen_range(4 * 1024 * 1024..16 * 1024 * 1024); - let mut content = vec![0; size]; - rng.fill_bytes(&mut content); - - (Bytes::from(content), size) - } - - #[derive(Debug, Clone, Default)] - struct MockReadService { - data: Bytes, - } - - impl MockReadService { - fn new(data: Bytes) -> Self { - Self { data } - } - } - - #[async_trait] - impl Accessor for MockReadService { - type Reader = MockReader; - type Writer = (); - type Lister = (); - type BlockingReader = MockReader; - type BlockingWriter = (); - type BlockingLister = (); - - fn info(&self) -> AccessorInfo { - let mut am = AccessorInfo::default(); - am.set_native_capability(Capability { - read: true, - ..Default::default() - }); - - am - } - - async fn read(&self, _: &str, args: OpRead) -> Result<(RpRead, Self::Reader)> { - let bs = args.range().apply_on_bytes(self.data.clone()); - - Ok(( - RpRead::new(), - MockReader { - inner: oio::Cursor::from(bs), - }, - )) - } - - fn blocking_read(&self, _: &str, args: OpRead) -> Result<(RpRead, Self::BlockingReader)> { - let bs = args.range().apply_on_bytes(self.data.clone()); - - Ok(( - RpRead::new(), - MockReader { - inner: oio::Cursor::from(bs), - }, - )) - } - } - - struct MockReader { - inner: oio::Cursor, - } - - impl oio::Read for MockReader { - async fn seek(&mut self, pos: SeekFrom) -> Result { - let _ = pos; - - Err(Error::new( - ErrorKind::Unsupported, - "output reader doesn't support seeking", - )) - } - - async fn read_at(&self, offset: u64, limit: usize) -> Result { - oio::Read::read(&mut self.inner, limit).await - } - } - - impl BlockingRead for MockReader { - fn read_at(&self, offset: u64, limit: usize) -> Result { - self.inner.read_at(offset, limit) - } - - fn seek(&mut self, _pos: SeekFrom) -> Result { - Err(Error::new( - ErrorKind::Unsupported, - "output reader doesn't support seeking", - )) - } - } - - #[tokio::test] - async fn test_read_from_buf() -> anyhow::Result<()> { - let bs = Bytes::copy_from_slice(&b"Hello, World!"[..]); - - let acc = Arc::new(MockReadService::new(bs.clone())); - let r = Box::new(RangeReader::new(acc, "x", OpRead::default())) as oio::Reader; - - let buf_cap = 10; - let r = Box::new(BufferReader::new(r, buf_cap)) as oio::Reader; - let mut r = Reader::new(r); - - let bs = r.read(5).await?; - assert_eq!(bs.len(), 5); - assert_eq!(bs.as_ref(), b"Hello"); - - let bs = r.read(5).await?; - assert_eq!(bs.len(), 5); - assert_eq!(bs.as_ref(), b", Wor"); - - let bs = r.read(3).await?; - assert_eq!(bs.len(), 3); - assert_eq!(bs.as_ref(), b"ld!"); - - Ok(()) - } - - #[tokio::test] - async fn test_seek() -> anyhow::Result<()> { - let bs = Bytes::copy_from_slice(&b"Hello, World!"[..]); - let acc = Arc::new(MockReadService::new(bs.clone())); - let r = Box::new(RangeReader::new(acc, "x", OpRead::default())) as oio::Reader; - - let buf_cap = 10; - let r = Box::new(BufferReader::new(r, buf_cap)) as oio::Reader; - let mut r = Reader::new(r); - - // The underlying reader buffers the b"Hello, Wor". - let buf = r.read(5).await?; - assert_eq!(buf.len(), 5); - assert_eq!(buf.as_ref(), b"Hello"); - - let pos = r.seek(SeekFrom::Start(7)).await?; - assert_eq!(pos, 7); - let buf = r.read(5).await?; - assert_eq!(&buf, &bs[7..10]); - assert_eq!(buf.len(), 3); - - // Should perform a relative seek. - let pos = r.seek(SeekFrom::Start(0)).await?; - assert_eq!(pos, 0); - let buf = r.read(9).await?; - assert_eq!(&buf, &bs[0..9]); - assert_eq!(buf.len(), 9); - - // Should perform a non-relative seek. - let pos = r.seek(SeekFrom::Start(11)).await?; - assert_eq!(pos, 11); - let buf = r.read(9).await?; - assert_eq!(&buf, &bs[11..13]); - assert_eq!(buf.len(), 2); - - Ok(()) - } - - #[tokio::test] - async fn test_read_all() -> anyhow::Result<()> { - let (bs, _) = gen_bytes(); - let acc = Arc::new(MockReadService::new(bs.clone())); - - let r = Box::new(RangeReader::new( - acc, - "x", - OpRead::default().with_range(BytesRange::from(..)), - )) as oio::Reader; - - let r = Box::new(BufferReader::new(r, 4096 * 1024)) as oio::Reader; - let mut r = Reader::new(r); - - let mut buf = Vec::new(); - r.read_to_end(&mut buf).await?; - assert_eq!(bs.len(), buf.len(), "read size"); - assert_eq!( - format!("{:x}", Sha256::digest(&bs)), - format!("{:x}", Sha256::digest(&buf)), - "read content" - ); - - let n = r.seek(SeekFrom::Start(0)).await?; - assert_eq!(n, 0, "seek position must be 0"); - - let mut buf = Vec::new(); - r.read_to_end(&mut buf).await?; - assert_eq!(bs.len(), buf.len(), "read twice size"); - assert_eq!( - format!("{:x}", Sha256::digest(&bs)), - format!("{:x}", Sha256::digest(&buf)), - "read twice content" - ); - - Ok(()) - } - - #[tokio::test] - async fn test_bypass_seek_relative() -> anyhow::Result<()> { - let bs = Bytes::copy_from_slice( - &b"Hello, World! I'm going to tests a seek relative related bug!"[..], - ); - let acc = Arc::new(MockReadService::new(bs.clone())); - let r = Box::new(RangeReader::new( - acc, - "x", - OpRead::default().with_range(BytesRange::from(..)), - )) as oio::Reader; - let r = Box::new(BufferReader::new(r, 10)) as oio::Reader; - let mut r = Reader::new(r); - - let mut cur = 0; - for _ in 0..3 { - let bs = r.read(5).await?; - assert_eq!(bs.len(), 5); - cur += 5; - } - - let ret_cur = r.seek(SeekFrom::Current(-15)).await?; - assert_eq!(cur - 15, ret_cur); - - Ok(()) - } - - #[tokio::test] - async fn test_bypass_read_and_seek_relative() -> anyhow::Result<()> { - let bs = Bytes::copy_from_slice( - &b"Hello, World! I'm going to tests a seek relative related bug!"[..], - ); - let acc = Arc::new(MockReadService::new(bs.clone())); - let r = Box::new(RangeReader::new( - acc, - "x", - OpRead::default().with_range(BytesRange::from(..)), - )) as oio::Reader; - let r = Box::new(BufferReader::new(r, 5)) as oio::Reader; - let mut r = Reader::new(r); - - let mut cur = 0; - for _ in 0..3 { - let bs = r.read(6).await?; - assert_eq!(bs.len(), 6); - cur += 6; - } - - let ret_cur = r.seek(SeekFrom::Current(6)).await?; - assert_eq!(cur + 6, ret_cur); - - Ok(()) - } - - #[tokio::test] - async fn test_read_part() -> anyhow::Result<()> { - let (bs, _) = gen_bytes(); - let acc = Arc::new(MockReadService::new(bs.clone())); - - let r = Box::new(RangeReader::new( - acc, - "x", - OpRead::default().with_range(BytesRange::from(4096..4096 + 4096)), - )) as oio::Reader; - let r = Box::new(BufferReader::new(r, 4096 * 1024)) as oio::Reader; - let mut r = Reader::new(r); - - let mut buf = Vec::new(); - r.read_to_end(&mut buf).await?; - assert_eq!(4096, buf.len(), "read size"); - assert_eq!( - format!("{:x}", Sha256::digest(&bs[4096..4096 + 4096])), - format!("{:x}", Sha256::digest(&buf)), - "read content" - ); - - let n = r.seek(SeekFrom::Start(0)).await?; - assert_eq!(n, 0, "seek position must be 0"); - - let mut buf = Vec::new(); - r.read_to_end(&mut buf).await?; - assert_eq!(4096, buf.len(), "read twice size"); - assert_eq!( - format!("{:x}", Sha256::digest(&bs[4096..4096 + 4096])), - format!("{:x}", Sha256::digest(&buf)), - "read twice content" - ); - - let n = r.seek(SeekFrom::Start(1024)).await?; - assert_eq!(1024, n, "seek to 1024"); - - let buf = r.read_exact(1024).await?; - assert_eq!( - format!("{:x}", Sha256::digest(&bs[4096 + 1024..4096 + 2048])), - format!("{:x}", Sha256::digest(&buf)), - "read after seek 1024" - ); - - let n = r.seek(SeekFrom::Current(1024)).await?; - assert_eq!(3072, n, "seek to 3072"); - - let buf = r.read_exact(1024).await?; - assert_eq!( - format!("{:x}", Sha256::digest(&bs[4096 + 3072..4096 + 3072 + 1024])), - format!("{:x}", Sha256::digest(&buf)), - "read after seek to 3072" - ); - - Ok(()) - } - - #[tokio::test] - async fn test_blocking_read_from_buf() -> anyhow::Result<()> { - let bs = Bytes::copy_from_slice(&b"Hello, World!"[..]); - let r = Box::new(oio::Cursor::from(bs.clone())) as oio::BlockingReader; - let buf_cap = 10; - let r = Box::new(BufferReader::new(r, buf_cap)) as oio::BlockingReader; - let mut r = BlockingReader::new(r); - - let buf = r.read(5)?; - assert_eq!(buf.len(), 5); - assert_eq!(buf.as_ref(), b"Hello"); - - let buf = r.read(5)?; - assert_eq!(buf.len(), 5); - assert_eq!(buf.as_ref(), b", Wor"); - - let buf = r.read(3)?; - assert_eq!(buf.len(), 3); - assert_eq!(buf.as_ref(), b"ld!"); - - Ok(()) - } - - #[tokio::test] - async fn test_blocking_seek() -> anyhow::Result<()> { - let bs = Bytes::copy_from_slice(&b"Hello, World!"[..]); - let r = Box::new(oio::Cursor::from(bs.clone())) as oio::BlockingReader; - let buf_cap = 10; - let r = Box::new(BufferReader::new(r, buf_cap)) as oio::BlockingReader; - let mut r = BlockingReader::new(r); - - // The underlying reader buffers the b"Hello, Wor". - let buf = r.read(5)?; - assert_eq!(buf.len(), 5); - assert_eq!(buf.as_ref(), b"Hello"); - - let pos = r.seek(SeekFrom::Start(7))?; - assert_eq!(pos, 7); - let buf = r.read(5)?; - assert_eq!(&buf[..], &bs[7..10]); - assert_eq!(buf.len(), 3); - - // Should perform a relative seek. - let pos = r.seek(SeekFrom::Start(0))?; - assert_eq!(pos, 0); - let buf = r.read(9)?; - assert_eq!(&buf[..], &bs[0..9]); - assert_eq!(buf.len(), 9); - - // Should perform a non-relative seek. - let pos = r.seek(SeekFrom::Start(11))?; - assert_eq!(pos, 11); - let buf = r.read(9)?; - assert_eq!(&buf[..], &bs[11..13]); - assert_eq!(buf.len(), 2); - - Ok(()) - } - - #[tokio::test] - async fn test_blocking_read_all() -> anyhow::Result<()> { - let (bs, _) = gen_bytes(); - let r = Box::new(oio::Cursor::from(bs.clone())) as oio::BlockingReader; - let r = Box::new(BufferReader::new(r, 4096 * 1024)) as oio::BlockingReader; - let mut r = BlockingReader::new(r); - - let mut buf = Vec::new(); - r.read_to_end(&mut buf)?; - assert_eq!(bs.len(), buf.len(), "read size"); - assert_eq!( - format!("{:x}", Sha256::digest(&bs)), - format!("{:x}", Sha256::digest(&buf)), - "read content" - ); - - let n = r.seek(SeekFrom::Start(0))?; - assert_eq!(n, 0, "seek position must be 0"); - - let mut buf = Vec::new(); - r.read_to_end(&mut buf)?; - assert_eq!(bs.len(), buf.len(), "read twice size"); - assert_eq!( - format!("{:x}", Sha256::digest(&bs)), - format!("{:x}", Sha256::digest(&buf)), - "read twice content" - ); - - Ok(()) - } - - #[tokio::test] - async fn test_blocking_bypass_seek_relative() -> anyhow::Result<()> { - let bs = Bytes::copy_from_slice( - &b"Hello, World! I'm going to tests a seek relative related bug!"[..], - ); - let r = Box::new(oio::Cursor::from(bs.clone())) as oio::BlockingReader; - let r = Box::new(BufferReader::new(r, 10)) as oio::BlockingReader; - let mut r = BlockingReader::new(r); - - let mut cur = 0; - for _ in 0..3 { - let bs = r.read(5)?; - assert_eq!(bs.len(), 5); - cur += 5; - } - - let ret_cur = r.seek(SeekFrom::Current(-15))?; - assert_eq!(cur - 15, ret_cur); - - Ok(()) - } - - #[tokio::test] - async fn test_blocking_bypass_read_and_seek_relative() -> anyhow::Result<()> { - let bs = Bytes::copy_from_slice( - &b"Hello, World! I'm going to tests a seek relative related bug!"[..], - ); - let r = Box::new(oio::Cursor::from(bs.clone())) as oio::BlockingReader; - let r = Box::new(BufferReader::new(r, 5)) as oio::BlockingReader; - let mut r = BlockingReader::new(r); - - let mut cur = 0; - for _ in 0..3 { - let bs = r.read(6)?; - assert_eq!(bs.len(), 6); - cur += 6; - } - - let ret_cur = r.seek(SeekFrom::Current(6))?; - assert_eq!(cur + 6, ret_cur); - - Ok(()) - } - - #[tokio::test] - async fn test_blocking_read_part() -> anyhow::Result<()> { - let (bs, _) = gen_bytes(); - let acc = Arc::new(MockReadService::new(bs.clone())); - let r = Box::new(RangeReader::new( - acc, - "x", - OpRead::default().with_range(BytesRange::from(4096..4096 + 4096)), - )) as oio::BlockingReader; - let r = Box::new(BufferReader::new(r, 4096 * 1024)) as oio::BlockingReader; - let mut r = BlockingReader::new(r); - - let mut buf = Vec::new(); - r.read_to_end(&mut buf)?; - assert_eq!(4096, buf.len(), "read size"); - assert_eq!( - format!("{:x}", Sha256::digest(&bs[4096..4096 + 4096])), - format!("{:x}", Sha256::digest(&buf)), - "read content" - ); - - let n = r.seek(SeekFrom::Start(0))?; - assert_eq!(n, 0, "seek position must be 0"); - - let mut buf = Vec::new(); - r.read_to_end(&mut buf)?; - assert_eq!(4096, buf.len(), "read twice size"); - assert_eq!( - format!("{:x}", Sha256::digest(&bs[4096..4096 + 4096])), - format!("{:x}", Sha256::digest(&buf)), - "read twice content" - ); - - let n = r.seek(SeekFrom::Start(1024))?; - assert_eq!(1024, n, "seek to 1024"); - - let buf = r.read_exact(1024)?; - assert_eq!( - format!("{:x}", Sha256::digest(&bs[4096 + 1024..4096 + 2048])), - format!("{:x}", Sha256::digest(&buf)), - "read after seek 1024" - ); - - let n = r.seek(SeekFrom::Current(1024))?; - assert_eq!(3072, n, "seek to 3072"); - - let buf = r.read_exact(1024)?; - assert_eq!( - format!("{:x}", Sha256::digest(&bs[4096 + 3072..4096 + 3072 + 1024])), - format!("{:x}", Sha256::digest(&buf)), - "read after seek to 3072" - ); - - Ok(()) - } -} diff --git a/core/src/types/reader.rs b/core/src/types/reader.rs index 8b94af8f581b..36a328b21d4e 100644 --- a/core/src/types/reader.rs +++ b/core/src/types/reader.rs @@ -162,7 +162,7 @@ impl Reader { self.read_to_end_at(buf, 0).await } - pub async fn read_to_end_at(&self, buf: &mut impl BufMut, mut offset: u64) -> Result { + pub async fn read_to_end_at(&self, buf: &mut imp l BufMut, mut offset: u64) -> Result { let mut size = 0; loop { // TODO: io size should be tuned based on storage From 02c02e6157b4bdcb172e4bf85c3da8f69f03d2a7 Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Wed, 20 Mar 2024 18:13:59 +0800 Subject: [PATCH 048/111] Build pass! Signed-off-by: Xuanwo --- core/benches/ops/read.rs | 5 +- core/src/types/blocking_reader.rs | 151 ++++++++ core/src/types/mod.rs | 5 +- core/src/types/operator/operator.rs | 10 +- core/src/types/reader.rs | 575 +++++++++++++--------------- core/tests/behavior/async_read.rs | 2 +- core/tests/behavior/main.rs | 1 - 7 files changed, 433 insertions(+), 316 deletions(-) create mode 100644 core/src/types/blocking_reader.rs diff --git a/core/benches/ops/read.rs b/core/benches/ops/read.rs index 401e72ff7b80..4ee2fd9117f6 100644 --- a/core/benches/ops/read.rs +++ b/core/benches/ops/read.rs @@ -56,6 +56,7 @@ fn bench_read_full(c: &mut Criterion, name: &str, op: Operator) { .range(0..=size.bytes() as u64) .await .unwrap(); + let r = r.into_futures_read(); io::copy(r, &mut io::sink()).await.unwrap(); }) }); @@ -87,6 +88,7 @@ fn bench_read_part(c: &mut Criterion, name: &str, op: Operator) { group.bench_with_input(size.to_string(), &(op.clone(), &path), |b, (op, path)| { b.to_async(&*TEST_RUNTIME).iter(|| async { let r = op.reader_with(path).range(offset..).await.unwrap(); + let r = r.into_futures_read(); io::copy(r, &mut io::sink()).await.unwrap(); }) }); @@ -128,7 +130,8 @@ fn bench_read_parallel(c: &mut Criterion, name: &str, op: Operator) { .range(offset..=offset + size.bytes() as u64) .await .unwrap(); - r.read_exact(*buf_size).await.unwrap(); + todo!(); + // r.read_exact(*buf_size).await.unwrap(); let mut d = 0; // mock same little cpu work diff --git a/core/src/types/blocking_reader.rs b/core/src/types/blocking_reader.rs new file mode 100644 index 000000000000..bccb8cd1c09e --- /dev/null +++ b/core/src/types/blocking_reader.rs @@ -0,0 +1,151 @@ +// 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; +use std::io::SeekFrom; +use std::ops::{Range, RangeBounds}; +use std::pin::Pin; +use std::task::ready; +use std::task::Context; +use std::task::Poll; + +use bytes::{Buf, BufMut, Bytes, BytesMut}; +use futures::Stream; +use tokio::io::ReadBuf; + +use crate::raw::oio::BlockingRead; +use crate::raw::*; +use crate::*; + +/// BlockingReader is designed to read data from given path in an blocking +/// manner. +pub struct BlockingReader { + pub(crate) inner: oio::BlockingReader, +} + +impl BlockingReader { + /// Create a new blocking reader. + /// + /// Create will use internal information to decide the most suitable + /// implementation for users. + /// + /// We don't want to expose those details to users so keep this function + /// in crate only. + pub(crate) fn create(acc: FusedAccessor, path: &str, op: OpRead) -> crate::Result { + let (_, r) = acc.blocking_read(path, op)?; + + Ok(BlockingReader { inner: r }) + } + + /// Create a new reader from an `oio::BlockingReader`. + pub(crate) fn new(r: oio::BlockingReader) -> Self { + BlockingReader { inner: r } + } + + /// Read given range bytes of data from reader. + pub fn read_at(&self, buf: &mut impl BufMut, offset: u64) -> crate::Result { + let bs = self.inner.read_at(offset, buf.remaining_mut())?; + let n = bs.remaining(); + buf.put(bs); + Ok(n) + } + + /// Read given range bytes of data from reader. + pub fn read_range(&self, buf: &mut impl BufMut, range: Range) -> crate::Result { + if range.is_empty() { + return Ok(0); + } + let (mut offset, mut size) = (range.start, range.end - range.start); + + let mut read = 0; + + loop { + let bs = self.inner.read_at(offset, size as usize)?; + let n = bs.remaining(); + read += n; + buf.put(bs); + if n == 0 { + return Ok(read); + } + + offset += n as u64; + + debug_assert!( + size >= n as u64, + "read should not return more bytes than expected" + ); + size -= n as u64; + if size == 0 { + return Ok(read); + } + } + } + + pub fn read_to_end(&self, buf: &mut impl BufMut) -> crate::Result { + self.read_to_end_at(buf, 0) + } + + pub fn read_to_end_at(&self, buf: &mut impl BufMut, mut offset: u64) -> crate::Result { + let mut size = 0; + loop { + // TODO: io size should be tuned based on storage + let bs = self.inner.read_at(offset, 4 * 1024 * 1024)?; + let n = bs.remaining(); + size += n; + + buf.put(bs); + if n == 0 { + return Ok(size); + } + + offset += n as u64; + } + } +} + +// impl io::Read for BlockingReader { +// #[inline] +// fn read(&mut self, buf: &mut [u8]) -> io::Result { +// let bs = self.inner.read(buf.len()).map_err(format_std_io_error)?; +// buf[..bs.len()].copy_from_slice(&bs); +// Ok(bs.len()) +// } +// } +// +// impl io::Seek for BlockingReader { +// #[inline] +// fn seek(&mut self, pos: io::SeekFrom) -> io::Result { +// self.inner.seek(pos).map_err(format_std_io_error) +// } +// } +// +// impl Iterator for BlockingReader { +// type Item = io::Result; +// +// #[inline] +// fn next(&mut self) -> Option { +// match self +// .inner +// .read(4 * 1024 * 1024) +// .map_err(format_std_io_error) +// { +// Ok(bs) if bs.is_empty() => None, +// Ok(bs) => Some(Ok(bs)), +// Err(err) => Some(Err(err)), +// } +// } +// } diff --git a/core/src/types/mod.rs b/core/src/types/mod.rs index 0f75fa50c842..804334f6e299 100644 --- a/core/src/types/mod.rs +++ b/core/src/types/mod.rs @@ -26,9 +26,11 @@ pub use metadata::Metadata; pub use metadata::Metakey; mod reader; -pub use reader::BlockingReader; pub use reader::Reader; +mod blocking_reader; +pub use blocking_reader::BlockingReader; + mod writer; pub use writer::BlockingWriter; pub use writer::Writer; @@ -57,4 +59,5 @@ mod scheme; pub use scheme::Scheme; mod capability; + pub use capability::Capability; diff --git a/core/src/types/operator/operator.rs b/core/src/types/operator/operator.rs index 0d864c4ced73..f7f3c3c8bc0f 100644 --- a/core/src/types/operator/operator.rs +++ b/core/src/types/operator/operator.rs @@ -524,12 +524,10 @@ impl Operator { (range.size().unwrap(), range) }; - let (_, r) = inner.read(&path, args.with_range(range)).await?; - let mut r = Reader::new(r); - // let mut buf = Vec::with_capacity(size_hint as usize); - // r.read_to_end(&mut buf).await?; - todo!() - // Ok(buf) + let r = Reader::create(inner, &path, args.with_range(range)).await?; + let mut buf = Vec::with_capacity(size_hint as usize); + r.read_to_end(&mut buf).await?; + Ok(buf) }, ) } diff --git a/core/src/types/reader.rs b/core/src/types/reader.rs index 36a328b21d4e..3e452190736e 100644 --- a/core/src/types/reader.rs +++ b/core/src/types/reader.rs @@ -66,15 +66,14 @@ use crate::*; /// /// [`Reader`] provides [`Reader::into_tokio_read`] to remove extra APIs upon self. pub struct Reader { + acc: FusedAccessor, + path: String, + op: OpRead, + inner: oio::Reader, } impl Reader { - /// Create a new reader from an `oio::Reader`. - pub(crate) fn new(r: oio::Reader) -> Self { - Reader { inner: r } - } - /// Create a new reader. /// /// Create will use internal information to decide the most suitable @@ -83,9 +82,14 @@ impl Reader { /// We don't want to expose those details to users so keep this function /// in crate only. pub(crate) async fn create(acc: FusedAccessor, path: &str, op: OpRead) -> Result { - let (_, r) = acc.read(path, op).await?; - - Ok(Reader { inner: r }) + let (_, r) = acc.read(path, op.clone()).await?; + + Ok(Reader { + acc, + path: path.to_string(), + op, + inner: r, + }) } /// Convert [`Reader`] into an [`futures::AsyncRead`] and [`futures::AsyncSeek`] @@ -95,14 +99,13 @@ impl Reader { /// /// The returning type also implements `Send`, `Sync` and `Unpin`, so users can use it /// as `Box` and calling `poll_read_unpin` on it. - // #[inline] - // #[cfg(not(target_arch = "wasm32"))] - // pub fn into_futures_read( - // self, - // ) -> impl futures::AsyncRead + futures::AsyncSeek + Send + Sync + Unpin { - // // self - // todo!() - // } + #[inline] + #[cfg(not(target_arch = "wasm32"))] + pub fn into_futures_read( + self, + ) -> impl futures::AsyncRead + futures::AsyncSeek + Send + Sync + Unpin { + futures_io_adapter::FuturesReader::new(self.acc, self.path, self.op) + } /// Convert [`Reader`] into an [`tokio::io::AsyncRead`] and [`tokio::io::AsyncSeek`] /// @@ -162,7 +165,7 @@ impl Reader { self.read_to_end_at(buf, 0).await } - pub async fn read_to_end_at(&self, buf: &mut imp l BufMut, mut offset: u64) -> Result { + pub async fn read_to_end_at(&self, buf: &mut impl BufMut, mut offset: u64) -> Result { let mut size = 0; loop { // TODO: io size should be tuned based on storage @@ -180,322 +183,282 @@ impl Reader { } } -enum State { - Idle(Option), - Reading(BoxedStaticFuture<(oio::Reader, Result)>), - Seeking(BoxedStaticFuture<(oio::Reader, Result)>), -} +mod futures_io_adapter { + use super::*; + use futures::io::{AsyncRead, AsyncSeek}; + use std::pin::Pin; + use std::task::{Context, Poll}; -/// # Safety -/// -/// Reader will only be used with `&mut self`. -unsafe impl Sync for State {} - -// impl futures::AsyncRead for Reader { -// fn poll_read( -// mut self: Pin<&mut Self>, -// cx: &mut Context<'_>, -// buf: &mut [u8], -// ) -> Poll> { -// use oio::Read; -// -// match &mut self.state { -// State::Idle(r) => { -// let mut r = r.take().expect("reader must be valid"); -// let size = buf.len(); -// let fut = async move { -// let res = r.read(size).await; -// (r, res) -// }; -// self.state = State::Reading(Box::pin(fut)); -// self.poll_read(cx, buf) -// } -// State::Reading(fut) => { -// let (r, res) = ready!(fut.as_mut().poll(cx)); -// self.state = State::Idle(Some(r)); -// let bs = res.map_err(format_std_io_error)?; -// let n = bs.len(); -// buf[..n].copy_from_slice(&bs); -// Poll::Ready(Ok(n)) -// } -// State::Seeking(_) => Poll::Ready(Err(io::Error::new( -// io::ErrorKind::Interrupted, -// "another io operation is in progress", -// ))), -// } -// } -// } - -// impl futures::AsyncSeek for Reader { -// fn poll_seek( -// mut self: Pin<&mut Self>, -// cx: &mut Context<'_>, -// pos: io::SeekFrom, -// ) -> Poll> { -// use oio::Read; -// -// match &mut self.state { -// State::Idle(r) => { -// let mut r = r.take().expect("reader must be valid"); -// let fut = async move { -// let res = r.seek(pos).await; -// (r, res) -// }; -// self.state = State::Seeking(Box::pin(fut)); -// self.poll_seek(cx, pos) -// } -// State::Seeking(fut) => { -// let (r, res) = ready!(fut.as_mut().poll(cx)); -// self.state = State::Idle(Some(r)); -// Poll::Ready(res.map_err(format_std_io_error)) -// } -// State::Reading(_) => Poll::Ready(Err(io::Error::new( -// io::ErrorKind::Interrupted, -// "another io operation is in progress", -// ))), -// } -// } -// } -// -// impl tokio::io::AsyncRead for Reader { -// fn poll_read( -// mut self: Pin<&mut Self>, -// cx: &mut Context<'_>, -// buf: &mut tokio::io::ReadBuf<'_>, -// ) -> Poll> { -// use oio::Read; -// -// loop { -// match &mut self.state { -// State::Idle(r) => { -// // Safety: We make sure that we will set filled correctly. -// unsafe { buf.assume_init(buf.remaining()) } -// let size = buf.initialize_unfilled().len(); -// -// let mut r = r.take().expect("reader must be valid"); -// let fut = async move { -// let res = r.read(size).await; -// (r, res) -// }; -// self.state = State::Reading(Box::pin(fut)); -// } -// State::Reading(fut) => { -// let (r, res) = ready!(fut.as_mut().poll(cx)); -// self.state = State::Idle(Some(r)); -// let bs = res.map_err(format_std_io_error)?; -// let n = bs.len(); -// buf.initialize_unfilled()[..n].copy_from_slice(&bs); -// buf.advance(n); -// return Poll::Ready(Ok(())); -// } -// State::Seeking(_) => { -// return Poll::Ready(Err(io::Error::new( -// io::ErrorKind::Interrupted, -// "another io operation is in progress", -// ))) -// } -// } -// } -// } -// } - -// impl tokio::io::AsyncSeek for Reader { -// fn start_seek(mut self: Pin<&mut Self>, pos: io::SeekFrom) -> io::Result<()> { -// use oio::Read; -// -// match &mut self.state { -// State::Idle(r) => { -// let mut r = r.take().expect("reader must be valid"); -// let fut = async move { -// let res = r.seek(pos).await; -// (r, res) -// }; -// self.state = State::Seeking(Box::pin(fut)); -// Ok(()) -// } -// State::Seeking(_) | State::Reading(_) => Err(io::Error::new( -// io::ErrorKind::Interrupted, -// "another io operation is in progress", -// )), -// } -// } -// -// fn poll_complete(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { -// match &mut self.state { -// State::Idle(_) => { -// // AsyncSeek recommends calling poll_complete before start_seek. -// // We don't have to guarantee that the value returned by -// // poll_complete called without start_seek is correct, -// // so we'll return 0. -// Poll::Ready(Ok(0)) -// } -// State::Seeking(fut) => { -// let (r, res) = ready!(fut.as_mut().poll(cx)); -// self.state = State::Idle(Some(r)); -// Poll::Ready(res.map_err(format_std_io_error)) -// } -// State::Reading(_) => Poll::Ready(Err(io::Error::new( -// io::ErrorKind::Interrupted, -// "another io operation is in progress", -// ))), -// } -// } -// } - -// impl Stream for Reader { -// type Item = io::Result; -// -// fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { -// use oio::Read; -// -// match &mut self.state { -// State::Idle(r) => { -// let mut r = r.take().expect("reader must be valid"); -// let fut = async move { -// // TODO: should allow user to tune this value. -// let res = r.read(4 * 1024 * 1024).await; -// (r, res) -// }; -// self.state = State::Reading(Box::pin(fut)); -// self.poll_next(cx) -// } -// State::Reading(fut) => { -// let (r, res) = ready!(fut.as_mut().poll(cx)); -// self.state = State::Idle(Some(r)); -// let bs = res.map_err(format_std_io_error)?; -// if bs.is_empty() { -// Poll::Ready(None) -// } else { -// Poll::Ready(Some(Ok(bs))) -// } -// } -// State::Seeking(_) => Poll::Ready(Some(Err(io::Error::new( -// io::ErrorKind::Interrupted, -// "another io operation is in progress", -// )))), -// } -// } -// } - -/// BlockingReader is designed to read data from given path in an blocking -/// manner. -pub struct BlockingReader { - pub(crate) inner: oio::BlockingReader, -} + /// TODO: we can implement async buf read. + pub struct FuturesReader { + pub acc: FusedAccessor, + pub path: String, + pub op: OpRead, -impl BlockingReader { - /// Create a new blocking reader. - /// - /// Create will use internal information to decide the most suitable - /// implementation for users. - /// - /// We don't want to expose those details to users so keep this function - /// in crate only. - pub(crate) fn create(acc: FusedAccessor, path: &str, op: OpRead) -> Result { - let (_, r) = acc.blocking_read(path, op)?; + pub state: State, + pub offset: u64, + } - Ok(BlockingReader { inner: r }) + enum State { + Idle(Option), + Stating(BoxedStaticFuture>), + Reading(BoxedStaticFuture<(oio::Reader, Result)>), } - /// Create a new reader from an `oio::BlockingReader`. - pub(crate) fn new(r: oio::BlockingReader) -> Self { - BlockingReader { inner: r } + /// # Safety + /// + /// Reader will only be used with `&mut self`. + unsafe impl Sync for State {} + + impl FuturesReader { + pub fn new(acc: FusedAccessor, path: String, op: OpRead) -> Self { + Self { + acc, + path, + op, + state: State::Idle(None), + offset: 0, + } + } } - /// Read given range bytes of data from reader. - pub fn read_at(&self, buf: &mut impl BufMut, offset: u64) -> Result { - let bs = self.inner.read_at(offset, buf.remaining_mut())?; - let n = bs.remaining(); - buf.put(bs); - Ok(n) + impl AsyncRead for FuturesReader { + fn poll_read( + mut self: Pin<&mut Self>, + cx: &mut Context<'_>, + mut buf: &mut [u8], + ) -> Poll> { + use oio::Read; + + match &mut self.state { + State::Idle(r) => { + let mut r = r.take().expect("reader must be valid"); + let size = buf.len(); + let offset = self.offset; + let fut = async move { + let res = r.read_at(offset, size).await; + (r, res) + }; + self.state = State::Reading(Box::pin(fut)); + self.poll_read(cx, buf) + } + State::Reading(fut) => { + let (r, res) = ready!(fut.as_mut().poll(cx)); + self.state = State::Idle(Some(r)); + let bs = res.map_err(format_std_io_error)?; + let n = bs.remaining(); + buf.put(bs); + self.offset += n as u64; + Poll::Ready(Ok(n)) + } + State::Stating(_) => Poll::Ready(Err(io::Error::new( + io::ErrorKind::Interrupted, + "another io operation is in progress", + ))), + } + } } - /// Read given range bytes of data from reader. - pub fn read_range(&self, buf: &mut impl BufMut, range: Range) -> Result { - if range.is_empty() { - return Ok(0); + impl AsyncSeek for FuturesReader { + fn poll_seek( + mut self: Pin<&mut Self>, + cx: &mut Context<'_>, + pos: io::SeekFrom, + ) -> Poll> { + use oio::Read; + + match &mut self.state { + State::Idle(_) => match pos { + SeekFrom::Start(n) => { + self.offset = n; + Poll::Ready(Ok(n)) + } + SeekFrom::End(_) => todo!(), + SeekFrom::Current(amt) => { + let offset = self.offset as i64 + amt; + if offset < 0 { + return Poll::Ready(Err(io::Error::new( + io::ErrorKind::InvalidInput, + "invalid seek to a negative position", + ))); + } + self.offset = offset as u64; + Poll::Ready(Ok(self.offset)) + } + }, + State::Stating(fut) => { + todo!() + } + State::Reading(_) => Poll::Ready(Err(io::Error::new( + io::ErrorKind::Interrupted, + "another io operation is in progress", + ))), + } } - let (mut offset, mut size) = (range.start, range.end - range.start); + } +} - let mut read = 0; +mod tokio_io_adapter { + use super::*; + use std::pin::Pin; + use std::task::{Context, Poll}; + use tokio::io::{AsyncRead, AsyncSeek}; - loop { - let bs = self.inner.read_at(offset, size as usize)?; - let n = bs.remaining(); - read += n; - buf.put(bs); - if n == 0 { - return Ok(read); - } + /// TODO: we can implement async buf read. + pub struct TokioReader { + acc: FusedAccessor, + path: String, + op: OpRead, - offset += n as u64; + state: State, + offset: u64, + } - debug_assert!( - size >= n as u64, - "read should not return more bytes than expected" - ); - size -= n as u64; - if size == 0 { - return Ok(read); + enum State { + Idle(Option), + Stating(BoxedStaticFuture>), + Reading(BoxedStaticFuture<(oio::Reader, Result)>), + } + + /// # Safety + /// + /// Reader will only be used with `&mut self`. + unsafe impl Sync for State {} + + impl AsyncRead for TokioReader { + fn poll_read( + mut self: Pin<&mut Self>, + cx: &mut Context<'_>, + buf: &mut ReadBuf<'_>, + ) -> Poll> { + use oio::Read; + + match &mut self.state { + State::Idle(r) => { + let mut r = r.take().expect("reader must be valid"); + let size = buf.remaining_mut(); + let offset = self.offset; + let fut = async move { + let res = r.read_at(offset, size).await; + (r, res) + }; + self.state = State::Reading(Box::pin(fut)); + self.poll_read(cx, buf) + } + State::Reading(fut) => { + let (r, res) = ready!(fut.as_mut().poll(cx)); + self.state = State::Idle(Some(r)); + let bs = res.map_err(format_std_io_error)?; + let n = bs.remaining(); + buf.put(bs); + self.offset += n as u64; + Poll::Ready(Ok(())) + } + State::Stating(_) => Poll::Ready(Err(io::Error::new( + io::ErrorKind::Interrupted, + "another io operation is in progress", + ))), + } + } + } + + impl AsyncSeek for TokioReader { + fn start_seek(mut self: Pin<&mut Self>, pos: io::SeekFrom) -> io::Result<()> { + match &mut self.state { + State::Idle(_) => match pos { + SeekFrom::Start(n) => { + self.offset = n; + Ok(()) + } + SeekFrom::End(_) => todo!(), + SeekFrom::Current(amt) => { + let offset = self.offset as i64 + amt; + if offset < 0 { + return Err(io::Error::new( + io::ErrorKind::InvalidInput, + "invalid seek to a negative position", + )); + } + self.offset = offset as u64; + Ok(()) + } + }, + State::Stating(fut) => { + todo!() + } + State::Reading(_) => Err(io::Error::new( + io::ErrorKind::Interrupted, + "another io operation is in progress", + )), } } + + fn poll_complete(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + Poll::Ready(Ok(self.offset)) + } } +} + +mod stream_adapter { + use super::*; + use crate::raw::{oio, BoxedStaticFuture, RpStat}; + use bytes::Bytes; + use futures::Stream; + use std::io; + use std::pin::Pin; - pub fn read_to_end(&self, buf: &mut impl BufMut) -> Result { - self.read_to_end_at(buf, 0) + pub struct StreamReader { + state: State, + + buffer: oio::Buffer, + offset: u64, } - pub fn read_to_end_at(&self, buf: &mut impl BufMut, mut offset: u64) -> Result { - let mut size = 0; - loop { - // TODO: io size should be tuned based on storage - let bs = self.inner.read_at(offset, 4 * 1024 * 1024)?; - let n = bs.remaining(); - size += n; + enum State { + Idle(Option), + Reading(BoxedStaticFuture<(oio::Reader, crate::Result)>), + } - buf.put(bs); - if n == 0 { - return Ok(size); + impl Stream for StreamReader { + type Item = io::Result; + + fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + if self.buffer.has_remaining() { + let n = self.buffer.chunk().len(); + let bs = self.buffer.copy_to_bytes(n); + return Poll::Ready(Some(Ok(bs))); } - offset += n as u64; + let offset = self.offset; + + match &mut self.state { + State::Idle(r) => { + let mut r = r.take().expect("reader must be valid"); + let fut = async move { + // TODO: should allow user to tune this value. + let res = r.read_at_dyn(offset, 4 * 1024 * 1024).await; + (r, res) + }; + self.state = State::Reading(Box::pin(fut)); + self.poll_next(cx) + } + State::Reading(fut) => { + let (r, res) = ready!(fut.as_mut().poll(cx)); + self.state = State::Idle(Some(r)); + let bs = res.map_err(format_std_io_error)?; + if bs.has_remaining() { + self.offset += bs.remaining() as u64; + self.buffer = bs; + self.poll_next(cx) + } else { + Poll::Ready(None) + } + } + } } } } -// impl io::Read for BlockingReader { -// #[inline] -// fn read(&mut self, buf: &mut [u8]) -> io::Result { -// let bs = self.inner.read(buf.len()).map_err(format_std_io_error)?; -// buf[..bs.len()].copy_from_slice(&bs); -// Ok(bs.len()) -// } -// } -// -// impl io::Seek for BlockingReader { -// #[inline] -// fn seek(&mut self, pos: io::SeekFrom) -> io::Result { -// self.inner.seek(pos).map_err(format_std_io_error) -// } -// } -// -// impl Iterator for BlockingReader { -// type Item = io::Result; -// -// #[inline] -// fn next(&mut self) -> Option { -// match self -// .inner -// .read(4 * 1024 * 1024) -// .map_err(format_std_io_error) -// { -// Ok(bs) if bs.is_empty() => None, -// Ok(bs) => Some(Ok(bs)), -// Err(err) => Some(Err(err)), -// } -// } -// } - #[cfg(test)] mod tests { use rand::rngs::ThreadRng; diff --git a/core/tests/behavior/async_read.rs b/core/tests/behavior/async_read.rs index 226d287d7a28..d19870742a9f 100644 --- a/core/tests/behavior/async_read.rs +++ b/core/tests/behavior/async_read.rs @@ -48,7 +48,7 @@ pub fn tests(op: &Operator, tests: &mut Vec) { test_read_with_special_chars, test_read_with_override_cache_control, test_read_with_override_content_disposition, - test_read_with_override_content_type, + test_read_with_override_content_type )) } diff --git a/core/tests/behavior/main.rs b/core/tests/behavior/main.rs index 20eb832c82b4..cf2bc5192959 100644 --- a/core/tests/behavior/main.rs +++ b/core/tests/behavior/main.rs @@ -65,7 +65,6 @@ fn main() -> anyhow::Result<()> { async_copy::tests(&op, &mut tests); async_create_dir::tests(&op, &mut tests); async_delete::tests(&op, &mut tests); - async_fuzz::tests(&op, &mut tests); async_list::tests(&op, &mut tests); async_presign::tests(&op, &mut tests); async_read::tests(&op, &mut tests); From d7b5b61a2a4a11d3ae9b893cbd894a26094c3513 Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Wed, 20 Mar 2024 18:14:57 +0800 Subject: [PATCH 049/111] Fix build Signed-off-by: Xuanwo --- core/src/types/reader.rs | 27 +++++++++++++++++++-------- 1 file changed, 19 insertions(+), 8 deletions(-) diff --git a/core/src/types/reader.rs b/core/src/types/reader.rs index 3e452190736e..572178fbaa89 100644 --- a/core/src/types/reader.rs +++ b/core/src/types/reader.rs @@ -114,14 +114,13 @@ impl Reader { /// /// The returning type also implements `Send`, `Sync` and `Unpin`, so users can use it /// as `Box` and calling `poll_read_unpin` on it. - // #[inline] - // #[cfg(not(target_arch = "wasm32"))] - // pub fn into_tokio_read( - // self, - // ) -> impl tokio::io::AsyncRead + tokio::io::AsyncSeek + Send + Sync + Unpin { - // // self - // todo!() - // } + #[inline] + #[cfg(not(target_arch = "wasm32"))] + pub fn into_tokio_read( + self, + ) -> impl tokio::io::AsyncRead + tokio::io::AsyncSeek + Send + Sync + Unpin { + tokio_io_adapter::TokioReader::new(self.acc, self.path, self.op) + } pub async fn read_at(&self, buf: &mut impl BufMut, offset: u64) -> Result { let bs = self.inner.read_at_dyn(offset, buf.remaining_mut()).await?; @@ -325,6 +324,18 @@ mod tokio_io_adapter { /// Reader will only be used with `&mut self`. unsafe impl Sync for State {} + impl TokioReader { + pub fn new(acc: FusedAccessor, path: String, op: OpRead) -> Self { + Self { + acc, + path, + op, + state: State::Idle(None), + offset: 0, + } + } + } + impl AsyncRead for TokioReader { fn poll_read( mut self: Pin<&mut Self>, From 8458c30125cd2e8f3846a85e3356f293ba90c106 Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Wed, 20 Mar 2024 18:16:34 +0800 Subject: [PATCH 050/111] Remove not needed api Signed-off-by: Xuanwo --- core/src/raw/oio/stream/into_stream.rs | 94 ------------------- .../raw/oio/stream/into_stream_from_reader.rs | 92 ------------------ core/src/raw/oio/stream/mod.rs | 6 -- 3 files changed, 192 deletions(-) delete mode 100644 core/src/raw/oio/stream/into_stream.rs delete mode 100644 core/src/raw/oio/stream/into_stream_from_reader.rs diff --git a/core/src/raw/oio/stream/into_stream.rs b/core/src/raw/oio/stream/into_stream.rs deleted file mode 100644 index 3e34bbf4bab5..000000000000 --- a/core/src/raw/oio/stream/into_stream.rs +++ /dev/null @@ -1,94 +0,0 @@ -// 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. - -#[cfg(not(target_arch = "wasm32"))] -pub use non_wasm32_impl::*; - -#[cfg(not(target_arch = "wasm32"))] -mod non_wasm32_impl { - use std::task::Context; - use std::task::Poll; - - use bytes::Bytes; - use futures::TryStreamExt; - - use crate::raw::oio; - - /// Convert given futures stream into [`oio::Stream`]. - pub fn into_stream(stream: S) -> IntoStream - where - S: futures::Stream> + Send + Sync + Unpin, - { - IntoStream { inner: stream } - } - - pub struct IntoStream { - inner: S, - } - - impl oio::Stream for IntoStream - where - S: futures::Stream> + Send + Sync + Unpin, - { - fn poll_next(&mut self, cx: &mut Context<'_>) -> Poll>> { - self.inner.try_poll_next_unpin(cx) - } - } -} - -#[cfg(target_arch = "wasm32")] -pub use wasm32_impl::*; -#[cfg(target_arch = "wasm32")] -mod wasm32_impl { - use std::task::Context; - use std::task::Poll; - - use bytes::Bytes; - use futures::TryStreamExt; - - use crate::raw::oio; - - /// Convert given futures stream into [`oio::Stream`]. - pub fn into_stream(stream: S) -> IntoStream - where - S: futures::Stream> + Unpin, - { - IntoStream { inner: stream } - } - - pub struct IntoStream { - inner: S, - } - - /// # Safety - /// - /// wasm32 is a special target that we only have one event-loop for this stream. - unsafe impl Send for IntoStream {} - /// # Safety - /// - /// IntoStream only has mutable references. - unsafe impl Sync for IntoStream {} - - impl oio::Stream for IntoStream - where - S: futures::Stream> + Unpin, - { - fn poll_next(&mut self, cx: &mut Context<'_>) -> Poll>> { - self.inner.try_poll_next_unpin(cx) - } - } -} diff --git a/core/src/raw/oio/stream/into_stream_from_reader.rs b/core/src/raw/oio/stream/into_stream_from_reader.rs deleted file mode 100644 index cead7d11d72d..000000000000 --- a/core/src/raw/oio/stream/into_stream_from_reader.rs +++ /dev/null @@ -1,92 +0,0 @@ -// 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::pin::Pin; -use std::task::ready; -use std::task::Context; -use std::task::Poll; - -use bytes::BufMut; -use bytes::Bytes; -use bytes::BytesMut; -use futures::AsyncRead; -use tokio::io::ReadBuf; - -use crate::raw::*; -use crate::*; - -// TODO: 64KiB is picked based on experiences, should be configurable -const DEFAULT_CAPACITY: usize = 64 * 1024; - -/// Convert given futures reader into [`oio::Stream`]. -pub fn into_stream_from_reader(r: R) -> FromReaderStream -where - R: AsyncRead + Send + Sync + Unpin, -{ - FromReaderStream { - inner: Some(r), - buf: BytesMut::new(), - } -} - -pub struct FromReaderStream { - inner: Option, - buf: BytesMut, -} - -impl oio::Stream for FromReaderStream -where - S: AsyncRead + Send + Sync + Unpin, -{ - fn poll_next(&mut self, cx: &mut Context<'_>) -> Poll>> { - let reader = match self.inner.as_mut() { - Some(r) => r, - None => return Poll::Ready(None), - }; - - if self.buf.capacity() == 0 { - self.buf.reserve(DEFAULT_CAPACITY); - } - - let dst = self.buf.spare_capacity_mut(); - let mut buf = ReadBuf::uninit(dst); - - // Safety: the buf must contains enough space for reading - unsafe { buf.assume_init(buf.capacity()) }; - - match ready!(Pin::new(reader).poll_read(cx, buf.initialized_mut())) { - Ok(0) => { - // Set inner to None while reaching EOF. - self.inner = None; - Poll::Ready(None) - } - Ok(n) => { - // Safety: read_exact makes sure this buffer has been filled. - unsafe { self.buf.advance_mut(n) } - - let chunk = self.buf.split(); - Poll::Ready(Some(Ok(chunk.freeze()))) - } - Err(err) => Poll::Ready(Some(Err(Error::new( - ErrorKind::Unexpected, - "read data from reader into stream", - ) - .set_temporary() - .set_source(err)))), - } - } -} diff --git a/core/src/raw/oio/stream/mod.rs b/core/src/raw/oio/stream/mod.rs index c71d243cea11..572724dab92f 100644 --- a/core/src/raw/oio/stream/mod.rs +++ b/core/src/raw/oio/stream/mod.rs @@ -19,9 +19,3 @@ mod api; pub use api::Stream; pub use api::StreamExt; pub use api::Streamer; - -mod into_stream_from_reader; -pub use into_stream_from_reader::into_stream_from_reader; - -mod into_stream; -pub use into_stream::into_stream; From 3fb9bcb1bedfe99db0c31155258fe1048e0f3ad9 Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Wed, 20 Mar 2024 18:18:14 +0800 Subject: [PATCH 051/111] FIx tests Signed-off-by: Xuanwo --- core/tests/behavior/async_fuzz.rs | 6 +++--- core/tests/behavior/async_read.rs | 2 +- core/tests/behavior/main.rs | 2 +- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/core/tests/behavior/async_fuzz.rs b/core/tests/behavior/async_fuzz.rs index 9f27db993f96..2dc3647914b5 100644 --- a/core/tests/behavior/async_fuzz.rs +++ b/core/tests/behavior/async_fuzz.rs @@ -25,9 +25,9 @@ use opendal::raw::BytesRange; use crate::*; -// pub fn tests(op: &Operator, tests: &mut Vec) { -// tests.extend(async_trials!(op)) -// } +pub fn tests(op: &Operator, tests: &mut Vec) { + tests.extend(async_trials!(op)) +} async fn test_fuzz_read( op: Operator, diff --git a/core/tests/behavior/async_read.rs b/core/tests/behavior/async_read.rs index d19870742a9f..e64ae883493b 100644 --- a/core/tests/behavior/async_read.rs +++ b/core/tests/behavior/async_read.rs @@ -670,7 +670,7 @@ pub async fn test_read_only_reader_from(op: Operator) -> anyhow::Result<()> { /// Read tail should match. pub async fn test_read_only_reader_tail(op: Operator) -> anyhow::Result<()> { - let mut r = op.reader_with("normal_file.txt").range(..1024).await?; + let r = op.reader_with("normal_file.txt").range(..1024).await?; let mut bs = Vec::new(); r.read_to_end(&mut bs).await?; diff --git a/core/tests/behavior/main.rs b/core/tests/behavior/main.rs index cf2bc5192959..666d36a72227 100644 --- a/core/tests/behavior/main.rs +++ b/core/tests/behavior/main.rs @@ -24,7 +24,7 @@ pub use utils::*; mod async_copy; mod async_create_dir; mod async_delete; -mod async_fuzz; +// mod async_fuzz; mod async_list; mod async_presign; mod async_read; From 8f2c11e2d8a4835139b510c08581c1a0b00b8a9e Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Thu, 21 Mar 2024 13:47:51 +0800 Subject: [PATCH 052/111] Save new api Signed-off-by: Xuanwo --- core/src/layers/retry.rs | 2 +- core/src/layers/timeout.rs | 2 +- core/src/raw/tests/read.rs | 4 +- core/src/types/blocking_reader.rs | 71 +++++++++--------- core/src/types/operator/blocking_operator.rs | 2 +- core/src/types/reader.rs | 78 +++++++++++--------- 6 files changed, 85 insertions(+), 74 deletions(-) diff --git a/core/src/layers/retry.rs b/core/src/layers/retry.rs index 9ceea0bec931..b266c525b20f 100644 --- a/core/src/layers/retry.rs +++ b/core/src/layers/retry.rs @@ -1119,7 +1119,7 @@ mod tests { let mut r = op.reader("retryable_error").await.unwrap(); let mut content = Vec::new(); let size = r - .read_to_end_at(&mut content, 0) + .read_to_end(&mut content) .await .expect("read must succeed"); assert_eq!(size, 13); diff --git a/core/src/layers/timeout.rs b/core/src/layers/timeout.rs index 5b3de5aef53d..8360cd0bf864 100644 --- a/core/src/layers/timeout.rs +++ b/core/src/layers/timeout.rs @@ -429,7 +429,7 @@ mod tests { let mut reader = op.reader("test").await.unwrap(); - let res = reader.read_at(&mut Vec::with_capacity(4), 0).await; + let res = reader.read(&mut Vec::default(), 0, 4).await; assert!(res.is_err()); let err = res.unwrap_err(); assert_eq!(err.kind(), ErrorKind::Unexpected); diff --git a/core/src/raw/tests/read.rs b/core/src/raw/tests/read.rs index 4a2743905ab5..307bfb47b2f7 100644 --- a/core/src/raw/tests/read.rs +++ b/core/src/raw/tests/read.rs @@ -113,7 +113,7 @@ impl ReadChecker { ReadAction::Read(offset, size) => { let mut bs = Vec::with_capacity(size); let n = r - .read_at(&mut bs, offset as u64) + .read(&mut bs, offset as u64, size) .await .expect("read must success"); self.check_read(offset, size, &bs[..n]); @@ -131,7 +131,7 @@ impl ReadChecker { ReadAction::Read(offset, size) => { let mut bs = Vec::with_capacity(size); let n = r - .read_at(&mut bs, offset as u64) + .read(&mut bs, offset as u64, size) .expect("read must success"); self.check_read(offset, size, &bs[..n]); } diff --git a/core/src/types/blocking_reader.rs b/core/src/types/blocking_reader.rs index bccb8cd1c09e..10361b2ec890 100644 --- a/core/src/types/blocking_reader.rs +++ b/core/src/types/blocking_reader.rs @@ -15,6 +15,7 @@ // specific language governing permissions and limitations // under the License. +use std::collections::Bound; use std::io; use std::io::SeekFrom; use std::ops::{Range, RangeBounds}; @@ -56,25 +57,47 @@ impl BlockingReader { BlockingReader { inner: r } } - /// Read given range bytes of data from reader. - pub fn read_at(&self, buf: &mut impl BufMut, offset: u64) -> crate::Result { - let bs = self.inner.read_at(offset, buf.remaining_mut())?; + #[inline] + pub fn read(&self, buf: &mut impl BufMut, offset: u64, limit: usize) -> Result { + let bs = self.inner.read_at(offset, limit)?; let n = bs.remaining(); buf.put(bs); Ok(n) } /// Read given range bytes of data from reader. - pub fn read_range(&self, buf: &mut impl BufMut, range: Range) -> crate::Result { - if range.is_empty() { - return Ok(0); + pub fn read_range(&self, buf: &mut impl BufMut, range: impl RangeBounds) -> Result { + let start = match range.start_bound().cloned() { + Bound::Included(start) => start, + Bound::Excluded(start) => start + 1, + Bound::Unbounded => 0, + }; + + let end = match range.end_bound().cloned() { + Bound::Included(end) => Some(end + 1), + Bound::Excluded(end) => Some(end), + Bound::Unbounded => None, + }; + + // If range is empty, return Ok(0) directly. + if let Some(end) = end { + if end <= start { + return Ok(0); + } } - let (mut offset, mut size) = (range.start, range.end - range.start); - let mut read = 0; + let mut offset = start; + let mut size = match end { + Some(end) => Some(end - start), + None => None, + }; + let mut read = 0; loop { - let bs = self.inner.read_at(offset, size as usize)?; + let bs = self + .inner + // TODO: use service preferred io size instead. + .read_at(offset, size.unwrap_or(4 * 1024 * 1024) as usize)?; let n = bs.remaining(); read += n; buf.put(bs); @@ -84,36 +107,16 @@ impl BlockingReader { offset += n as u64; - debug_assert!( - size >= n as u64, - "read should not return more bytes than expected" - ); - size -= n as u64; - if size == 0 { + size = size.map(|v| v - n as u64); + if size == Some(0) { return Ok(read); } } } - pub fn read_to_end(&self, buf: &mut impl BufMut) -> crate::Result { - self.read_to_end_at(buf, 0) - } - - pub fn read_to_end_at(&self, buf: &mut impl BufMut, mut offset: u64) -> crate::Result { - let mut size = 0; - loop { - // TODO: io size should be tuned based on storage - let bs = self.inner.read_at(offset, 4 * 1024 * 1024)?; - let n = bs.remaining(); - size += n; - - buf.put(bs); - if n == 0 { - return Ok(size); - } - - offset += n as u64; - } + #[inline] + pub fn read_to_end(&self, buf: &mut impl BufMut) -> Result { + self.read_range(buf, ..) } } diff --git a/core/src/types/operator/blocking_operator.rs b/core/src/types/operator/blocking_operator.rs index 1afbd3a11ba8..06576986d3ee 100644 --- a/core/src/types/operator/blocking_operator.rs +++ b/core/src/types/operator/blocking_operator.rs @@ -410,7 +410,7 @@ impl BlockingOperator { let (_, r) = inner.blocking_read(&path, args.with_range(range))?; let mut r = BlockingReader::new(r); let mut buf = Vec::with_capacity(size_hint as usize); - r.read_to_end_at(&mut buf, 0)?; + r.read_to_end(&mut buf)?; Ok(buf) }, diff --git a/core/src/types/reader.rs b/core/src/types/reader.rs index 572178fbaa89..abfdb34d87ab 100644 --- a/core/src/types/reader.rs +++ b/core/src/types/reader.rs @@ -17,7 +17,7 @@ use std::io; use std::io::SeekFrom; -use std::ops::{Range, RangeBounds}; +use std::ops::{Bound, Range, RangeBounds}; use std::pin::Pin; use std::task::ready; use std::task::Context; @@ -122,24 +122,52 @@ impl Reader { tokio_io_adapter::TokioReader::new(self.acc, self.path, self.op) } - pub async fn read_at(&self, buf: &mut impl BufMut, offset: u64) -> Result { - let bs = self.inner.read_at_dyn(offset, buf.remaining_mut()).await?; + #[inline] + pub async fn read(&self, buf: &mut impl BufMut, offset: u64, limit: usize) -> Result { + let bs = self.inner.read_at_dyn(offset, limit).await?; let n = bs.remaining(); buf.put(bs); Ok(n) } /// Read given range bytes of data from reader. - pub async fn read_range(&self, buf: &mut impl BufMut, range: Range) -> Result { - if range.is_empty() { - return Ok(0); + pub async fn read_range( + &self, + buf: &mut impl BufMut, + range: impl RangeBounds, + ) -> Result { + let start = match range.start_bound().cloned() { + Bound::Included(start) => start, + Bound::Excluded(start) => start + 1, + Bound::Unbounded => 0, + }; + + let end = match range.end_bound().cloned() { + Bound::Included(end) => Some(end + 1), + Bound::Excluded(end) => Some(end), + Bound::Unbounded => None, + }; + + // If range is empty, return Ok(0) directly. + if let Some(end) = end { + if end <= start { + return Ok(0); + } } - let (mut offset, mut size) = (range.start, range.end - range.start); - let mut read = 0; + let mut offset = start; + let mut size = match end { + Some(end) => Some(end - start), + None => None, + }; + let mut read = 0; loop { - let bs = self.inner.read_at_dyn(offset, size as usize).await?; + let bs = self + .inner + // TODO: use service preferred io size instead. + .read_at_dyn(offset, size.unwrap_or(4 * 1024 * 1024) as usize) + .await?; let n = bs.remaining(); read += n; buf.put(bs); @@ -149,36 +177,16 @@ impl Reader { offset += n as u64; - debug_assert!( - size >= n as u64, - "read should not return more bytes than expected" - ); - size -= n as u64; - if size == 0 { + size = size.map(|v| v - n as u64); + if size == Some(0) { return Ok(read); } } } + #[inline] pub async fn read_to_end(&self, buf: &mut impl BufMut) -> Result { - self.read_to_end_at(buf, 0).await - } - - pub async fn read_to_end_at(&self, buf: &mut impl BufMut, mut offset: u64) -> Result { - let mut size = 0; - loop { - // TODO: io size should be tuned based on storage - let bs = self.inner.read_at_dyn(offset, 4 * 1024 * 1024).await?; - let n = bs.remaining(); - size += n; - - buf.put(bs); - if n == 0 { - return Ok(size); - } - - offset += n as u64; - } + self.read_range(buf, ..).await } } @@ -501,7 +509,7 @@ mod tests { let mut reader = op.reader(path).await.unwrap(); let mut buf = Vec::new(); reader - .read_to_end_at(&mut buf, 0) + .read_to_end(&mut buf) .await .expect("read to end must succeed"); @@ -521,7 +529,7 @@ mod tests { let mut reader = op.reader(path).await.unwrap(); let mut buf = Vec::new(); reader - .read_to_end_at(&mut buf, 0) + .read_to_end(&mut buf) .await .expect("read to end must succeed"); assert_eq!(buf, content); From 4651e8a478c8615c94024bfba84bbb13b2df74c4 Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Thu, 21 Mar 2024 14:22:37 +0800 Subject: [PATCH 053/111] Fix fs Signed-off-by: Xuanwo --- core/src/layers/retry.rs | 2 - core/src/services/alluxio/backend.rs | 2 +- core/src/services/alluxio/reader.rs | 8 +-- core/src/services/fs/backend.rs | 9 +-- core/src/services/fs/reader.rs | 86 +++++++++++++++++++++++++--- 5 files changed, 86 insertions(+), 21 deletions(-) diff --git a/core/src/layers/retry.rs b/core/src/layers/retry.rs index b266c525b20f..35039de748a2 100644 --- a/core/src/layers/retry.rs +++ b/core/src/layers/retry.rs @@ -968,7 +968,6 @@ mod tests { RpRead::new(), MockReader { attempt: self.attempt.clone(), - pos: 0, }, )) } @@ -1039,7 +1038,6 @@ mod tests { #[derive(Debug, Clone, Default)] struct MockReader { attempt: Arc>, - pos: u64, } impl oio::Read for MockReader { diff --git a/core/src/services/alluxio/backend.rs b/core/src/services/alluxio/backend.rs index 65e55c7beaea..71669a8e7e89 100644 --- a/core/src/services/alluxio/backend.rs +++ b/core/src/services/alluxio/backend.rs @@ -226,7 +226,7 @@ impl Accessor for AlluxioBackend { async fn read(&self, path: &str, args: OpRead) -> Result<(RpRead, Self::Reader)> { let stream_id = self.core.open_file(path).await?; - let r = AlluxioReader::new(self.core.clone(), path, args.clone(), stream_id); + let r = AlluxioReader::new(self.core.clone(), stream_id, args.clone()); Ok((RpRead::new(), r)) } diff --git a/core/src/services/alluxio/reader.rs b/core/src/services/alluxio/reader.rs index ac9a15cd6f80..05a6c2a63709 100644 --- a/core/src/services/alluxio/reader.rs +++ b/core/src/services/alluxio/reader.rs @@ -26,18 +26,16 @@ use crate::*; pub struct AlluxioReader { core: Arc, - op: OpRead, - path: String, stream_id: u64, + op: OpRead, } impl AlluxioReader { - pub fn new(core: Arc, path: &str, op: OpRead, stream_id: u64) -> Self { + pub fn new(core: Arc, stream_id: u64, op: OpRead) -> Self { AlluxioReader { core, - path: path.to_string(), - op, stream_id, + op, } } } diff --git a/core/src/services/fs/backend.rs b/core/src/services/fs/backend.rs index 9f78976b58ac..121064a99802 100644 --- a/core/src/services/fs/backend.rs +++ b/core/src/services/fs/backend.rs @@ -242,10 +242,10 @@ impl FsBackend { #[async_trait] impl Accessor for FsBackend { - type Reader = FsReader; + type Reader = FsReader; type Writer = FsWriter; type Lister = Option>; - type BlockingReader = Bytes; + type BlockingReader = FsReader; type BlockingWriter = FsWriter; type BlockingLister = Option>; @@ -329,7 +329,7 @@ impl Accessor for FsBackend { .await .map_err(new_std_io_error)?; - let r = FsReader::new(f); + let r = FsReader::new(f.into_std().await); Ok((RpRead::new(), r)) } @@ -477,7 +477,8 @@ impl Accessor for FsBackend { .open(p) .map_err(new_std_io_error)?; - todo!() + let r = FsReader::new(f); + Ok((RpRead::new(), r)) } fn blocking_write(&self, path: &str, op: OpWrite) -> Result<(RpWrite, Self::BlockingWriter)> { diff --git a/core/src/services/fs/reader.rs b/core/src/services/fs/reader.rs index fda7e317269c..881a5d4b473e 100644 --- a/core/src/services/fs/reader.rs +++ b/core/src/services/fs/reader.rs @@ -15,23 +15,91 @@ // specific language governing permissions and limitations // under the License. -use crate::raw::oio; +use crate::raw::{new_std_io_error, oio}; use crate::*; -use std::future::Future; -use std::path::PathBuf; +use tokio::io::ReadBuf; -pub struct FsReader { - f: F, +pub struct FsReader { + f: std::fs::File, } -impl FsReader { - pub fn new(f: F) -> Self { +impl FsReader { + pub fn new(f: std::fs::File) -> Self { Self { f } } + + fn try_clone(&self) -> Result { + let f = self.f.try_clone().map_err(|err| { + Error::new( + ErrorKind::Unexpected, + "tokio fs clone file description failed", + ) + .set_source(err) + })?; + + Ok(Self { f }) + } + + #[cfg(target_family = "unix")] + pub fn read_at_inner(&self, buf: &mut [u8], offset: u64) -> Result { + use std::os::unix::fs::FileExt; + self.f.read_at(buf, offset).map_err(new_std_io_error) + } + + #[cfg(target_family = "windows")] + pub fn read_at_inner(&self, buf: &mut [u8], offset: u64) -> Result { + use std::os::windows::fs::FileExt; + self.f.seek_read(buf, offset).map_err(new_std_io_error) + } } -impl oio::Read for FsReader { +impl oio::Read for FsReader { async fn read_at(&self, offset: u64, limit: usize) -> Result { - todo!() + let handle = self.try_clone()?; + + match tokio::runtime::Handle::try_current() { + Ok(runtime) => runtime + .spawn_blocking(move || oio::BlockingRead::read_at(&handle, offset, limit)) + .await + .map_err(|err| { + Error::new(ErrorKind::Unexpected, "tokio spawn io task failed").set_source(err) + })?, + Err(_) => Err(Error::new( + ErrorKind::Unexpected, + "no tokio runtime found, failed to run io task", + )), + } + } +} + +impl oio::BlockingRead for FsReader { + fn read_at(&self, mut offset: u64, limit: usize) -> Result { + let mut bs = Vec::with_capacity(limit); + + let buf = bs.spare_capacity_mut(); + let mut read_buf: ReadBuf = ReadBuf::uninit(buf); + + // SAFETY: Read at most `size` bytes into `read_buf`. + unsafe { + read_buf.assume_init(limit); + } + + loop { + // If the buffer is full, we are done. + if read_buf.initialize_unfilled().is_empty() { + break; + } + let n = self.read_at_inner(read_buf.initialize_unfilled(), offset)?; + if n == 0 { + break; + } + read_buf.advance(n); + offset += n as u64; + } + + // Safety: We make sure that bs contains `n` more bytes. + let filled = read_buf.filled().len(); + unsafe { bs.set_len(filled) } + Ok(oio::Buffer::from(bs)) } } From a7f6c8927dee4d30f7a44b0e1009e31731fb47dc Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Thu, 21 Mar 2024 15:00:41 +0800 Subject: [PATCH 054/111] Save work Signed-off-by: Xuanwo --- bin/oli/Cargo.lock | 4 +- core/Cargo.lock | 4 +- core/Cargo.toml | 2 +- core/src/services/hdfs/backend.rs | 34 ++++++++++------ core/src/services/hdfs/reader.rs | 64 ++++++++++++++++++++++++++----- 5 files changed, 83 insertions(+), 25 deletions(-) diff --git a/bin/oli/Cargo.lock b/bin/oli/Cargo.lock index 843d9d7050da..02b8fc353cb9 100644 --- a/bin/oli/Cargo.lock +++ b/bin/oli/Cargo.lock @@ -1271,9 +1271,9 @@ dependencies = [ [[package]] name = "hdrs" -version = "0.3.1" +version = "0.3.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "00a7b465f2d12e45db2af56af13a1cbfe1d5616d54355f5610b26c0961dec7b7" +checksum = "f7c42a693bfe5dc8fcad1f24044c5ec355c5f157b8ce63c7d62f51cecbc7878d" dependencies = [ "blocking", "errno", diff --git a/core/Cargo.lock b/core/Cargo.lock index dc0c364c59f4..38f3ac047f06 100644 --- a/core/Cargo.lock +++ b/core/Cargo.lock @@ -2783,9 +2783,9 @@ dependencies = [ [[package]] name = "hdrs" -version = "0.3.1" +version = "0.3.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "00a7b465f2d12e45db2af56af13a1cbfe1d5616d54355f5610b26c0961dec7b7" +checksum = "f7c42a693bfe5dc8fcad1f24044c5ec355c5f157b8ce63c7d62f51cecbc7878d" dependencies = [ "blocking", "errno", diff --git a/core/Cargo.toml b/core/Cargo.toml index df57c1399b5d..5b84d27ad45c 100644 --- a/core/Cargo.toml +++ b/core/Cargo.toml @@ -274,7 +274,7 @@ foundationdb = { version = "0.8.0", features = [ "embedded-fdb-include", ], optional = true } # for services-hdfs -hdrs = { version = "0.3.0", optional = true, features = ["async_file"] } +hdrs = { version = "0.3.2", optional = true, features = ["async_file"] } # for services-upyun hmac = { version = "0.12.1", optional = true } # for services-libsql diff --git a/core/src/services/hdfs/backend.rs b/core/src/services/hdfs/backend.rs index 31edfb299496..6b9d29a092d6 100644 --- a/core/src/services/hdfs/backend.rs +++ b/core/src/services/hdfs/backend.rs @@ -250,7 +250,7 @@ impl Accessor for HdfsBackend { type Reader = HdfsReader; type Writer = HdfsWriter; type Lister = Option; - type BlockingReader = Bytes; + type BlockingReader = HdfsReader; type BlockingWriter = HdfsWriter; type BlockingLister = Option; @@ -311,15 +311,27 @@ impl Accessor for HdfsBackend { async fn read(&self, path: &str, _: OpRead) -> Result<(RpRead, Self::Reader)> { let p = build_rooted_abs_path(&self.root, path); - let f = self - .client - .open_file() - .read(true) - .async_open(&p) - .await - .map_err(new_std_io_error)?; - - todo!() + let client = self.client.clone(); + let f = match tokio::runtime::Handle::try_current() { + Ok(runtime) => runtime + .spawn_blocking(move || { + client + .open_file() + .read(true) + .open(&p) + .map_err(new_std_io_error) + }) + .await + .map_err(|err| { + Error::new(ErrorKind::Unexpected, "tokio spawn io task failed").set_source(err) + })?, + Err(_) => Err(Error::new( + ErrorKind::Unexpected, + "no tokio runtime found, failed to run io task", + )), + }?; + + Ok((RpRead::new(), HdfsReader::new(f))) } async fn write(&self, path: &str, op: OpWrite) -> Result<(RpWrite, Self::Writer)> { @@ -508,7 +520,7 @@ impl Accessor for HdfsBackend { .open(&p) .map_err(new_std_io_error)?; - todo!() + Ok((RpRead::new(), HdfsReader::new(f))) } fn blocking_write(&self, path: &str, op: OpWrite) -> Result<(RpWrite, Self::BlockingWriter)> { diff --git a/core/src/services/hdfs/reader.rs b/core/src/services/hdfs/reader.rs index 2566b802e9a7..059fb22f4d35 100644 --- a/core/src/services/hdfs/reader.rs +++ b/core/src/services/hdfs/reader.rs @@ -15,27 +15,73 @@ // specific language governing permissions and limitations // under the License. -use crate::raw::{oio, OpRead}; +use crate::raw::{new_std_io_error, oio, OpRead}; +use crate::{Error, ErrorKind}; +use hdrs::File; use http::StatusCode; use std::future::Future; use std::sync::Arc; +use tokio::io::ReadBuf; pub struct HdfsReader { - path: String, - op: OpRead, + f: Arc, } impl HdfsReader { - pub fn new(path: &str, op: OpRead) -> Self { - HdfsReader { - path: path.to_string(), - op, - } + pub fn new(f: File) -> Self { + Self { f: Arc::new(f) } } } impl oio::Read for HdfsReader { async fn read_at(&self, offset: u64, limit: usize) -> crate::Result { - todo!() + let r = Self { f: self.f.clone() }; + match tokio::runtime::Handle::try_current() { + Ok(runtime) => runtime + .spawn_blocking(move || oio::BlockingRead::read_at(&r, offset, limit)) + .await + .map_err(|err| { + Error::new(ErrorKind::Unexpected, "tokio spawn io task failed").set_source(err) + })?, + Err(_) => Err(Error::new( + ErrorKind::Unexpected, + "no tokio runtime found, failed to run io task", + )), + } + } +} + +impl oio::BlockingRead for HdfsReader { + fn read_at(&self, mut offset: u64, limit: usize) -> crate::Result { + let mut bs = Vec::with_capacity(limit); + + let buf = bs.spare_capacity_mut(); + let mut read_buf: ReadBuf = ReadBuf::uninit(buf); + + // SAFETY: Read at most `size` bytes into `read_buf`. + unsafe { + read_buf.assume_init(limit); + } + + loop { + // If the buffer is full, we are done. + if read_buf.initialize_unfilled().is_empty() { + break; + } + let n = self + .f + .read_at(read_buf.initialize_unfilled(), offset) + .map_err(new_std_io_error)?; + if n == 0 { + break; + } + read_buf.advance(n); + offset += n as u64; + } + + // Safety: We make sure that bs contains `n` more bytes. + let filled = read_buf.filled().len(); + unsafe { bs.set_len(filled) } + Ok(oio::Buffer::from(bs)) } } From f23955b3f6d96a6437914f87add4564f240ef0e8 Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Thu, 21 Mar 2024 15:04:34 +0800 Subject: [PATCH 055/111] Fix build Signed-off-by: Xuanwo --- core/benches/ops/read.rs | 5 ++-- core/src/layers/chaos.rs | 4 ++-- core/src/layers/concurrent_limit.rs | 4 ++-- core/src/layers/dtrace.rs | 2 +- core/src/layers/error_context.rs | 2 +- core/src/layers/logging.rs | 2 +- core/src/layers/metrics.rs | 8 +++---- core/src/layers/minitrace.rs | 2 +- core/src/layers/oteltrace.rs | 2 +- core/src/layers/prometheus.rs | 2 +- core/src/layers/prometheus_client.rs | 4 ++-- core/src/layers/retry.rs | 2 -- core/src/layers/throttle.rs | 2 +- core/src/layers/timeout.rs | 2 +- core/src/layers/tracing.rs | 2 +- core/src/raw/enum_utils.rs | 4 ++-- core/src/raw/http_util/bytes_range.rs | 2 +- core/src/raw/http_util/multipart.rs | 2 +- core/src/raw/oio/buf/buffer.rs | 4 ++-- core/src/raw/oio/stream/api.rs | 2 +- core/src/raw/tests/read.rs | 4 ++-- core/src/services/alluxio/reader.rs | 2 +- core/src/services/azblob/reader.rs | 2 +- core/src/services/azdls/reader.rs | 2 +- core/src/services/azfile/lister.rs | 2 +- core/src/services/b2/core.rs | 2 +- core/src/services/chainsafe/reader.rs | 2 +- core/src/services/dbfs/reader.rs | 6 ++--- core/src/services/dropbox/reader.rs | 2 +- core/src/services/fs/backend.rs | 2 +- core/src/services/ftp/backend.rs | 2 +- core/src/services/ftp/reader.rs | 2 +- core/src/services/gcs/reader.rs | 2 +- core/src/services/gdrive/reader.rs | 2 +- core/src/services/ghac/reader.rs | 4 ++-- core/src/services/github/reader.rs | 2 +- core/src/services/hdfs/backend.rs | 2 +- core/src/services/hdfs/reader.rs | 6 ++--- core/src/services/hdfs_native/reader.rs | 8 +++---- core/src/services/http/reader.rs | 4 ++-- core/src/services/huggingface/core.rs | 2 +- core/src/services/huggingface/reader.rs | 2 +- core/src/services/icloud/backend.rs | 2 +- core/src/services/icloud/reader.rs | 2 +- core/src/services/ipfs/reader.rs | 4 ++-- core/src/services/ipmfs/reader.rs | 4 ++-- core/src/services/koofr/reader.rs | 2 +- core/src/services/obs/reader.rs | 2 +- core/src/services/onedrive/reader.rs | 4 ++-- core/src/services/oss/reader.rs | 2 +- core/src/services/pcloud/reader.rs | 2 +- core/src/services/seafile/backend.rs | 4 ++-- core/src/services/seafile/reader.rs | 2 +- core/src/services/sftp/backend.rs | 6 ++--- core/src/services/supabase/reader.rs | 2 +- core/src/services/swift/core.rs | 2 +- core/src/services/swift/reader.rs | 2 +- core/src/services/upyun/reader.rs | 2 +- core/src/services/vercel_artifacts/reader.rs | 4 ++-- core/src/services/vercel_blob/reader.rs | 2 +- core/src/services/webdav/reader.rs | 2 +- core/src/services/webhdfs/backend.rs | 2 +- core/src/services/webhdfs/reader.rs | 6 ++--- core/src/services/yandex_disk/backend.rs | 2 +- core/src/services/yandex_disk/reader.rs | 2 +- core/src/types/blocking_reader.rs | 22 +++++++++--------- core/src/types/operator/blocking_operator.rs | 2 +- core/src/types/reader.rs | 24 ++++++++++---------- core/tests/behavior/async_read.rs | 16 ++++++------- 69 files changed, 124 insertions(+), 127 deletions(-) diff --git a/core/benches/ops/read.rs b/core/benches/ops/read.rs index 4ee2fd9117f6..b2bbe871d355 100644 --- a/core/benches/ops/read.rs +++ b/core/benches/ops/read.rs @@ -121,16 +121,15 @@ fn bench_read_parallel(c: &mut Criterion, name: &str, op: Operator) { group.bench_with_input( format!("{}x{}", parallel, size.to_string()), &(op.clone(), &path, buf_size), - |b, (op, path, buf_size)| { + |b, (op, path, _buf_size)| { b.to_async(&*TEST_RUNTIME).iter(|| async { let futures = (0..parallel) .map(|_| async { - let mut r = op + let _r = op .reader_with(path) .range(offset..=offset + size.bytes() as u64) .await .unwrap(); - todo!(); // r.read_exact(*buf_size).await.unwrap(); let mut d = 0; diff --git a/core/src/layers/chaos.rs b/core/src/layers/chaos.rs index ac646b9ae379..b5f58352505e 100644 --- a/core/src/layers/chaos.rs +++ b/core/src/layers/chaos.rs @@ -15,11 +15,11 @@ // specific language governing permissions and limitations // under the License. -use std::io; + use std::sync::{Arc, Mutex}; use async_trait::async_trait; -use bytes::Bytes; + use futures::FutureExt; use rand::prelude::*; use rand::rngs::StdRng; diff --git a/core/src/layers/concurrent_limit.rs b/core/src/layers/concurrent_limit.rs index c4ef48a1dcdd..9cf85355378c 100644 --- a/core/src/layers/concurrent_limit.rs +++ b/core/src/layers/concurrent_limit.rs @@ -16,9 +16,9 @@ // under the License. use std::fmt::Debug; -use std::future::Future; -use std::io::SeekFrom; + + use std::sync::Arc; use async_trait::async_trait; diff --git a/core/src/layers/dtrace.rs b/core/src/layers/dtrace.rs index d8ef35ca6375..2713cda6c1e0 100644 --- a/core/src/layers/dtrace.rs +++ b/core/src/layers/dtrace.rs @@ -19,7 +19,7 @@ use std::ffi::CString; use std::fmt::Debug; use std::fmt::Formatter; -use std::io; + use async_trait::async_trait; use bytes::Buf; diff --git a/core/src/layers/error_context.rs b/core/src/layers/error_context.rs index 8996ad0f03d2..0e40252f961b 100644 --- a/core/src/layers/error_context.rs +++ b/core/src/layers/error_context.rs @@ -18,7 +18,7 @@ use std::fmt::Debug; use std::fmt::Formatter; -use std::io::SeekFrom; + use async_trait::async_trait; use bytes::Bytes; diff --git a/core/src/layers/logging.rs b/core/src/layers/logging.rs index 96fb252aba01..9825a3dc5af7 100644 --- a/core/src/layers/logging.rs +++ b/core/src/layers/logging.rs @@ -17,7 +17,7 @@ use std::fmt::Debug; -use std::io; + use async_trait::async_trait; use bytes::{Buf, Bytes}; diff --git a/core/src/layers/metrics.rs b/core/src/layers/metrics.rs index 6b296e671560..28890a5f9e67 100644 --- a/core/src/layers/metrics.rs +++ b/core/src/layers/metrics.rs @@ -18,7 +18,7 @@ use std::fmt::Debug; use std::fmt::Formatter; use std::future::Future; -use std::io; + use std::sync::Arc; use std::time::Instant; @@ -456,7 +456,7 @@ impl LayeredAccessor for MetricsAccessor { async fn read(&self, path: &str, args: OpRead) -> Result<(RpRead, Self::Reader)> { self.handle.requests_total_read.increment(1); - let start = Instant::now(); + let _start = Instant::now(); self.inner .read(path, args) @@ -485,7 +485,7 @@ impl LayeredAccessor for MetricsAccessor { async fn write(&self, path: &str, args: OpWrite) -> Result<(RpWrite, Self::Writer)> { self.handle.requests_total_write.increment(1); - let start = Instant::now(); + let _start = Instant::now(); self.inner .write(path, args) @@ -618,7 +618,7 @@ impl LayeredAccessor for MetricsAccessor { fn blocking_read(&self, path: &str, args: OpRead) -> Result<(RpRead, Self::BlockingReader)> { self.handle.requests_total_blocking_read.increment(1); - let start = Instant::now(); + let _start = Instant::now(); let result = self.inner.blocking_read(path, args).map(|(rp, r)| { ( rp, diff --git a/core/src/layers/minitrace.rs b/core/src/layers/minitrace.rs index e256bcb4e453..9dc60d0bf82e 100644 --- a/core/src/layers/minitrace.rs +++ b/core/src/layers/minitrace.rs @@ -18,7 +18,7 @@ use std::fmt::Debug; use std::future::Future; -use std::io; + use async_trait::async_trait; use bytes::Bytes; diff --git a/core/src/layers/oteltrace.rs b/core/src/layers/oteltrace.rs index bd03f1ddd18e..fd2ba5bb4b94 100644 --- a/core/src/layers/oteltrace.rs +++ b/core/src/layers/oteltrace.rs @@ -16,7 +16,7 @@ // under the License. use std::future::Future; -use std::io; + use async_trait::async_trait; use bytes::Bytes; diff --git a/core/src/layers/prometheus.rs b/core/src/layers/prometheus.rs index f99edd06d456..c8e96e0d5272 100644 --- a/core/src/layers/prometheus.rs +++ b/core/src/layers/prometheus.rs @@ -18,7 +18,7 @@ use std::fmt::Debug; use std::fmt::Formatter; -use std::io; + use std::sync::Arc; use async_trait::async_trait; diff --git a/core/src/layers/prometheus_client.rs b/core/src/layers/prometheus_client.rs index 693d386f4ac8..a703fbb690fb 100644 --- a/core/src/layers/prometheus_client.rs +++ b/core/src/layers/prometheus_client.rs @@ -17,8 +17,8 @@ use std::fmt::Debug; use std::fmt::Formatter; -use std::future::Future; -use std::io; + + use std::sync::Arc; use std::time::Duration; diff --git a/core/src/layers/retry.rs b/core/src/layers/retry.rs index 35039de748a2..c9ea25b51c45 100644 --- a/core/src/layers/retry.rs +++ b/core/src/layers/retry.rs @@ -19,8 +19,6 @@ use std::fmt::Debug; use std::fmt::Formatter; use std::future::Future; -use std::io; - use std::sync::Arc; use std::time::Duration; diff --git a/core/src/layers/throttle.rs b/core/src/layers/throttle.rs index ab2595b671e9..9f7373f40fce 100644 --- a/core/src/layers/throttle.rs +++ b/core/src/layers/throttle.rs @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -use std::io::SeekFrom; + use std::num::NonZeroU32; use std::sync::Arc; diff --git a/core/src/layers/timeout.rs b/core/src/layers/timeout.rs index 8360cd0bf864..d917095c6ca8 100644 --- a/core/src/layers/timeout.rs +++ b/core/src/layers/timeout.rs @@ -16,7 +16,7 @@ // under the License. use std::future::Future; -use std::io::SeekFrom; + use std::time::Duration; diff --git a/core/src/layers/tracing.rs b/core/src/layers/tracing.rs index 7510c9720c79..8e1e439ee320 100644 --- a/core/src/layers/tracing.rs +++ b/core/src/layers/tracing.rs @@ -18,7 +18,7 @@ use std::fmt::Debug; use std::future::Future; -use std::io; + use async_trait::async_trait; use bytes::Bytes; diff --git a/core/src/raw/enum_utils.rs b/core/src/raw/enum_utils.rs index 99bedb1ac516..315aa5c11c85 100644 --- a/core/src/raw/enum_utils.rs +++ b/core/src/raw/enum_utils.rs @@ -38,8 +38,8 @@ //! This module is used to provide some enums for the above code. We should remove this module once //! type_alias_impl_trait has been stabilized. -use std::future::Future; -use std::io::SeekFrom; + + use bytes::Bytes; diff --git a/core/src/raw/http_util/bytes_range.rs b/core/src/raw/http_util/bytes_range.rs index 9db8a95fb4ca..7b4e8312b394 100644 --- a/core/src/raw/http_util/bytes_range.rs +++ b/core/src/raw/http_util/bytes_range.rs @@ -140,7 +140,7 @@ impl BytesRange { } (Some(base), None) => Some(Self(Some(base + offset), Some(limit as u64))), (None, None) => Some(Self(Some(offset), Some(limit as u64))), - (None, Some(size)) => unimplemented!(), + (None, Some(_size)) => unimplemented!(), } } diff --git a/core/src/raw/http_util/multipart.rs b/core/src/raw/http_util/multipart.rs index 27f80f2c65dd..81a9b2c6ae9d 100644 --- a/core/src/raw/http_util/multipart.rs +++ b/core/src/raw/http_util/multipart.rs @@ -24,7 +24,7 @@ use std::task::Poll; use bytes::Bytes; use bytes::BytesMut; -use futures::stream; + use http::header::CONTENT_DISPOSITION; use http::header::CONTENT_LENGTH; use http::header::CONTENT_TYPE; diff --git a/core/src/raw/oio/buf/buffer.rs b/core/src/raw/oio/buf/buffer.rs index 40986766f8a0..59dc2c8c92b9 100644 --- a/core/src/raw/oio/buf/buffer.rs +++ b/core/src/raw/oio/buf/buffer.rs @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -use crate::raw::oio; + use bytes::{Buf, BufMut, Bytes, BytesMut}; use std::collections::VecDeque; @@ -41,7 +41,7 @@ impl Buffer { } /// Merge two buffer together without copying internal bytes. - pub fn merge(mut self, buf: Buffer) -> Self { + pub fn merge(self, buf: Buffer) -> Self { let mut vec = match self.0 { Inner::Contiguous(b) => { // NOTE: we will have at least two bytes in the vec. diff --git a/core/src/raw/oio/stream/api.rs b/core/src/raw/oio/stream/api.rs index dc53682fd370..9837e308616d 100644 --- a/core/src/raw/oio/stream/api.rs +++ b/core/src/raw/oio/stream/api.rs @@ -47,7 +47,7 @@ impl Stream for () { } impl Stream for Bytes { - fn poll_next(&mut self, cx: &mut Context<'_>) -> Poll>> { + fn poll_next(&mut self, _cx: &mut Context<'_>) -> Poll>> { if self.has_remaining() { Poll::Ready(Some(Ok(self.copy_to_bytes(self.remaining())))) } else { diff --git a/core/src/raw/tests/read.rs b/core/src/raw/tests/read.rs index 307bfb47b2f7..404c8763e08a 100644 --- a/core/src/raw/tests/read.rs +++ b/core/src/raw/tests/read.rs @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -use bytes::{Bytes, BytesMut}; +use bytes::{Bytes}; use rand::thread_rng; use rand::RngCore; use sha2::Digest; @@ -125,7 +125,7 @@ impl ReadChecker { /// Check will check the correctness of the read process via given actions. /// /// Check will panic if any check failed. - pub fn blocking_check(&mut self, mut r: BlockingReader, actions: &[ReadAction]) { + pub fn blocking_check(&mut self, r: BlockingReader, actions: &[ReadAction]) { for action in actions { match *action { ReadAction::Read(offset, size) => { diff --git a/core/src/services/alluxio/reader.rs b/core/src/services/alluxio/reader.rs index 05a6c2a63709..ba6bb16a6095 100644 --- a/core/src/services/alluxio/reader.rs +++ b/core/src/services/alluxio/reader.rs @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -use std::future::Future; + use std::sync::Arc; use super::core::*; diff --git a/core/src/services/azblob/reader.rs b/core/src/services/azblob/reader.rs index 147765830d6f..bea274ffe604 100644 --- a/core/src/services/azblob/reader.rs +++ b/core/src/services/azblob/reader.rs @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -use std::future::Future; + use std::sync::Arc; use http::StatusCode; diff --git a/core/src/services/azdls/reader.rs b/core/src/services/azdls/reader.rs index 7a42159bd39b..811b5a32ca23 100644 --- a/core/src/services/azdls/reader.rs +++ b/core/src/services/azdls/reader.rs @@ -19,7 +19,7 @@ use super::core::AzdlsCore; use super::error::parse_error; use crate::raw::{oio, OpRead}; use http::StatusCode; -use std::future::Future; + use std::sync::Arc; pub struct AzdlsReader { diff --git a/core/src/services/azfile/lister.rs b/core/src/services/azfile/lister.rs index daadad24b6c1..2ee372eba1fb 100644 --- a/core/src/services/azfile/lister.rs +++ b/core/src/services/azfile/lister.rs @@ -20,7 +20,7 @@ use std::sync::Arc; use http::StatusCode; use quick_xml::de; -use quick_xml::de::from_str; + use serde::Deserialize; use super::core::AzfileCore; diff --git a/core/src/services/b2/core.rs b/core/src/services/b2/core.rs index 985b7e7cf795..c2b3a2e3feed 100644 --- a/core/src/services/b2/core.rs +++ b/core/src/services/b2/core.rs @@ -132,7 +132,7 @@ impl B2Core { &self, path: &str, range: BytesRange, - args: &OpRead, + _args: &OpRead, ) -> Result> { let path = build_abs_path(&self.root, path); diff --git a/core/src/services/chainsafe/reader.rs b/core/src/services/chainsafe/reader.rs index 1a59eede51f1..8e3540956154 100644 --- a/core/src/services/chainsafe/reader.rs +++ b/core/src/services/chainsafe/reader.rs @@ -19,7 +19,7 @@ use super::core::ChainsafeCore; use super::error::parse_error; use crate::raw::{oio, OpRead}; use http::StatusCode; -use std::future::Future; + use std::sync::Arc; pub struct ChainsafeReader { diff --git a/core/src/services/dbfs/reader.rs b/core/src/services/dbfs/reader.rs index 4a4bbd4c7a23..3bc018623cd2 100644 --- a/core/src/services/dbfs/reader.rs +++ b/core/src/services/dbfs/reader.rs @@ -15,8 +15,8 @@ // specific language governing permissions and limitations // under the License. -use std::future::Future; -use std::io::SeekFrom; + + use std::sync::Arc; use base64::engine::general_purpose; @@ -88,7 +88,7 @@ impl DbfsReader { unsafe impl Sync for DbfsReader {} impl oio::Read for DbfsReader { - async fn read_at(&self, offset: u64, limit: usize) -> Result { + async fn read_at(&self, _offset: u64, _limit: usize) -> Result { todo!() } } diff --git a/core/src/services/dropbox/reader.rs b/core/src/services/dropbox/reader.rs index 39d04733fe6c..8d1742269865 100644 --- a/core/src/services/dropbox/reader.rs +++ b/core/src/services/dropbox/reader.rs @@ -19,7 +19,7 @@ use super::core::DropboxCore; use super::error::parse_error; use crate::raw::{oio, OpRead}; use http::StatusCode; -use std::future::Future; + use std::sync::Arc; pub struct DropboxReader { diff --git a/core/src/services/fs/backend.rs b/core/src/services/fs/backend.rs index 121064a99802..92079d4c9bf5 100644 --- a/core/src/services/fs/backend.rs +++ b/core/src/services/fs/backend.rs @@ -20,7 +20,7 @@ use std::path::Path; use std::path::PathBuf; use async_trait::async_trait; -use bytes::Bytes; + use chrono::DateTime; use log::debug; use uuid::Uuid; diff --git a/core/src/services/ftp/backend.rs b/core/src/services/ftp/backend.rs index 6a96caaa1051..4157482d1df0 100644 --- a/core/src/services/ftp/backend.rs +++ b/core/src/services/ftp/backend.rs @@ -25,7 +25,7 @@ use async_tls::TlsConnector; use async_trait::async_trait; use bb8::PooledConnection; use bb8::RunError; -use futures::AsyncRead; + use futures::AsyncReadExt; use http::Uri; use log::debug; diff --git a/core/src/services/ftp/reader.rs b/core/src/services/ftp/reader.rs index 7a34165a69a0..bb0e6c1ffe5d 100644 --- a/core/src/services/ftp/reader.rs +++ b/core/src/services/ftp/reader.rs @@ -46,7 +46,7 @@ impl oio::Read for FtpReader { let mut ftp_stream = self.core.ftp_connect(Operation::Read).await?; - let meta = self.core.ftp_stat(&self.path).await?; + let _meta = self.core.ftp_stat(&self.path).await?; match (range.offset(), range.size()) { (Some(offset), Some(size)) => { diff --git a/core/src/services/gcs/reader.rs b/core/src/services/gcs/reader.rs index 60dd7d785a0a..e0cea1a3d983 100644 --- a/core/src/services/gcs/reader.rs +++ b/core/src/services/gcs/reader.rs @@ -19,7 +19,7 @@ use super::core::GcsCore; use super::error::parse_error; use crate::raw::{oio, OpRead}; use http::StatusCode; -use std::future::Future; + use std::sync::Arc; pub struct GcsReader { diff --git a/core/src/services/gdrive/reader.rs b/core/src/services/gdrive/reader.rs index afad3cba4176..348cef06255f 100644 --- a/core/src/services/gdrive/reader.rs +++ b/core/src/services/gdrive/reader.rs @@ -19,7 +19,7 @@ use super::core::GdriveCore; use super::error::parse_error; use crate::raw::{oio, OpRead}; use http::StatusCode; -use std::future::Future; + use std::sync::Arc; pub struct GdriveReader { diff --git a/core/src/services/ghac/reader.rs b/core/src/services/ghac/reader.rs index 590ad0903e2b..2a6a02fd5a2d 100644 --- a/core/src/services/ghac/reader.rs +++ b/core/src/services/ghac/reader.rs @@ -19,8 +19,8 @@ use super::error::parse_error; use crate::raw::{oio, OpRead}; use crate::services::ghac::backend::GhacBackend; use http::StatusCode; -use std::future::Future; -use std::sync::Arc; + + pub struct GhacReader { core: GhacBackend, diff --git a/core/src/services/github/reader.rs b/core/src/services/github/reader.rs index 31561188e43d..650cbd06aac5 100644 --- a/core/src/services/github/reader.rs +++ b/core/src/services/github/reader.rs @@ -19,7 +19,7 @@ use super::core::GithubCore; use super::error::parse_error; use crate::raw::{oio, OpRead}; use http::StatusCode; -use std::future::Future; + use std::sync::Arc; pub struct GithubReader { diff --git a/core/src/services/hdfs/backend.rs b/core/src/services/hdfs/backend.rs index 6b9d29a092d6..2a55bb38f366 100644 --- a/core/src/services/hdfs/backend.rs +++ b/core/src/services/hdfs/backend.rs @@ -23,7 +23,7 @@ use std::path::PathBuf; use std::sync::Arc; use async_trait::async_trait; -use bytes::Bytes; + use futures::AsyncWriteExt; use log::debug; use serde::Deserialize; diff --git a/core/src/services/hdfs/reader.rs b/core/src/services/hdfs/reader.rs index 059fb22f4d35..91512abc7cc2 100644 --- a/core/src/services/hdfs/reader.rs +++ b/core/src/services/hdfs/reader.rs @@ -15,11 +15,11 @@ // specific language governing permissions and limitations // under the License. -use crate::raw::{new_std_io_error, oio, OpRead}; +use crate::raw::{new_std_io_error, oio}; use crate::{Error, ErrorKind}; use hdrs::File; -use http::StatusCode; -use std::future::Future; + + use std::sync::Arc; use tokio::io::ReadBuf; diff --git a/core/src/services/hdfs_native/reader.rs b/core/src/services/hdfs_native/reader.rs index f6e078cba182..b6864b04b0fa 100644 --- a/core/src/services/hdfs_native/reader.rs +++ b/core/src/services/hdfs_native/reader.rs @@ -15,13 +15,13 @@ // specific language governing permissions and limitations // under the License. -use std::io::SeekFrom; -use bytes::Bytes; + + use hdfs_native::file::FileReader; use crate::raw::oio; -use crate::*; + pub struct HdfsNativeReader { _f: FileReader, @@ -34,7 +34,7 @@ impl HdfsNativeReader { } impl oio::Read for HdfsNativeReader { - async fn read_at(&self, offset: u64, limit: usize) -> crate::Result { + async fn read_at(&self, _offset: u64, _limit: usize) -> crate::Result { todo!() } } diff --git a/core/src/services/http/reader.rs b/core/src/services/http/reader.rs index f6266b30d688..602da90be2ba 100644 --- a/core/src/services/http/reader.rs +++ b/core/src/services/http/reader.rs @@ -19,8 +19,8 @@ use super::error::parse_error; use crate::raw::{oio, OpRead}; use crate::services::http::backend::HttpBackend; use http::StatusCode; -use std::future::Future; -use std::sync::Arc; + + pub struct HttpReader { core: HttpBackend, diff --git a/core/src/services/huggingface/core.rs b/core/src/services/huggingface/core.rs index 2557b552d8b6..ed9d271c160b 100644 --- a/core/src/services/huggingface/core.rs +++ b/core/src/services/huggingface/core.rs @@ -125,7 +125,7 @@ impl HuggingfaceCore { &self, path: &str, range: BytesRange, - args: &OpRead, + _args: &OpRead, ) -> Result> { let p = build_abs_path(&self.root, path) .trim_end_matches('/') diff --git a/core/src/services/huggingface/reader.rs b/core/src/services/huggingface/reader.rs index 4b2172db8b67..a5404b06e838 100644 --- a/core/src/services/huggingface/reader.rs +++ b/core/src/services/huggingface/reader.rs @@ -19,7 +19,7 @@ use super::core::HuggingfaceCore; use super::error::parse_error; use crate::raw::{oio, OpRead}; use http::StatusCode; -use std::future::Future; + use std::sync::Arc; pub struct HuggingfaceReader { diff --git a/core/src/services/icloud/backend.rs b/core/src/services/icloud/backend.rs index a414f92ca228..7a2075335cc3 100644 --- a/core/src/services/icloud/backend.rs +++ b/core/src/services/icloud/backend.rs @@ -21,7 +21,7 @@ use std::fmt::Formatter; use std::sync::Arc; use async_trait::async_trait; -use http::StatusCode; + use serde::Deserialize; use tokio::sync::Mutex; diff --git a/core/src/services/icloud/reader.rs b/core/src/services/icloud/reader.rs index 4309ee0f1525..441203f33367 100644 --- a/core/src/services/icloud/reader.rs +++ b/core/src/services/icloud/reader.rs @@ -18,7 +18,7 @@ use super::core::{parse_error, IcloudCore}; use crate::raw::{oio, OpRead}; use http::StatusCode; -use std::future::Future; + use std::sync::Arc; pub struct IcloudReader { diff --git a/core/src/services/ipfs/reader.rs b/core/src/services/ipfs/reader.rs index c43ee0f1c23d..50eaffb080aa 100644 --- a/core/src/services/ipfs/reader.rs +++ b/core/src/services/ipfs/reader.rs @@ -19,8 +19,8 @@ use super::error::parse_error; use crate::raw::{oio, OpRead}; use crate::services::ipfs::backend::IpfsBackend; use http::StatusCode; -use std::future::Future; -use std::sync::Arc; + + pub struct IpfsReader { core: IpfsBackend, diff --git a/core/src/services/ipmfs/reader.rs b/core/src/services/ipmfs/reader.rs index ecb7f41dbba6..682bcff1fa16 100644 --- a/core/src/services/ipmfs/reader.rs +++ b/core/src/services/ipmfs/reader.rs @@ -19,8 +19,8 @@ use super::error::parse_error; use crate::raw::{oio, OpRead}; use crate::services::ipmfs::backend::IpmfsBackend; use http::StatusCode; -use std::future::Future; -use std::sync::Arc; + + pub struct IpmfsReader { core: IpmfsBackend, diff --git a/core/src/services/koofr/reader.rs b/core/src/services/koofr/reader.rs index 081034afbfe1..0206a4413c6c 100644 --- a/core/src/services/koofr/reader.rs +++ b/core/src/services/koofr/reader.rs @@ -19,7 +19,7 @@ use super::core::KoofrCore; use super::error::parse_error; use crate::raw::{oio, OpRead}; use http::StatusCode; -use std::future::Future; + use std::sync::Arc; pub struct KoofrReader { diff --git a/core/src/services/obs/reader.rs b/core/src/services/obs/reader.rs index cb9c083a9cc7..5e81eb209c57 100644 --- a/core/src/services/obs/reader.rs +++ b/core/src/services/obs/reader.rs @@ -19,7 +19,7 @@ use super::core::ObsCore; use super::error::parse_error; use crate::raw::{oio, OpRead}; use http::StatusCode; -use std::future::Future; + use std::sync::Arc; pub struct ObsReader { diff --git a/core/src/services/onedrive/reader.rs b/core/src/services/onedrive/reader.rs index 72c0c9fe7572..3a250d7847b3 100644 --- a/core/src/services/onedrive/reader.rs +++ b/core/src/services/onedrive/reader.rs @@ -19,8 +19,8 @@ use super::error::parse_error; use crate::raw::{oio, OpRead}; use crate::services::onedrive::backend::OnedriveBackend; use http::StatusCode; -use std::future::Future; -use std::sync::Arc; + + pub struct OnedriveReader { core: OnedriveBackend, diff --git a/core/src/services/oss/reader.rs b/core/src/services/oss/reader.rs index 05197f78bb92..a803dcf0fbbc 100644 --- a/core/src/services/oss/reader.rs +++ b/core/src/services/oss/reader.rs @@ -19,7 +19,7 @@ use super::core::OssCore; use super::error::parse_error; use crate::raw::{oio, OpRead}; use http::StatusCode; -use std::future::Future; + use std::sync::Arc; pub struct OssReader { diff --git a/core/src/services/pcloud/reader.rs b/core/src/services/pcloud/reader.rs index f403d35c2260..0d87c2f37303 100644 --- a/core/src/services/pcloud/reader.rs +++ b/core/src/services/pcloud/reader.rs @@ -19,7 +19,7 @@ use super::core::PcloudCore; use super::error::parse_error; use crate::raw::{oio, OpRead}; use http::StatusCode; -use std::future::Future; + use std::sync::Arc; pub struct PcloudReader { diff --git a/core/src/services/seafile/backend.rs b/core/src/services/seafile/backend.rs index 031c9ef97977..08a772d2b93d 100644 --- a/core/src/services/seafile/backend.rs +++ b/core/src/services/seafile/backend.rs @@ -21,7 +21,7 @@ use std::fmt::Formatter; use std::sync::Arc; use async_trait::async_trait; -use http::StatusCode; + use log::debug; use serde::Deserialize; use tokio::sync::RwLock; @@ -29,7 +29,7 @@ use tokio::sync::RwLock; use super::core::parse_dir_detail; use super::core::parse_file_detail; use super::core::SeafileCore; -use super::error::parse_error; + use super::lister::SeafileLister; use super::writer::SeafileWriter; use super::writer::SeafileWriters; diff --git a/core/src/services/seafile/reader.rs b/core/src/services/seafile/reader.rs index 487795bc44a0..0cf002bc94e6 100644 --- a/core/src/services/seafile/reader.rs +++ b/core/src/services/seafile/reader.rs @@ -19,7 +19,7 @@ use super::core::SeafileCore; use super::error::parse_error; use crate::raw::{oio, OpRead}; use http::StatusCode; -use std::future::Future; + use std::sync::Arc; pub struct SeafileReader { diff --git a/core/src/services/sftp/backend.rs b/core/src/services/sftp/backend.rs index 7e4c43c17f7a..f7c1b7500ef4 100644 --- a/core/src/services/sftp/backend.rs +++ b/core/src/services/sftp/backend.rs @@ -20,7 +20,7 @@ use std::fmt::Debug; use std::fmt::Formatter; use std::path::Path; use std::path::PathBuf; -use std::pin::Pin; + use async_trait::async_trait; use bytes::Bytes; @@ -28,7 +28,7 @@ use futures::StreamExt; use log::debug; use openssh::KnownHosts; use openssh::SessionBuilder; -use openssh_sftp_client::file::TokioCompatFile; + use openssh_sftp_client::Sftp; use openssh_sftp_client::SftpOptions; use serde::Deserialize; @@ -318,7 +318,7 @@ impl Accessor for SftpBackend { fs.set_cwd(&self.root); let path = fs.canonicalize(path).await.map_err(parse_sftp_error)?; - let f = client + let _f = client .open(path.as_path()) .await .map_err(parse_sftp_error)?; diff --git a/core/src/services/supabase/reader.rs b/core/src/services/supabase/reader.rs index 5a63b8c2c62f..ec22eb0ff59d 100644 --- a/core/src/services/supabase/reader.rs +++ b/core/src/services/supabase/reader.rs @@ -19,7 +19,7 @@ use super::core::SupabaseCore; use super::error::parse_error; use crate::raw::{oio, OpRead}; use http::StatusCode; -use std::future::Future; + use std::sync::Arc; pub struct SupabaseReader { diff --git a/core/src/services/swift/core.rs b/core/src/services/swift/core.rs index 0ebcec1317eb..c1cd294e97c9 100644 --- a/core/src/services/swift/core.rs +++ b/core/src/services/swift/core.rs @@ -131,7 +131,7 @@ impl SwiftCore { &self, path: &str, range: BytesRange, - arg: &OpRead, + _arg: &OpRead, ) -> Result> { let p = build_abs_path(&self.root, path) .trim_end_matches('/') diff --git a/core/src/services/swift/reader.rs b/core/src/services/swift/reader.rs index 00142c3c7275..2312c9f99459 100644 --- a/core/src/services/swift/reader.rs +++ b/core/src/services/swift/reader.rs @@ -19,7 +19,7 @@ use super::core::SwiftCore; use super::error::parse_error; use crate::raw::{oio, OpRead}; use http::StatusCode; -use std::future::Future; + use std::sync::Arc; pub struct SwiftReader { diff --git a/core/src/services/upyun/reader.rs b/core/src/services/upyun/reader.rs index 1f47e313c540..04b3f886347e 100644 --- a/core/src/services/upyun/reader.rs +++ b/core/src/services/upyun/reader.rs @@ -19,7 +19,7 @@ use super::core::UpyunCore; use super::error::parse_error; use crate::raw::{oio, OpRead}; use http::StatusCode; -use std::future::Future; + use std::sync::Arc; pub struct UpyunReader { diff --git a/core/src/services/vercel_artifacts/reader.rs b/core/src/services/vercel_artifacts/reader.rs index 29edfabeb825..1bc64847d150 100644 --- a/core/src/services/vercel_artifacts/reader.rs +++ b/core/src/services/vercel_artifacts/reader.rs @@ -19,8 +19,8 @@ use super::error::parse_error; use crate::raw::{oio, OpRead}; use crate::services::vercel_artifacts::backend::VercelArtifactsBackend; use http::StatusCode; -use std::future::Future; -use std::sync::Arc; + + pub struct VercelArtifactsReader { core: VercelArtifactsBackend, diff --git a/core/src/services/vercel_blob/reader.rs b/core/src/services/vercel_blob/reader.rs index 467272f6223c..71b3af1e03a0 100644 --- a/core/src/services/vercel_blob/reader.rs +++ b/core/src/services/vercel_blob/reader.rs @@ -19,7 +19,7 @@ use super::core::VercelBlobCore; use super::error::parse_error; use crate::raw::{oio, OpRead}; use http::StatusCode; -use std::future::Future; + use std::sync::Arc; pub struct VercelBlobReader { diff --git a/core/src/services/webdav/reader.rs b/core/src/services/webdav/reader.rs index 2817b8276d1d..7cc8f8d954dd 100644 --- a/core/src/services/webdav/reader.rs +++ b/core/src/services/webdav/reader.rs @@ -19,7 +19,7 @@ use super::core::WebdavCore; use super::error::parse_error; use crate::raw::{oio, OpRead}; use http::StatusCode; -use std::future::Future; + use std::sync::Arc; pub struct WebdavReader { diff --git a/core/src/services/webhdfs/backend.rs b/core/src/services/webhdfs/backend.rs index d63129c1ef1f..93a12131cdf2 100644 --- a/core/src/services/webhdfs/backend.rs +++ b/core/src/services/webhdfs/backend.rs @@ -30,7 +30,7 @@ use serde::Deserialize; use tokio::sync::OnceCell; use super::error::parse_error; -use super::error::parse_error_msg; + use super::lister::WebhdfsLister; use super::message::BooleanResp; use super::message::FileStatusType; diff --git a/core/src/services/webhdfs/reader.rs b/core/src/services/webhdfs/reader.rs index 415a1167c19d..b980e5ad9e28 100644 --- a/core/src/services/webhdfs/reader.rs +++ b/core/src/services/webhdfs/reader.rs @@ -16,12 +16,12 @@ // under the License. use super::error::{parse_error, parse_error_msg}; -use crate::raw::{oio, OpRead, RpRead}; +use crate::raw::{oio, OpRead}; use crate::services::webhdfs::backend::WebhdfsBackend; use bytes::Buf; use http::StatusCode; -use std::future::Future; -use std::sync::Arc; + + pub struct WebhdfsReader { core: WebhdfsBackend, diff --git a/core/src/services/yandex_disk/backend.rs b/core/src/services/yandex_disk/backend.rs index 09b4d2c34425..58ceb9485aef 100644 --- a/core/src/services/yandex_disk/backend.rs +++ b/core/src/services/yandex_disk/backend.rs @@ -22,7 +22,7 @@ use std::sync::Arc; use async_trait::async_trait; use bytes::Buf; -use http::Request; + use http::StatusCode; use log::debug; use serde::Deserialize; diff --git a/core/src/services/yandex_disk/reader.rs b/core/src/services/yandex_disk/reader.rs index dd417c5f3850..2103b574bbd4 100644 --- a/core/src/services/yandex_disk/reader.rs +++ b/core/src/services/yandex_disk/reader.rs @@ -19,7 +19,7 @@ use super::core::YandexDiskCore; use super::error::parse_error; use crate::raw::{new_request_build_error, oio, AsyncBody, OpRead}; use http::{header, Request, StatusCode}; -use std::future::Future; + use std::sync::Arc; pub struct YandexDiskReader { diff --git a/core/src/types/blocking_reader.rs b/core/src/types/blocking_reader.rs index 10361b2ec890..33f4d823ebab 100644 --- a/core/src/types/blocking_reader.rs +++ b/core/src/types/blocking_reader.rs @@ -16,17 +16,17 @@ // under the License. use std::collections::Bound; -use std::io; -use std::io::SeekFrom; -use std::ops::{Range, RangeBounds}; -use std::pin::Pin; -use std::task::ready; -use std::task::Context; -use std::task::Poll; - -use bytes::{Buf, BufMut, Bytes, BytesMut}; -use futures::Stream; -use tokio::io::ReadBuf; + + +use std::ops::{RangeBounds}; + + + + + +use bytes::{Buf, BufMut}; + + use crate::raw::oio::BlockingRead; use crate::raw::*; diff --git a/core/src/types/operator/blocking_operator.rs b/core/src/types/operator/blocking_operator.rs index 06576986d3ee..28a888403a92 100644 --- a/core/src/types/operator/blocking_operator.rs +++ b/core/src/types/operator/blocking_operator.rs @@ -408,7 +408,7 @@ impl BlockingOperator { }; let (_, r) = inner.blocking_read(&path, args.with_range(range))?; - let mut r = BlockingReader::new(r); + let r = BlockingReader::new(r); let mut buf = Vec::with_capacity(size_hint as usize); r.read_to_end(&mut buf)?; diff --git a/core/src/types/reader.rs b/core/src/types/reader.rs index abfdb34d87ab..62347e0b5c14 100644 --- a/core/src/types/reader.rs +++ b/core/src/types/reader.rs @@ -17,13 +17,13 @@ use std::io; use std::io::SeekFrom; -use std::ops::{Bound, Range, RangeBounds}; -use std::pin::Pin; +use std::ops::{Bound, RangeBounds}; + use std::task::ready; use std::task::Context; use std::task::Poll; -use bytes::{Buf, BufMut, Bytes, BytesMut}; +use bytes::{Buf, BufMut}; use futures::Stream; use tokio::io::ReadBuf; @@ -239,7 +239,7 @@ mod futures_io_adapter { match &mut self.state { State::Idle(r) => { - let mut r = r.take().expect("reader must be valid"); + let r = r.take().expect("reader must be valid"); let size = buf.len(); let offset = self.offset; let fut = async move { @@ -269,10 +269,10 @@ mod futures_io_adapter { impl AsyncSeek for FuturesReader { fn poll_seek( mut self: Pin<&mut Self>, - cx: &mut Context<'_>, + _cx: &mut Context<'_>, pos: io::SeekFrom, ) -> Poll> { - use oio::Read; + match &mut self.state { State::Idle(_) => match pos { @@ -293,7 +293,7 @@ mod futures_io_adapter { Poll::Ready(Ok(self.offset)) } }, - State::Stating(fut) => { + State::Stating(_fut) => { todo!() } State::Reading(_) => Poll::Ready(Err(io::Error::new( @@ -354,7 +354,7 @@ mod tokio_io_adapter { match &mut self.state { State::Idle(r) => { - let mut r = r.take().expect("reader must be valid"); + let r = r.take().expect("reader must be valid"); let size = buf.remaining_mut(); let offset = self.offset; let fut = async move { @@ -402,7 +402,7 @@ mod tokio_io_adapter { Ok(()) } }, - State::Stating(fut) => { + State::Stating(_fut) => { todo!() } State::Reading(_) => Err(io::Error::new( @@ -412,7 +412,7 @@ mod tokio_io_adapter { } } - fn poll_complete(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + fn poll_complete(self: Pin<&mut Self>, _cx: &mut Context<'_>) -> Poll> { Poll::Ready(Ok(self.offset)) } } @@ -420,7 +420,7 @@ mod tokio_io_adapter { mod stream_adapter { use super::*; - use crate::raw::{oio, BoxedStaticFuture, RpStat}; + use crate::raw::{oio, BoxedStaticFuture}; use bytes::Bytes; use futures::Stream; use std::io; @@ -452,7 +452,7 @@ mod stream_adapter { match &mut self.state { State::Idle(r) => { - let mut r = r.take().expect("reader must be valid"); + let r = r.take().expect("reader must be valid"); let fut = async move { // TODO: should allow user to tune this value. let res = r.read_at_dyn(offset, 4 * 1024 * 1024).await; diff --git a/core/tests/behavior/async_read.rs b/core/tests/behavior/async_read.rs index e64ae883493b..aa8da5c3d611 100644 --- a/core/tests/behavior/async_read.rs +++ b/core/tests/behavior/async_read.rs @@ -156,7 +156,7 @@ pub async fn test_reader_range(op: Operator) -> anyhow::Result<()> { .await .expect("write must succeed"); - let mut r = op.reader_with(&path).range(offset..offset + length).await?; + let r = op.reader_with(&path).range(offset..offset + length).await?; let mut bs = Vec::new(); r.read_to_end(&mut bs).await?; @@ -186,7 +186,7 @@ pub async fn test_reader_range_with_buffer(op: Operator) -> anyhow::Result<()> { .await .expect("write must succeed"); - let mut r = op + let r = op .reader_with(&path) .range(offset..offset + length) .buffer(4096) @@ -220,7 +220,7 @@ pub async fn test_reader_from(op: Operator) -> anyhow::Result<()> { .await .expect("write must succeed"); - let mut r = op.reader_with(&path).range(offset..).await?; + let r = op.reader_with(&path).range(offset..).await?; let mut bs = Vec::new(); r.read_to_end(&mut bs).await?; @@ -248,7 +248,7 @@ pub async fn test_reader_from_with_buffer(op: Operator) -> anyhow::Result<()> { .await .expect("write must succeed"); - let mut r = op.reader_with(&path).range(offset..).buffer(4096).await?; + let r = op.reader_with(&path).range(offset..).buffer(4096).await?; let mut bs = Vec::new(); r.read_to_end(&mut bs).await?; @@ -276,7 +276,7 @@ pub async fn test_reader_tail(op: Operator) -> anyhow::Result<()> { .await .expect("write must succeed"); - let mut r = match op.reader_with(&path).range(..length).await { + let r = match op.reader_with(&path).range(..length).await { Ok(r) => r, // Not all services support range with tail range, let's tolerate this. Err(err) if err.kind() == ErrorKind::Unsupported => { @@ -312,7 +312,7 @@ pub async fn test_reader_tail_with_buffer(op: Operator) -> anyhow::Result<()> { .await .expect("write must succeed"); - let mut r = match op.reader_with(&path).range(..length).buffer(4096).await { + let r = match op.reader_with(&path).range(..length).buffer(4096).await { Ok(r) => r, // Not all services support range with tail range, let's tolerate this. Err(err) if err.kind() == ErrorKind::Unsupported => { @@ -636,7 +636,7 @@ pub async fn test_read_only_read_with_range(op: Operator) -> anyhow::Result<()> /// Read range should match. pub async fn test_read_only_reader_with_range(op: Operator) -> anyhow::Result<()> { - let mut r = op.reader_with("normal_file.txt").range(1024..2048).await?; + let r = op.reader_with("normal_file.txt").range(1024..2048).await?; let mut bs = Vec::new(); r.read_to_end(&mut bs).await?; @@ -653,7 +653,7 @@ pub async fn test_read_only_reader_with_range(op: Operator) -> anyhow::Result<() /// Read from should match. pub async fn test_read_only_reader_from(op: Operator) -> anyhow::Result<()> { - let mut r = op.reader_with("normal_file.txt").range(29458..).await?; + let r = op.reader_with("normal_file.txt").range(29458..).await?; let mut bs = Vec::new(); r.read_to_end(&mut bs).await?; From 06a095c65fbd7f611425d6c98d74fcc8a4fed311 Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Thu, 21 Mar 2024 15:40:37 +0800 Subject: [PATCH 056/111] refactor bytes range Signed-off-by: Xuanwo --- core/src/raw/http_util/bytes_range.rs | 170 ++++++------------- core/src/raw/ops.rs | 22 +-- core/src/types/operator/blocking_operator.rs | 19 +-- core/src/types/operator/operator.rs | 99 +---------- core/src/types/operator/operator_futures.rs | 17 +- 5 files changed, 73 insertions(+), 254 deletions(-) diff --git a/core/src/raw/http_util/bytes_range.rs b/core/src/raw/http_util/bytes_range.rs index 7b4e8312b394..93d4ee413ae3 100644 --- a/core/src/raw/http_util/bytes_range.rs +++ b/core/src/raw/http_util/bytes_range.rs @@ -37,22 +37,15 @@ use crate::Result; /// ```text /// Range: bytes=- /// Range: bytes=- -/// Range: bytes=- /// ``` /// /// # Notes /// -/// BytesRange support construct via rust native range syntax like `..`, `1024..`, `..2048`. -/// But it's has different semantic on `RangeTo`: `..`. -/// In rust, `..` means all items that `< end`, but in BytesRange, `..` means the -/// tailing part of content, a.k.a, the last `` bytes of content. -/// -/// - `0..1024` will be converted to header `range: bytes=0-1024` -/// - `..1024` will be converted to header `range: bytes=-1024` +/// We don't support tailing read like `Range: bytes=-` #[derive(Default, Debug, Clone, Copy, Eq, PartialEq)] pub struct BytesRange( /// Offset of the range. - Option, + u64, /// Size of the range. Option, ); @@ -68,12 +61,12 @@ impl BytesRange { /// /// - offset=None => `bytes=-`, read `` bytes from end. /// - offset=Some(0) => `bytes=0-`, read `` bytes from start. - pub fn new(offset: Option, size: Option) -> Self { + pub fn new(offset: u64, size: Option) -> Self { BytesRange(offset, size) } /// Get offset of BytesRange. - pub fn offset(&self) -> Option { + pub fn offset(&self) -> u64 { self.0 } @@ -86,7 +79,7 @@ impl BytesRange { /// /// If this range is full, we don't need to specify it in http request. pub fn is_full(&self) -> bool { - self.0.unwrap_or_default() == 0 && self.1.is_none() + self.0 == 0 && self.1.is_none() } /// Convert bytes range into Range header. @@ -102,77 +95,31 @@ impl BytesRange { /// Convert bytes range into rust range. pub fn to_range(&self) -> impl RangeBounds { ( - match self.0 { - Some(offset) => Bound::Included(offset), - None => Bound::Unbounded, - }, + Bound::Included(self.0), match self.1 { - Some(size) => Bound::Excluded(self.0.unwrap_or_default() + size), + Some(size) => Bound::Excluded(self.0 + size), None => Bound::Unbounded, }, ) } - /// Complete range with total size. - pub fn complete(&self, total_size: u64) -> Self { - match (self.offset(), self.size()) { - (Some(_), Some(_)) => *self, - (Some(offset), None) => Self(Some(offset), Some(total_size - offset)), - (None, Some(size)) => Self(Some(total_size - size), Some(size)), - (None, None) => Self(Some(0), Some(total_size)), - } - } - - /// Apply range on offset and limit. - /// - /// Returning `None ` - pub fn apply_on_offset(&self, offset: u64, limit: usize) -> Option { - match (self.offset(), self.size()) { - (Some(base), Some(size)) => { - if offset >= size { - None - } else { - Some(Self( - Some(base + offset), - Some((size - offset).min(limit as u64)), - )) - } - } - (Some(base), None) => Some(Self(Some(base + offset), Some(limit as u64))), - (None, None) => Some(Self(Some(offset), Some(limit as u64))), - (None, Some(_size)) => unimplemented!(), - } - } - /// apply_on_bytes will apply range on bytes. pub fn apply_on_bytes(&self, mut bs: Bytes) -> Bytes { - match (self.0, self.1) { - (None, None) => bs, - (None, Some(size)) => { - if size as usize >= bs.len() { - return bs; - } - bs.split_off(bs.len() - size as usize) - } - (Some(offset), None) => bs.split_off(offset as usize), - (Some(offset), Some(size)) => { - let mut bs = bs.split_off(offset as usize); - if (size as usize) < bs.len() { - let _ = bs.split_off(size as usize); - } - bs + let mut bs = bs.split_off(self.0 as usize); + if let Some(size) = self.1 { + if (size as usize) < bs.len() { + let _ = bs.split_off(size as usize); } } + bs } } impl Display for BytesRange { fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { - match (self.0, self.1) { - (Some(offset), None) => write!(f, "{offset}-"), - (None, Some(size)) => write!(f, "-{size}"), - (Some(offset), Some(size)) => write!(f, "{}-{}", offset, offset + size - 1), - (None, None) => write!(f, "0-"), + match self.1 { + None => write!(f, "{}-", self.0), + Some(size) => write!(f, "{}-{}", self.0, self.0 + size - 1), } } } @@ -210,20 +157,22 @@ impl FromStr for BytesRange { if v[1].is_empty() { // - Ok(BytesRange::new( - Some(v[0].parse().map_err(parse_int_error)?), + v[0].parse().map_err(parse_int_error)?, None, )) } else if v[0].is_empty() { // - - Ok(BytesRange::new( - None, - Some(v[1].parse::().map_err(parse_int_error)? + 1), - )) + Err(Error::new( + ErrorKind::Unexpected, + "header range with tailing is not supported", + ) + .with_operation("BytesRange::from_str") + .with_context("value", value)) } else { // - let start: u64 = v[0].parse().map_err(parse_int_error)?; let end: u64 = v[1].parse().map_err(parse_int_error)?; - Ok(BytesRange::new(Some(start), Some(end - start + 1))) + Ok(BytesRange::new(start, Some(end - start + 1))) } } } @@ -234,9 +183,9 @@ where { fn from(range: T) -> Self { let offset = match range.start_bound().cloned() { - Bound::Included(n) => Some(n), - Bound::Excluded(n) => Some(n + 1), - Bound::Unbounded => None, + Bound::Included(n) => n, + Bound::Excluded(n) => n + 1, + Bound::Unbounded => 0, }; let size = match range.end_bound().cloned() { Bound::Included(n) => Some(n + 1 - offset.unwrap_or_default()), @@ -254,69 +203,48 @@ mod tests { #[test] fn test_bytes_range_to_string() { - let h = BytesRange::new(None, Some(1024)); - assert_eq!(h.to_string(), "-1024"); - - let h = BytesRange::new(Some(0), Some(1024)); + let h = BytesRange::new(0, Some(1024)); assert_eq!(h.to_string(), "0-1023"); - let h = BytesRange::new(Some(1024), None); + let h = BytesRange::new(1024, None); assert_eq!(h.to_string(), "1024-"); - let h = BytesRange::new(Some(1024), Some(1024)); + let h = BytesRange::new(1024, Some(1024)); assert_eq!(h.to_string(), "1024-2047"); } #[test] fn test_bytes_range_to_header() { - let h = BytesRange::new(None, Some(1024)); - assert_eq!(h.to_header(), "bytes=-1024"); - - let h = BytesRange::new(Some(0), Some(1024)); + let h = BytesRange::new(0, Some(1024)); assert_eq!(h.to_header(), "bytes=0-1023"); - let h = BytesRange::new(Some(1024), None); + let h = BytesRange::new(1024, None); assert_eq!(h.to_header(), "bytes=1024-"); - let h = BytesRange::new(Some(1024), Some(1024)); + let h = BytesRange::new(1024, Some(1024)); assert_eq!(h.to_header(), "bytes=1024-2047"); } #[test] fn test_bytes_range_from_range_bounds() { - assert_eq!(BytesRange::new(None, None), BytesRange::from(..)); - assert_eq!(BytesRange::new(Some(10), None), BytesRange::from(10..)); - assert_eq!(BytesRange::new(None, Some(11)), BytesRange::from(..=10)); - assert_eq!(BytesRange::new(None, Some(10)), BytesRange::from(..10)); - assert_eq!( - BytesRange::new(Some(10), Some(10)), - BytesRange::from(10..20) - ); - assert_eq!( - BytesRange::new(Some(10), Some(11)), - BytesRange::from(10..=20) - ); + assert_eq!(BytesRange::new(0, None), BytesRange::from(..)); + assert_eq!(BytesRange::new(10, None), BytesRange::from(10..)); + assert_eq!(BytesRange::new(0, Some(11)), BytesRange::from(..=10)); + assert_eq!(BytesRange::new(0, Some(10)), BytesRange::from(..10)); + assert_eq!(BytesRange::new(10, Some(10)), BytesRange::from(10..20)); + assert_eq!(BytesRange::new(10, Some(11)), BytesRange::from(10..=20)); } #[test] fn test_bytes_range_from_str() -> Result<()> { let cases = vec![ - ( - "range-start", - "bytes=123-", - BytesRange::new(Some(123), None), - ), - ("suffix", "bytes=-123", BytesRange::new(None, Some(124))), - ( - "range", - "bytes=123-124", - BytesRange::new(Some(123), Some(2)), - ), - ("one byte", "bytes=0-0", BytesRange::new(Some(0), Some(1))), + ("range-start", "bytes=123-", BytesRange::new(123, None)), + ("range", "bytes=123-124", BytesRange::new(123, Some(2))), + ("one byte", "bytes=0-0", BytesRange::new(0, Some(1))), ( "lower case header", "bytes=0-0", - BytesRange::new(Some(0), Some(1)), + BytesRange::new(0, Some(1)), ), ]; @@ -334,17 +262,17 @@ mod tests { let bs = Bytes::from_static("Hello, World!".as_bytes()); let cases = vec![ - ("full", (None, None), "Hello, World!"), - ("with_offset", (Some(1), None), "ello, World!"), - ("with_size", (None, Some(1)), "!"), - ("with_larger_size", (None, Some(100)), "Hello, World!"), - ("with_offset_and_size", (Some(1), Some(1)), "e"), + ("full", (0, None), "Hello, World!"), + ("with_offset", (1, None), "ello, World!"), + ("with_size", (0, Some(1)), "!"), + ("with_larger_size", (0, Some(100)), "Hello, World!"), + ("with_offset_and_size", (1, Some(1)), "e"), ( "with_offset_and_larger_size", - (Some(1), Some(100)), + (1, Some(100)), "ello, World!", ), - ("with_empty_offset", (Some(0), Some(100)), "Hello, World!"), + ("with_empty_offset", (0, Some(100)), "Hello, World!"), ]; for (name, input, expected) in cases { diff --git a/core/src/raw/ops.rs b/core/src/raw/ops.rs index 955a5884210e..a87e2ce2d4af 100644 --- a/core/src/raw/ops.rs +++ b/core/src/raw/ops.rs @@ -300,16 +300,14 @@ impl BatchOperation { /// Args for `read` operation. #[derive(Debug, Clone, Default)] pub struct OpRead { - br: BytesRange, + /// NOTE: range is not apply for reader. + range: BytesRange, if_match: Option, if_none_match: Option, override_content_type: Option, override_cache_control: Option, override_content_disposition: Option, version: Option, - /// The maximum buffer capability. - /// `None` stand for disable buffer. - buffer: Option, } impl OpRead { @@ -337,13 +335,13 @@ impl OpRead { /// Create a new OpRead with range. pub fn with_range(mut self, range: BytesRange) -> Self { - self.br = range; + self.range = range; self } /// Get range from OpRead. pub fn range(&self) -> BytesRange { - self.br + self.range } /// Sets the content-disposition header that should be send back by the remote read operation. @@ -412,18 +410,6 @@ impl OpRead { pub fn version(&self) -> Option<&str> { self.version.as_deref() } - - /// Set the buffer capability. - pub fn with_buffer(mut self, buffer: usize) -> Self { - self.buffer = Some(buffer); - - self - } - - /// Get buffer from option. - pub fn buffer(&self) -> Option { - self.buffer - } } /// Args for `stat` operation. diff --git a/core/src/types/operator/blocking_operator.rs b/core/src/types/operator/blocking_operator.rs index 28a888403a92..6fe3ba0862c3 100644 --- a/core/src/types/operator/blocking_operator.rs +++ b/core/src/types/operator/blocking_operator.rs @@ -395,21 +395,10 @@ impl BlockingOperator { ); } - let range = args.range(); - let (size_hint, range) = if let Some(size) = range.size() { - (size, range) - } else { - let size = inner - .blocking_stat(&path, OpStat::default())? - .into_metadata() - .content_length(); - let range = range.complete(size); - (range.size().unwrap(), range) - }; - - let (_, r) = inner.blocking_read(&path, args.with_range(range))?; - let r = BlockingReader::new(r); - let mut buf = Vec::with_capacity(size_hint as usize); + let size_hint = args.range().size(); + + let r = BlockingReader::create(inner, &path, args)?; + let mut buf = Vec::with_capacity(size_hint.unwrap_or_default() as _); r.read_to_end(&mut buf)?; Ok(buf) diff --git a/core/src/types/operator/operator.rs b/core/src/types/operator/operator.rs index f7f3c3c8bc0f..35a737d7eecb 100644 --- a/core/src/types/operator/operator.rs +++ b/core/src/types/operator/operator.rs @@ -511,21 +511,10 @@ impl Operator { ); } - let range = args.range(); - let (size_hint, range) = if let Some(size) = range.size() { - (size, range) - } else { - let size = inner - .stat(&path, OpStat::default()) - .await? - .into_metadata() - .content_length(); - let range = range.complete(size); - (range.size().unwrap(), range) - }; - - let r = Reader::create(inner, &path, args.with_range(range)).await?; - let mut buf = Vec::with_capacity(size_hint as usize); + let size_hint = args.range().size(); + + let r = Reader::create(inner, &path, args).await?; + let mut buf = Vec::with_capacity(size_hint.unwrap_or_default() as _); r.read_to_end(&mut buf).await?; Ok(buf) }, @@ -568,86 +557,6 @@ impl Operator { /// /// # Options /// - /// ## `range` - /// - /// Set `range` for this `read` request. - /// - /// If we have a file with size `n`. - /// - /// - `..` means read bytes in range `[0, n)` of file. - /// - `0..1024` means read bytes in range `[0, 1024)` of file - /// - `1024..` means read bytes in range `[1024, n)` of file - /// - `..1024` means read bytes in range `(n - 1024, n)` of file - /// - /// ```no_run - /// # use opendal::Result; - /// # use opendal::Operator; - /// # use futures::TryStreamExt; - /// # async fn test(op: Operator) -> Result<()> { - /// let bs = op.reader_with("path/to/file").range(0..1024).await?; - /// # Ok(()) - /// # } - /// ``` - /// - /// ## `buffer` - /// - /// Set `buffer` for the reader. - /// - /// OpenDAL by default to read file without buffer. This is not efficient for cases like `seek` - /// after read or reading file with small chunks. To improve performance, we can set a buffer. - /// - /// The following example will create a reader with 4 MiB buffer internally. All seek operations - /// happened in buffered data will be zero cost. - /// - /// ```no_run - /// # use opendal::Result; - /// # use opendal::Operator; - /// # use futures::TryStreamExt; - /// # async fn test(op: Operator) -> Result<()> { - /// let bs = op - /// .reader_with("path/to/file") - /// .buffer(4 * 1024 * 1024) - /// .await?; - /// # Ok(()) - /// # } - /// ``` - /// - /// ## `if_match` - /// - /// Set `if_match` for this `read` request. - /// - /// This feature can be used to check if the file's `ETag` matches the given `ETag`. - /// - /// If file exists and it's etag doesn't match, an error with kind [`ErrorKind::ConditionNotMatch`] - /// will be returned. - /// - /// ```no_run - /// # use opendal::Result; - /// use opendal::Operator; - /// # async fn test(op: Operator, etag: &str) -> Result<()> { - /// let mut metadata = op.reader_with("path/to/file").if_match(etag).await?; - /// # Ok(()) - /// # } - /// ``` - /// - /// ## `if_none_match` - /// - /// Set `if_none_match` for this `read` request. - /// - /// This feature can be used to check if the file's `ETag` doesn't match the given `ETag`. - /// - /// If file exists and it's etag match, an error with kind [`ErrorKind::ConditionNotMatch`] - /// will be returned. - /// - /// ```no_run - /// # use opendal::Result; - /// use opendal::Operator; - /// # async fn test(op: Operator, etag: &str) -> Result<()> { - /// let mut metadata = op.reader_with("path/to/file").if_none_match(etag).await?; - /// # Ok(()) - /// # } - /// ``` - /// /// # Examples /// /// ```no_run diff --git a/core/src/types/operator/operator_futures.rs b/core/src/types/operator/operator_futures.rs index 2822db84ccac..2492a7c562db 100644 --- a/core/src/types/operator/operator_futures.rs +++ b/core/src/types/operator/operator_futures.rs @@ -213,11 +213,6 @@ impl FutureRead { self.map(|args| args.with_range(range.into())) } - /// Set the buffer capability to enable buffer for reader. - pub fn buffer(self, v: usize) -> Self { - self.map(|args| args.with_buffer(v)) - } - /// Set the If-Match for this operation. pub fn if_match(self, v: &str) -> Self { self.map(|args| args.with_if_match(v)) @@ -249,6 +244,18 @@ impl FutureRead { } } +/// Future that generated by [`Operator::read_with`] or [`Operator::reader_with`]. +/// +/// Users can add more options by public functions provided by this struct. +pub type FutureReader = OperatorFuture; + +impl FutureReader { + /// Set the version for this operation. + pub fn version(self, v: &str) -> Self { + self.map(|args| args.with_version(v)) + } +} + /// Future that generated by [`Operator::write_with`]. /// /// Users can add more options by public functions provided by this struct. From 9f828d90eaeb27684eb15bd8259682dc118dd093 Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Thu, 21 Mar 2024 15:58:22 +0800 Subject: [PATCH 057/111] Fix build Signed-off-by: Xuanwo --- core/benches/ops/read.rs | 48 +--- core/src/raw/adapters/kv/backend.rs | 30 +-- core/src/raw/adapters/typed_kv/backend.rs | 25 +- core/src/raw/http_util/bytes_range.rs | 4 +- core/src/services/alluxio/core.rs | 8 - core/src/services/alluxio/reader.rs | 5 +- core/src/services/azblob/reader.rs | 5 +- core/src/services/azdls/reader.rs | 6 +- core/src/services/azfile/core.rs | 10 - core/src/services/azfile/lister.rs | 1 + core/src/services/azfile/reader.rs | 4 +- core/src/services/b2/reader.rs | 4 +- core/src/services/chainsafe/reader.rs | 6 +- core/src/services/cos/reader.rs | 6 +- core/src/services/dbfs/reader.rs | 4 +- core/src/services/dropbox/reader.rs | 6 +- core/src/services/fs/reader.rs | 2 +- core/src/services/ftp/reader.rs | 51 ++-- core/src/services/gcs/reader.rs | 6 +- core/src/services/gdrive/reader.rs | 6 +- core/src/services/ghac/backend.rs | 13 +- core/src/services/ghac/reader.rs | 8 +- core/src/services/github/reader.rs | 6 +- core/src/services/hdfs/reader.rs | 3 +- core/src/services/http/reader.rs | 8 +- core/src/services/huggingface/reader.rs | 6 +- core/src/services/icloud/reader.rs | 6 +- core/src/services/ipfs/reader.rs | 8 +- core/src/services/ipmfs/backend.rs | 4 +- core/src/services/ipmfs/reader.rs | 8 +- core/src/services/koofr/reader.rs | 6 +- core/src/services/obs/reader.rs | 6 +- core/src/services/onedrive/reader.rs | 8 +- core/src/services/oss/reader.rs | 6 +- core/src/services/pcloud/reader.rs | 6 +- core/src/services/s3/reader.rs | 4 +- core/src/services/seafile/reader.rs | 6 +- core/src/services/sftp/writer.rs | 2 +- core/src/services/supabase/reader.rs | 6 +- core/src/services/swift/reader.rs | 6 +- core/src/services/upyun/reader.rs | 6 +- core/src/services/vercel_artifacts/reader.rs | 8 +- core/src/services/vercel_blob/reader.rs | 6 +- core/src/services/webdav/reader.rs | 6 +- core/src/services/webhdfs/backend.rs | 12 +- core/src/services/webhdfs/reader.rs | 8 +- core/src/services/yandex_disk/reader.rs | 6 +- core/src/types/operator/operator.rs | 6 +- core/src/types/operator/operator_functions.rs | 6 - core/src/types/operator/operator_futures.rs | 8 +- core/src/types/reader.rs | 4 +- core/tests/behavior/async_read.rs | 252 ------------------ 52 files changed, 107 insertions(+), 584 deletions(-) diff --git a/core/benches/ops/read.rs b/core/benches/ops/read.rs index b2bbe871d355..795c05f1fb75 100644 --- a/core/benches/ops/read.rs +++ b/core/benches/ops/read.rs @@ -28,7 +28,6 @@ use super::utils::*; pub fn bench(c: &mut Criterion) { if let Some(op) = init_test_service().unwrap() { bench_read_full(c, op.info().scheme().into_static(), op.clone()); - bench_read_part(c, op.info().scheme().into_static(), op.clone()); bench_read_parallel(c, op.info().scheme().into_static(), op.clone()); } } @@ -51,11 +50,7 @@ fn bench_read_full(c: &mut Criterion, name: &str, op: Operator) { group.throughput(criterion::Throughput::Bytes(size.bytes() as u64)); group.bench_with_input(size.to_string(), &(op.clone(), &path), |b, (op, path)| { b.to_async(&*TEST_RUNTIME).iter(|| async { - let r = op - .reader_with(path) - .range(0..=size.bytes() as u64) - .await - .unwrap(); + let r = op.reader_with(path).await.unwrap(); let r = r.into_futures_read(); io::copy(r, &mut io::sink()).await.unwrap(); }) @@ -67,38 +62,6 @@ fn bench_read_full(c: &mut Criterion, name: &str, op: Operator) { group.finish() } -/// Read from 1/4 to 3/4 and than drop the reader without consuming all data; -fn bench_read_part(c: &mut Criterion, name: &str, op: Operator) { - let mut group = c.benchmark_group(format!("service_{name}_read_part")); - - let mut rng = thread_rng(); - - for size in [ - Size::from_kibibytes(4), - Size::from_kibibytes(256), - Size::from_mebibytes(4), - Size::from_mebibytes(16), - ] { - let content = gen_bytes(&mut rng, (size.bytes() * 2) as usize); - let path = uuid::Uuid::new_v4().to_string(); - let offset = (size.bytes() / 2) as u64; - let temp_data = TempData::generate(op.clone(), &path, content.clone()); - - group.throughput(criterion::Throughput::Bytes(size.bytes() as u64)); - group.bench_with_input(size.to_string(), &(op.clone(), &path), |b, (op, path)| { - b.to_async(&*TEST_RUNTIME).iter(|| async { - let r = op.reader_with(path).range(offset..).await.unwrap(); - let r = r.into_futures_read(); - io::copy(r, &mut io::sink()).await.unwrap(); - }) - }); - - std::mem::drop(temp_data); - } - - group.finish() -} - fn bench_read_parallel(c: &mut Criterion, name: &str, op: Operator) { let mut group = c.benchmark_group(format!("service_{name}_read_parallel")); @@ -121,16 +84,15 @@ fn bench_read_parallel(c: &mut Criterion, name: &str, op: Operator) { group.bench_with_input( format!("{}x{}", parallel, size.to_string()), &(op.clone(), &path, buf_size), - |b, (op, path, _buf_size)| { + |b, (op, path, buf_size)| { b.to_async(&*TEST_RUNTIME).iter(|| async { let futures = (0..parallel) .map(|_| async { - let _r = op - .reader_with(path) - .range(offset..=offset + size.bytes() as u64) + let mut buf = Vec::with_capacity(*buf_size); + let r = op.reader_with(path).await.unwrap(); + r.read_range(&mut buf, offset..=offset + size.bytes() as u64) .await .unwrap(); - // r.read_exact(*buf_size).await.unwrap(); let mut d = 0; // mock same little cpu work diff --git a/core/src/raw/adapters/kv/backend.rs b/core/src/raw/adapters/kv/backend.rs index 1741e102698a..05a02b76acb5 100644 --- a/core/src/raw/adapters/kv/backend.rs +++ b/core/src/raw/adapters/kv/backend.rs @@ -104,9 +104,10 @@ impl Accessor for Backend { None => return Err(Error::new(ErrorKind::NotFound, "kv doesn't have this path")), }; - let bs = self.apply_range(bs, args.range()); + let bs = Bytes::from(bs); + let bs = args.range().apply_on_bytes(bs); - Ok((RpRead::new(), Bytes::from(bs))) + Ok((RpRead::new(), bs)) } fn blocking_read(&self, path: &str, args: OpRead) -> Result<(RpRead, Self::BlockingReader)> { @@ -117,8 +118,9 @@ impl Accessor for Backend { None => return Err(Error::new(ErrorKind::NotFound, "kv doesn't have this path")), }; - let bs = self.apply_range(bs, args.range()); - Ok((RpRead::new(), Bytes::from(bs))) + let bs = Bytes::from(bs); + let bs = args.range().apply_on_bytes(bs); + Ok((RpRead::new(), bs)) } async fn write(&self, path: &str, _: OpWrite) -> Result<(RpWrite, Self::Writer)> { @@ -198,26 +200,6 @@ impl Accessor for Backend { } } -impl Backend -where - S: Adapter, -{ - fn apply_range(&self, mut bs: Vec, br: BytesRange) -> Vec { - match (br.offset(), br.size()) { - (Some(offset), Some(size)) => { - let mut bs = bs.split_off(offset as usize); - if (size as usize) < bs.len() { - let _ = bs.split_off(size as usize); - } - bs - } - (Some(offset), None) => bs.split_off(offset as usize), - (None, Some(size)) => bs.split_off(bs.len() - size as usize), - (None, None) => bs, - } - } -} - pub struct KvLister { root: String, inner: IntoIter, diff --git a/core/src/raw/adapters/typed_kv/backend.rs b/core/src/raw/adapters/typed_kv/backend.rs index 03c79cea9136..cf4e0136ed04 100644 --- a/core/src/raw/adapters/typed_kv/backend.rs +++ b/core/src/raw/adapters/typed_kv/backend.rs @@ -111,8 +111,7 @@ impl Accessor for Backend { None => return Err(Error::new(ErrorKind::NotFound, "kv doesn't have this path")), }; - let bs = self.apply_range(bs, args.range()); - + let bs = args.range().apply_on_bytes(bs); Ok((RpRead::new(), bs)) } @@ -125,7 +124,7 @@ impl Accessor for Backend { None => return Err(Error::new(ErrorKind::NotFound, "kv doesn't have this path")), }; - let bs = self.apply_range(bs, args.range()); + let bs = args.range().apply_on_bytes(bs); Ok((RpRead::new(), bs)) } @@ -202,26 +201,6 @@ impl Accessor for Backend { } } -impl Backend -where - S: Adapter, -{ - fn apply_range(&self, mut bs: Bytes, br: BytesRange) -> Bytes { - match (br.offset(), br.size()) { - (Some(offset), Some(size)) => { - let mut bs = bs.split_off(offset as usize); - if (size as usize) < bs.len() { - let _ = bs.split_off(size as usize); - } - bs - } - (Some(offset), None) => bs.split_off(offset as usize), - (None, Some(size)) => bs.split_off(bs.len() - size as usize), - (None, None) => bs, - } - } -} - pub struct KvLister { root: String, inner: IntoIter, diff --git a/core/src/raw/http_util/bytes_range.rs b/core/src/raw/http_util/bytes_range.rs index 93d4ee413ae3..d112ebaf6990 100644 --- a/core/src/raw/http_util/bytes_range.rs +++ b/core/src/raw/http_util/bytes_range.rs @@ -188,8 +188,8 @@ where Bound::Unbounded => 0, }; let size = match range.end_bound().cloned() { - Bound::Included(n) => Some(n + 1 - offset.unwrap_or_default()), - Bound::Excluded(n) => Some(n - offset.unwrap_or_default()), + Bound::Included(n) => Some(n + 1 - offset), + Bound::Excluded(n) => Some(n - offset), Bound::Unbounded => None, }; diff --git a/core/src/services/alluxio/core.rs b/core/src/services/alluxio/core.rs index 53f0f3bc50ea..718e7a5315fa 100644 --- a/core/src/services/alluxio/core.rs +++ b/core/src/services/alluxio/core.rs @@ -306,14 +306,6 @@ impl AlluxioCore { )); if !range.is_full() { - // alluxio doesn't support read with suffix range. - if range.offset().is_none() && range.size().is_some() { - return Err(Error::new( - ErrorKind::Unsupported, - "azblob doesn't support read with suffix range", - )); - } - req = req.header(RANGE, range.to_header()); } diff --git a/core/src/services/alluxio/reader.rs b/core/src/services/alluxio/reader.rs index ba6bb16a6095..804227800a27 100644 --- a/core/src/services/alluxio/reader.rs +++ b/core/src/services/alluxio/reader.rs @@ -15,7 +15,6 @@ // specific language governing permissions and limitations // under the License. - use std::sync::Arc; use super::core::*; @@ -42,9 +41,7 @@ impl AlluxioReader { impl oio::Read for AlluxioReader { async fn read_at(&self, offset: u64, limit: usize) -> Result { - let Some(range) = self.op.range().apply_on_offset(offset, limit) else { - return Ok(oio::Buffer::new()); - }; + let range = BytesRange::new(offset, Some(limit as u64)); let resp = self.core.read(self.stream_id, range).await?; diff --git a/core/src/services/azblob/reader.rs b/core/src/services/azblob/reader.rs index bea274ffe604..077a68f22a0d 100644 --- a/core/src/services/azblob/reader.rs +++ b/core/src/services/azblob/reader.rs @@ -15,7 +15,6 @@ // specific language governing permissions and limitations // under the License. - use std::sync::Arc; use http::StatusCode; @@ -44,9 +43,7 @@ impl AzblobReader { impl oio::Read for AzblobReader { async fn read_at(&self, offset: u64, limit: usize) -> Result { - let Some(range) = self.op.range().apply_on_offset(offset, limit) else { - return Ok(oio::Buffer::new()); - }; + let range = BytesRange::new(offset, Some(limit as u64)); let resp = self .core diff --git a/core/src/services/azdls/reader.rs b/core/src/services/azdls/reader.rs index 811b5a32ca23..1c726e2c37d2 100644 --- a/core/src/services/azdls/reader.rs +++ b/core/src/services/azdls/reader.rs @@ -17,7 +17,7 @@ use super::core::AzdlsCore; use super::error::parse_error; -use crate::raw::{oio, OpRead}; +use crate::raw::*; use http::StatusCode; use std::sync::Arc; @@ -41,9 +41,7 @@ impl AzdlsReader { impl oio::Read for AzdlsReader { async fn read_at(&self, offset: u64, limit: usize) -> crate::Result { - let Some(range) = self.op.range().apply_on_offset(offset, limit) else { - return Ok(oio::Buffer::new()); - }; + let range = BytesRange::new(offset, Some(limit as u64)); let resp = self.core.azdls_read(&self.path, range).await?; diff --git a/core/src/services/azfile/core.rs b/core/src/services/azfile/core.rs index 1fa0814dc700..a332ca081edd 100644 --- a/core/src/services/azfile/core.rs +++ b/core/src/services/azfile/core.rs @@ -114,16 +114,6 @@ impl AzfileCore { let mut req = Request::get(&url); if !range.is_full() { - // azfile doesn't support read with suffix range. - // - // ref: https://learn.microsoft.com/en-us/rest/api/storageservices/specifying-the-range-header-for-file-service-operations - if range.offset().is_none() && range.size().is_some() { - return Err(Error::new( - ErrorKind::Unsupported, - "azblob doesn't support read with suffix range", - )); - } - req = req.header(RANGE, range.to_header()); } diff --git a/core/src/services/azfile/lister.rs b/core/src/services/azfile/lister.rs index 2ee372eba1fb..11cbe99a10df 100644 --- a/core/src/services/azfile/lister.rs +++ b/core/src/services/azfile/lister.rs @@ -150,6 +150,7 @@ struct Properties { #[cfg(test)] mod tests { use super::*; + use quick_xml::de::from_str; #[test] fn test_parse_list_result() { diff --git a/core/src/services/azfile/reader.rs b/core/src/services/azfile/reader.rs index acbcee310bd3..93292ebe98df 100644 --- a/core/src/services/azfile/reader.rs +++ b/core/src/services/azfile/reader.rs @@ -41,9 +41,7 @@ impl AzfileReader { impl oio::Read for AzfileReader { async fn read_at(&self, offset: u64, limit: usize) -> Result { - let Some(range) = self.op.range().apply_on_offset(offset, limit) else { - return Ok(oio::Buffer::new()); - }; + let range = BytesRange::new(offset, Some(limit as u64)); let resp = self.core.azfile_read(&self.path, range).await?; diff --git a/core/src/services/b2/reader.rs b/core/src/services/b2/reader.rs index 0837279ad3d3..17455e1eed21 100644 --- a/core/src/services/b2/reader.rs +++ b/core/src/services/b2/reader.rs @@ -41,9 +41,7 @@ impl B2Reader { impl oio::Read for B2Reader { async fn read_at(&self, offset: u64, limit: usize) -> Result { - let Some(range) = self.op.range().apply_on_offset(offset, limit) else { - return Ok(oio::Buffer::new()); - }; + let range = BytesRange::new(offset, Some(limit as u64)); let resp = self .core diff --git a/core/src/services/chainsafe/reader.rs b/core/src/services/chainsafe/reader.rs index 8e3540956154..c4b363ae2309 100644 --- a/core/src/services/chainsafe/reader.rs +++ b/core/src/services/chainsafe/reader.rs @@ -17,7 +17,7 @@ use super::core::ChainsafeCore; use super::error::parse_error; -use crate::raw::{oio, OpRead}; +use crate::raw::*; use http::StatusCode; use std::sync::Arc; @@ -41,9 +41,7 @@ impl ChainsafeReader { impl oio::Read for ChainsafeReader { async fn read_at(&self, offset: u64, limit: usize) -> crate::Result { - let Some(range) = self.op.range().apply_on_offset(offset, limit) else { - return Ok(oio::Buffer::new()); - }; + let range = BytesRange::new(offset, Some(limit as u64)); let resp = self.core.download_object(&self.path, range).await?; diff --git a/core/src/services/cos/reader.rs b/core/src/services/cos/reader.rs index e830d23aba35..558349f425d0 100644 --- a/core/src/services/cos/reader.rs +++ b/core/src/services/cos/reader.rs @@ -17,7 +17,7 @@ use super::core::CosCore; use super::error::parse_error; -use crate::raw::{oio, OpRead}; +use crate::raw::*; use http::StatusCode; use std::sync::Arc; @@ -40,9 +40,7 @@ impl CosReader { impl oio::Read for CosReader { async fn read_at(&self, offset: u64, limit: usize) -> crate::Result { - let Some(range) = self.op.range().apply_on_offset(offset, limit) else { - return Ok(oio::Buffer::new()); - }; + let range = BytesRange::new(offset, Some(limit as u64)); let resp = self .core diff --git a/core/src/services/dbfs/reader.rs b/core/src/services/dbfs/reader.rs index 3bc018623cd2..1c5858855855 100644 --- a/core/src/services/dbfs/reader.rs +++ b/core/src/services/dbfs/reader.rs @@ -15,8 +15,6 @@ // specific language governing permissions and limitations // under the License. - - use std::sync::Arc; use base64::engine::general_purpose; @@ -46,7 +44,7 @@ impl DbfsReader { DbfsReader { core, path, - offset: op.range().offset().unwrap_or(0), + offset: op.range().offset(), has_filled: 0, } } diff --git a/core/src/services/dropbox/reader.rs b/core/src/services/dropbox/reader.rs index 8d1742269865..89547506bcba 100644 --- a/core/src/services/dropbox/reader.rs +++ b/core/src/services/dropbox/reader.rs @@ -17,7 +17,7 @@ use super::core::DropboxCore; use super::error::parse_error; -use crate::raw::{oio, OpRead}; +use crate::raw::*; use http::StatusCode; use std::sync::Arc; @@ -41,9 +41,7 @@ impl DropboxReader { impl oio::Read for DropboxReader { async fn read_at(&self, offset: u64, limit: usize) -> crate::Result { - let Some(range) = self.op.range().apply_on_offset(offset, limit) else { - return Ok(oio::Buffer::new()); - }; + let range = BytesRange::new(offset, Some(limit as u64)); let resp = self.core.dropbox_get(&self.path, range, &self.op).await?; diff --git a/core/src/services/fs/reader.rs b/core/src/services/fs/reader.rs index 881a5d4b473e..ad6ba8d416e4 100644 --- a/core/src/services/fs/reader.rs +++ b/core/src/services/fs/reader.rs @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -use crate::raw::{new_std_io_error, oio}; +use crate::raw::*; use crate::*; use tokio::io::ReadBuf; diff --git a/core/src/services/ftp/reader.rs b/core/src/services/ftp/reader.rs index bb0e6c1ffe5d..25eb90ffe169 100644 --- a/core/src/services/ftp/reader.rs +++ b/core/src/services/ftp/reader.rs @@ -17,7 +17,7 @@ use super::backend::FtpBackend; use super::err::parse_error; -use crate::raw::{new_std_io_error, oio, OpRead, Operation}; +use crate::raw::*; use crate::*; use futures::AsyncReadExt; @@ -40,43 +40,22 @@ impl FtpReader { impl oio::Read for FtpReader { async fn read_at(&self, offset: u64, limit: usize) -> Result { - let Some(range) = self.op.range().apply_on_offset(offset, limit) else { - return Ok(oio::Buffer::new()); - }; - let mut ftp_stream = self.core.ftp_connect(Operation::Read).await?; - let _meta = self.core.ftp_stat(&self.path).await?; - - match (range.offset(), range.size()) { - (Some(offset), Some(size)) => { - ftp_stream - .resume_transfer(offset as usize) - .await - .map_err(parse_error)?; - let mut ds = ftp_stream - .retr_as_stream(&self.path) - .await - .map_err(parse_error)? - .take(size); - let mut bs = Vec::with_capacity(size as usize); - ds.read_to_end(&mut bs).await.map_err(new_std_io_error)?; - Ok(oio::Buffer::from(bs)) - } - (Some(offset), None) => { - ftp_stream - .resume_transfer(offset as usize) - .await - .map_err(parse_error)?; - let mut ds = ftp_stream - .retr_as_stream(&self.path) - .await - .map_err(parse_error)?; - let mut bs = vec![]; - ds.read_to_end(&mut bs).await.map_err(new_std_io_error)?; - Ok(oio::Buffer::from(bs)) - } - _ => unimplemented!(), + if offset != 0 { + ftp_stream + .resume_transfer(offset as usize) + .await + .map_err(parse_error)?; } + + let mut ds = ftp_stream + .retr_as_stream(&self.path) + .await + .map_err(parse_error)? + .take(limit as _); + let mut bs = Vec::with_capacity(limit); + ds.read_to_end(&mut bs).await.map_err(new_std_io_error)?; + Ok(oio::Buffer::from(bs)) } } diff --git a/core/src/services/gcs/reader.rs b/core/src/services/gcs/reader.rs index e0cea1a3d983..da42c3cb7d65 100644 --- a/core/src/services/gcs/reader.rs +++ b/core/src/services/gcs/reader.rs @@ -17,7 +17,7 @@ use super::core::GcsCore; use super::error::parse_error; -use crate::raw::{oio, OpRead}; +use crate::raw::*; use http::StatusCode; use std::sync::Arc; @@ -41,9 +41,7 @@ impl GcsReader { impl oio::Read for GcsReader { async fn read_at(&self, offset: u64, limit: usize) -> crate::Result { - let Some(range) = self.op.range().apply_on_offset(offset, limit) else { - return Ok(oio::Buffer::new()); - }; + let range = BytesRange::new(offset, Some(limit as u64)); let resp = self .core diff --git a/core/src/services/gdrive/reader.rs b/core/src/services/gdrive/reader.rs index 348cef06255f..cccc515003a7 100644 --- a/core/src/services/gdrive/reader.rs +++ b/core/src/services/gdrive/reader.rs @@ -17,7 +17,7 @@ use super::core::GdriveCore; use super::error::parse_error; -use crate::raw::{oio, OpRead}; +use crate::raw::*; use http::StatusCode; use std::sync::Arc; @@ -41,9 +41,7 @@ impl GdriveReader { impl oio::Read for GdriveReader { async fn read_at(&self, offset: u64, limit: usize) -> crate::Result { - let Some(range) = self.op.range().apply_on_offset(offset, limit) else { - return Ok(oio::Buffer::new()); - }; + let range = BytesRange::new(offset, Some(limit as u64)); let resp = self.core.gdrive_get(&self.path, range).await?; diff --git a/core/src/services/ghac/backend.rs b/core/src/services/ghac/backend.rs index 4fc3adbf7754..057ec753299a 100644 --- a/core/src/services/ghac/backend.rs +++ b/core/src/services/ghac/backend.rs @@ -387,18 +387,7 @@ impl GhacBackend { let mut req = Request::get(location); if !range.is_full() { - // ghac is backed by azblob, and azblob doesn't support - // read with suffix range - // - // ref: https://learn.microsoft.com/en-us/rest/api/storageservices/specifying-the-range-header-for-blob-service-operations - if range.offset().is_none() && range.size().is_some() { - return Err(Error::new( - ErrorKind::Unsupported, - "ghac doesn't support read with suffix range", - )); - } - - req = req.header(http::header::RANGE, range.to_header()); + req = req.header(header::RANGE, range.to_header()); } req.body(AsyncBody::Empty).map_err(new_request_build_error) diff --git a/core/src/services/ghac/reader.rs b/core/src/services/ghac/reader.rs index 2a6a02fd5a2d..01b1682bbf74 100644 --- a/core/src/services/ghac/reader.rs +++ b/core/src/services/ghac/reader.rs @@ -16,12 +16,10 @@ // under the License. use super::error::parse_error; -use crate::raw::{oio, OpRead}; +use crate::raw::*; use crate::services::ghac::backend::GhacBackend; use http::StatusCode; - - pub struct GhacReader { core: GhacBackend, @@ -41,9 +39,7 @@ impl GhacReader { impl oio::Read for GhacReader { async fn read_at(&self, offset: u64, limit: usize) -> crate::Result { - let Some(range) = self.op.range().apply_on_offset(offset, limit) else { - return Ok(oio::Buffer::new()); - }; + let range = BytesRange::new(offset, Some(limit as u64)); let req = self.core.ghac_get_location(&self.location, range).await?; let resp = self.core.client.send(req).await?; diff --git a/core/src/services/github/reader.rs b/core/src/services/github/reader.rs index 650cbd06aac5..580599c53659 100644 --- a/core/src/services/github/reader.rs +++ b/core/src/services/github/reader.rs @@ -17,7 +17,7 @@ use super::core::GithubCore; use super::error::parse_error; -use crate::raw::{oio, OpRead}; +use crate::raw::*; use http::StatusCode; use std::sync::Arc; @@ -41,9 +41,7 @@ impl GithubReader { impl oio::Read for GithubReader { async fn read_at(&self, offset: u64, limit: usize) -> crate::Result { - let Some(range) = self.op.range().apply_on_offset(offset, limit) else { - return Ok(oio::Buffer::new()); - }; + let range = BytesRange::new(offset, Some(limit as u64)); let resp = self.core.get(&self.path, range).await?; diff --git a/core/src/services/hdfs/reader.rs b/core/src/services/hdfs/reader.rs index 91512abc7cc2..c1e6392b91d2 100644 --- a/core/src/services/hdfs/reader.rs +++ b/core/src/services/hdfs/reader.rs @@ -15,11 +15,10 @@ // specific language governing permissions and limitations // under the License. -use crate::raw::{new_std_io_error, oio}; +use crate::raw::*; use crate::{Error, ErrorKind}; use hdrs::File; - use std::sync::Arc; use tokio::io::ReadBuf; diff --git a/core/src/services/http/reader.rs b/core/src/services/http/reader.rs index 602da90be2ba..f981d324e726 100644 --- a/core/src/services/http/reader.rs +++ b/core/src/services/http/reader.rs @@ -16,12 +16,10 @@ // under the License. use super::error::parse_error; -use crate::raw::{oio, OpRead}; +use crate::raw::*; use crate::services::http::backend::HttpBackend; use http::StatusCode; - - pub struct HttpReader { core: HttpBackend, @@ -41,9 +39,7 @@ impl HttpReader { impl oio::Read for HttpReader { async fn read_at(&self, offset: u64, limit: usize) -> crate::Result { - let Some(range) = self.op.range().apply_on_offset(offset, limit) else { - return Ok(oio::Buffer::new()); - }; + let range = BytesRange::new(offset, Some(limit as u64)); let resp = self.core.http_get(&self.path, range, &self.op).await?; diff --git a/core/src/services/huggingface/reader.rs b/core/src/services/huggingface/reader.rs index a5404b06e838..6531a22f4ee7 100644 --- a/core/src/services/huggingface/reader.rs +++ b/core/src/services/huggingface/reader.rs @@ -17,7 +17,7 @@ use super::core::HuggingfaceCore; use super::error::parse_error; -use crate::raw::{oio, OpRead}; +use crate::raw::*; use http::StatusCode; use std::sync::Arc; @@ -41,9 +41,7 @@ impl HuggingfaceReader { impl oio::Read for HuggingfaceReader { async fn read_at(&self, offset: u64, limit: usize) -> crate::Result { - let Some(range) = self.op.range().apply_on_offset(offset, limit) else { - return Ok(oio::Buffer::new()); - }; + let range = BytesRange::new(offset, Some(limit as u64)); let resp = self.core.hf_resolve(&self.path, range, &self.op).await?; diff --git a/core/src/services/icloud/reader.rs b/core/src/services/icloud/reader.rs index 441203f33367..ee38c2324441 100644 --- a/core/src/services/icloud/reader.rs +++ b/core/src/services/icloud/reader.rs @@ -16,7 +16,7 @@ // under the License. use super::core::{parse_error, IcloudCore}; -use crate::raw::{oio, OpRead}; +use crate::raw::*; use http::StatusCode; use std::sync::Arc; @@ -40,9 +40,7 @@ impl IcloudReader { impl oio::Read for IcloudReader { async fn read_at(&self, offset: u64, limit: usize) -> crate::Result { - let Some(range) = self.op.range().apply_on_offset(offset, limit) else { - return Ok(oio::Buffer::new()); - }; + let range = BytesRange::new(offset, Some(limit as u64)); let resp = self.core.read(&self.path, range, &self.op).await?; diff --git a/core/src/services/ipfs/reader.rs b/core/src/services/ipfs/reader.rs index 50eaffb080aa..fd26e0913885 100644 --- a/core/src/services/ipfs/reader.rs +++ b/core/src/services/ipfs/reader.rs @@ -16,12 +16,10 @@ // under the License. use super::error::parse_error; -use crate::raw::{oio, OpRead}; +use crate::raw::*; use crate::services::ipfs::backend::IpfsBackend; use http::StatusCode; - - pub struct IpfsReader { core: IpfsBackend, @@ -41,9 +39,7 @@ impl IpfsReader { impl oio::Read for IpfsReader { async fn read_at(&self, offset: u64, limit: usize) -> crate::Result { - let Some(range) = self.op.range().apply_on_offset(offset, limit) else { - return Ok(oio::Buffer::new()); - }; + let range = BytesRange::new(offset, Some(limit as u64)); let resp = self.core.ipfs_get(&self.path, range).await?; diff --git a/core/src/services/ipmfs/backend.rs b/core/src/services/ipmfs/backend.rs index ed561e9663e4..9e8b7519efa6 100644 --- a/core/src/services/ipmfs/backend.rs +++ b/core/src/services/ipmfs/backend.rs @@ -194,9 +194,7 @@ impl IpmfsBackend { percent_encode_path(&p) ); - if let Some(offset) = range.offset() { - write!(url, "&offset={offset}").expect("write into string must succeed") - } + write!(url, "&offset={}", range.offset()).expect("write into string must succeed"); if let Some(count) = range.size() { write!(url, "&count={count}").expect("write into string must succeed") } diff --git a/core/src/services/ipmfs/reader.rs b/core/src/services/ipmfs/reader.rs index 682bcff1fa16..6727b8273900 100644 --- a/core/src/services/ipmfs/reader.rs +++ b/core/src/services/ipmfs/reader.rs @@ -16,12 +16,10 @@ // under the License. use super::error::parse_error; -use crate::raw::{oio, OpRead}; +use crate::raw::*; use crate::services::ipmfs::backend::IpmfsBackend; use http::StatusCode; - - pub struct IpmfsReader { core: IpmfsBackend, @@ -41,9 +39,7 @@ impl IpmfsReader { impl oio::Read for IpmfsReader { async fn read_at(&self, offset: u64, limit: usize) -> crate::Result { - let Some(range) = self.op.range().apply_on_offset(offset, limit) else { - return Ok(oio::Buffer::new()); - }; + let range = BytesRange::new(offset, Some(limit as u64)); let resp = self.core.ipmfs_read(&self.path, range).await?; diff --git a/core/src/services/koofr/reader.rs b/core/src/services/koofr/reader.rs index 0206a4413c6c..78a1b90b9c6b 100644 --- a/core/src/services/koofr/reader.rs +++ b/core/src/services/koofr/reader.rs @@ -17,7 +17,7 @@ use super::core::KoofrCore; use super::error::parse_error; -use crate::raw::{oio, OpRead}; +use crate::raw::*; use http::StatusCode; use std::sync::Arc; @@ -41,9 +41,7 @@ impl KoofrReader { impl oio::Read for KoofrReader { async fn read_at(&self, offset: u64, limit: usize) -> crate::Result { - let Some(range) = self.op.range().apply_on_offset(offset, limit) else { - return Ok(oio::Buffer::new()); - }; + let range = BytesRange::new(offset, Some(limit as u64)); let resp = self.core.get(&self.path, range).await?; diff --git a/core/src/services/obs/reader.rs b/core/src/services/obs/reader.rs index 5e81eb209c57..0f8301c156f5 100644 --- a/core/src/services/obs/reader.rs +++ b/core/src/services/obs/reader.rs @@ -17,7 +17,7 @@ use super::core::ObsCore; use super::error::parse_error; -use crate::raw::{oio, OpRead}; +use crate::raw::*; use http::StatusCode; use std::sync::Arc; @@ -41,9 +41,7 @@ impl ObsReader { impl oio::Read for ObsReader { async fn read_at(&self, offset: u64, limit: usize) -> crate::Result { - let Some(range) = self.op.range().apply_on_offset(offset, limit) else { - return Ok(oio::Buffer::new()); - }; + let range = BytesRange::new(offset, Some(limit as u64)); let resp = self .core diff --git a/core/src/services/onedrive/reader.rs b/core/src/services/onedrive/reader.rs index 3a250d7847b3..a9b4f2cf97c8 100644 --- a/core/src/services/onedrive/reader.rs +++ b/core/src/services/onedrive/reader.rs @@ -16,12 +16,10 @@ // under the License. use super::error::parse_error; -use crate::raw::{oio, OpRead}; +use crate::raw::*; use crate::services::onedrive::backend::OnedriveBackend; use http::StatusCode; - - pub struct OnedriveReader { core: OnedriveBackend, @@ -41,9 +39,7 @@ impl OnedriveReader { impl oio::Read for OnedriveReader { async fn read_at(&self, offset: u64, limit: usize) -> crate::Result { - let Some(range) = self.op.range().apply_on_offset(offset, limit) else { - return Ok(oio::Buffer::new()); - }; + let range = BytesRange::new(offset, Some(limit as u64)); let resp = self.core.onedrive_get_content(&self.path, range).await?; diff --git a/core/src/services/oss/reader.rs b/core/src/services/oss/reader.rs index a803dcf0fbbc..fa2e692d3400 100644 --- a/core/src/services/oss/reader.rs +++ b/core/src/services/oss/reader.rs @@ -17,7 +17,7 @@ use super::core::OssCore; use super::error::parse_error; -use crate::raw::{oio, OpRead}; +use crate::raw::*; use http::StatusCode; use std::sync::Arc; @@ -41,9 +41,7 @@ impl OssReader { impl oio::Read for OssReader { async fn read_at(&self, offset: u64, limit: usize) -> crate::Result { - let Some(range) = self.op.range().apply_on_offset(offset, limit) else { - return Ok(oio::Buffer::new()); - }; + let range = BytesRange::new(offset, Some(limit as u64)); let resp = self .core diff --git a/core/src/services/pcloud/reader.rs b/core/src/services/pcloud/reader.rs index 0d87c2f37303..1dad52ff0a71 100644 --- a/core/src/services/pcloud/reader.rs +++ b/core/src/services/pcloud/reader.rs @@ -17,7 +17,7 @@ use super::core::PcloudCore; use super::error::parse_error; -use crate::raw::{oio, OpRead}; +use crate::raw::*; use http::StatusCode; use std::sync::Arc; @@ -41,9 +41,7 @@ impl PcloudReader { impl oio::Read for PcloudReader { async fn read_at(&self, offset: u64, limit: usize) -> crate::Result { - let Some(range) = self.op.range().apply_on_offset(offset, limit) else { - return Ok(oio::Buffer::new()); - }; + let range = BytesRange::new(offset, Some(limit as u64)); let resp = self.core.download(&self.link, range).await?; diff --git a/core/src/services/s3/reader.rs b/core/src/services/s3/reader.rs index d80b85cbb8cd..61a267c3b795 100644 --- a/core/src/services/s3/reader.rs +++ b/core/src/services/s3/reader.rs @@ -41,9 +41,7 @@ impl S3Reader { impl oio::Read for S3Reader { async fn read_at(&self, offset: u64, limit: usize) -> Result { - let Some(range) = self.op.range().apply_on_offset(offset, limit) else { - return Ok(oio::Buffer::new()); - }; + let range = BytesRange::new(offset, Some(limit as u64)); let resp = self.core.s3_get_object(&self.path, range, &self.op).await?; let status = resp.status(); diff --git a/core/src/services/seafile/reader.rs b/core/src/services/seafile/reader.rs index 0cf002bc94e6..1546b6a04db3 100644 --- a/core/src/services/seafile/reader.rs +++ b/core/src/services/seafile/reader.rs @@ -17,7 +17,7 @@ use super::core::SeafileCore; use super::error::parse_error; -use crate::raw::{oio, OpRead}; +use crate::raw::*; use http::StatusCode; use std::sync::Arc; @@ -41,9 +41,7 @@ impl SeafileReader { impl oio::Read for SeafileReader { async fn read_at(&self, offset: u64, limit: usize) -> crate::Result { - let Some(range) = self.op.range().apply_on_offset(offset, limit) else { - return Ok(oio::Buffer::new()); - }; + let range = BytesRange::new(offset, Some(limit as u64)); let resp = self.core.download_file(&self.path, range).await?; diff --git a/core/src/services/sftp/writer.rs b/core/src/services/sftp/writer.rs index 9bee111e2440..d06505c92da6 100644 --- a/core/src/services/sftp/writer.rs +++ b/core/src/services/sftp/writer.rs @@ -22,7 +22,7 @@ use openssh_sftp_client::file::File; use openssh_sftp_client::file::TokioCompatFile; use tokio::io::AsyncWriteExt; -use crate::raw::{new_std_io_error, oio}; +use crate::raw::*; use crate::*; pub struct SftpWriter { diff --git a/core/src/services/supabase/reader.rs b/core/src/services/supabase/reader.rs index ec22eb0ff59d..9293ff0e9961 100644 --- a/core/src/services/supabase/reader.rs +++ b/core/src/services/supabase/reader.rs @@ -17,7 +17,7 @@ use super::core::SupabaseCore; use super::error::parse_error; -use crate::raw::{oio, OpRead}; +use crate::raw::*; use http::StatusCode; use std::sync::Arc; @@ -41,9 +41,7 @@ impl SupabaseReader { impl oio::Read for SupabaseReader { async fn read_at(&self, offset: u64, limit: usize) -> crate::Result { - let Some(range) = self.op.range().apply_on_offset(offset, limit) else { - return Ok(oio::Buffer::new()); - }; + let range = BytesRange::new(offset, Some(limit as u64)); let resp = self.core.supabase_get_object(&self.path, range).await?; diff --git a/core/src/services/swift/reader.rs b/core/src/services/swift/reader.rs index 2312c9f99459..777530f14005 100644 --- a/core/src/services/swift/reader.rs +++ b/core/src/services/swift/reader.rs @@ -17,7 +17,7 @@ use super::core::SwiftCore; use super::error::parse_error; -use crate::raw::{oio, OpRead}; +use crate::raw::*; use http::StatusCode; use std::sync::Arc; @@ -41,9 +41,7 @@ impl SwiftReader { impl oio::Read for SwiftReader { async fn read_at(&self, offset: u64, limit: usize) -> crate::Result { - let Some(range) = self.op.range().apply_on_offset(offset, limit) else { - return Ok(oio::Buffer::new()); - }; + let range = BytesRange::new(offset, Some(limit as u64)); let resp = self.core.swift_read(&self.path, range, &self.op).await?; diff --git a/core/src/services/upyun/reader.rs b/core/src/services/upyun/reader.rs index 04b3f886347e..9846eb65cf0e 100644 --- a/core/src/services/upyun/reader.rs +++ b/core/src/services/upyun/reader.rs @@ -17,7 +17,7 @@ use super::core::UpyunCore; use super::error::parse_error; -use crate::raw::{oio, OpRead}; +use crate::raw::*; use http::StatusCode; use std::sync::Arc; @@ -41,9 +41,7 @@ impl UpyunReader { impl oio::Read for UpyunReader { async fn read_at(&self, offset: u64, limit: usize) -> crate::Result { - let Some(range) = self.op.range().apply_on_offset(offset, limit) else { - return Ok(oio::Buffer::new()); - }; + let range = BytesRange::new(offset, Some(limit as u64)); let resp = self.core.download_file(&self.path, range).await?; diff --git a/core/src/services/vercel_artifacts/reader.rs b/core/src/services/vercel_artifacts/reader.rs index 1bc64847d150..a36812ebc7db 100644 --- a/core/src/services/vercel_artifacts/reader.rs +++ b/core/src/services/vercel_artifacts/reader.rs @@ -16,12 +16,10 @@ // under the License. use super::error::parse_error; -use crate::raw::{oio, OpRead}; +use crate::raw::*; use crate::services::vercel_artifacts::backend::VercelArtifactsBackend; use http::StatusCode; - - pub struct VercelArtifactsReader { core: VercelArtifactsBackend, @@ -41,9 +39,7 @@ impl VercelArtifactsReader { impl oio::Read for VercelArtifactsReader { async fn read_at(&self, offset: u64, limit: usize) -> crate::Result { - let Some(range) = self.op.range().apply_on_offset(offset, limit) else { - return Ok(oio::Buffer::new()); - }; + let range = BytesRange::new(offset, Some(limit as u64)); let resp = self .core diff --git a/core/src/services/vercel_blob/reader.rs b/core/src/services/vercel_blob/reader.rs index 71b3af1e03a0..b41866616ed0 100644 --- a/core/src/services/vercel_blob/reader.rs +++ b/core/src/services/vercel_blob/reader.rs @@ -17,7 +17,7 @@ use super::core::VercelBlobCore; use super::error::parse_error; -use crate::raw::{oio, OpRead}; +use crate::raw::*; use http::StatusCode; use std::sync::Arc; @@ -41,9 +41,7 @@ impl VercelBlobReader { impl oio::Read for VercelBlobReader { async fn read_at(&self, offset: u64, limit: usize) -> crate::Result { - let Some(range) = self.op.range().apply_on_offset(offset, limit) else { - return Ok(oio::Buffer::new()); - }; + let range = BytesRange::new(offset, Some(limit as u64)); let resp = self.core.download(&self.path, range, &self.op).await?; diff --git a/core/src/services/webdav/reader.rs b/core/src/services/webdav/reader.rs index 7cc8f8d954dd..a87793a70153 100644 --- a/core/src/services/webdav/reader.rs +++ b/core/src/services/webdav/reader.rs @@ -17,7 +17,7 @@ use super::core::WebdavCore; use super::error::parse_error; -use crate::raw::{oio, OpRead}; +use crate::raw::*; use http::StatusCode; use std::sync::Arc; @@ -41,9 +41,7 @@ impl WebdavReader { impl oio::Read for WebdavReader { async fn read_at(&self, offset: u64, limit: usize) -> crate::Result { - let Some(range) = self.op.range().apply_on_offset(offset, limit) else { - return Ok(oio::Buffer::new()); - }; + let range = BytesRange::new(offset, Some(limit as u64)); let resp = self.core.webdav_get(&self.path, range, &self.op).await?; diff --git a/core/src/services/webhdfs/backend.rs b/core/src/services/webhdfs/backend.rs index 93a12131cdf2..cbd02a522eaf 100644 --- a/core/src/services/webhdfs/backend.rs +++ b/core/src/services/webhdfs/backend.rs @@ -410,17 +410,7 @@ impl WebhdfsBackend { } if !range.is_full() { - // Webhdfs does not support read from end - if range.offset().is_none() && range.size().is_some() { - return Err(Error::new( - ErrorKind::Unsupported, - "webhdfs doesn't support read with suffix range", - )); - }; - - if let Some(offset) = range.offset() { - url += &format!("&offset={offset}"); - } + url += &format!("&offset={}", range.offset()); if let Some(size) = range.size() { url += &format!("&length={size}") } diff --git a/core/src/services/webhdfs/reader.rs b/core/src/services/webhdfs/reader.rs index b980e5ad9e28..19474b188e35 100644 --- a/core/src/services/webhdfs/reader.rs +++ b/core/src/services/webhdfs/reader.rs @@ -16,13 +16,11 @@ // under the License. use super::error::{parse_error, parse_error_msg}; -use crate::raw::{oio, OpRead}; +use crate::raw::*; use crate::services::webhdfs::backend::WebhdfsBackend; use bytes::Buf; use http::StatusCode; - - pub struct WebhdfsReader { core: WebhdfsBackend, @@ -42,9 +40,7 @@ impl WebhdfsReader { impl oio::Read for WebhdfsReader { async fn read_at(&self, offset: u64, limit: usize) -> crate::Result { - let Some(range) = self.op.range().apply_on_offset(offset, limit) else { - return Ok(oio::Buffer::new()); - }; + let range = BytesRange::new(offset, Some(limit as u64)); let resp = self.core.webhdfs_read_file(&self.path, range).await?; diff --git a/core/src/services/yandex_disk/reader.rs b/core/src/services/yandex_disk/reader.rs index 2103b574bbd4..e3c6eddce394 100644 --- a/core/src/services/yandex_disk/reader.rs +++ b/core/src/services/yandex_disk/reader.rs @@ -17,7 +17,7 @@ use super::core::YandexDiskCore; use super::error::parse_error; -use crate::raw::{new_request_build_error, oio, AsyncBody, OpRead}; +use crate::raw::*; use http::{header, Request, StatusCode}; use std::sync::Arc; @@ -41,9 +41,7 @@ impl YandexDiskReader { impl oio::Read for YandexDiskReader { async fn read_at(&self, offset: u64, limit: usize) -> crate::Result { - let Some(range) = self.op.range().apply_on_offset(offset, limit) else { - return Ok(oio::Buffer::new()); - }; + let range = BytesRange::new(offset, Some(limit as u64)); // TODO: move this out of reader. let download_url = self.core.get_download_url(&self.path).await?; diff --git a/core/src/types/operator/operator.rs b/core/src/types/operator/operator.rs index 35a737d7eecb..70ab664b6092 100644 --- a/core/src/types/operator/operator.rs +++ b/core/src/types/operator/operator.rs @@ -568,13 +568,13 @@ impl Operator { /// # Ok(()) /// # } /// ``` - pub fn reader_with(&self, path: &str) -> FutureRead>> { + pub fn reader_with(&self, path: &str) -> FutureReader>> { let path = normalize_path(path); OperatorFuture::new( self.inner().clone(), path, - OpRead::default(), + (OpRead::default(), ()), |inner, path, args| async move { if !validate_path(&path, EntryMode::FILE) { return Err( @@ -585,7 +585,7 @@ impl Operator { ); } - Reader::create(inner.clone(), &path, args).await + Reader::create(inner.clone(), &path, args.0).await }, ) } diff --git a/core/src/types/operator/operator_functions.rs b/core/src/types/operator/operator_functions.rs index 07644e2089e5..4afa157f02e8 100644 --- a/core/src/types/operator/operator_functions.rs +++ b/core/src/types/operator/operator_functions.rs @@ -395,12 +395,6 @@ impl FunctionReader { pub fn call(self) -> Result { self.0.call() } - - /// Set the buffer capability to enable `BufferReader`. - pub fn buffer(mut self, cap: usize) -> Self { - self.0 = self.0.map_args(|args| args.with_buffer(cap)); - self - } } /// Function that generated by [`BlockingOperator::stat_with`]. diff --git a/core/src/types/operator/operator_futures.rs b/core/src/types/operator/operator_futures.rs index 2492a7c562db..d2c3940330df 100644 --- a/core/src/types/operator/operator_futures.rs +++ b/core/src/types/operator/operator_futures.rs @@ -247,12 +247,16 @@ impl FutureRead { /// Future that generated by [`Operator::read_with`] or [`Operator::reader_with`]. /// /// Users can add more options by public functions provided by this struct. -pub type FutureReader = OperatorFuture; +/// +/// # Notes +/// +/// `(OpRead, ())` is a trick to make sure `FutureReader` is different from `FutureRead` +pub type FutureReader = OperatorFuture<(OpRead, ()), F>; impl FutureReader { /// Set the version for this operation. pub fn version(self, v: &str) -> Self { - self.map(|args| args.with_version(v)) + self.map(|(args, ())| (args.with_version(v), ())) } } diff --git a/core/src/types/reader.rs b/core/src/types/reader.rs index 62347e0b5c14..e9942f2105bc 100644 --- a/core/src/types/reader.rs +++ b/core/src/types/reader.rs @@ -272,8 +272,6 @@ mod futures_io_adapter { _cx: &mut Context<'_>, pos: io::SeekFrom, ) -> Poll> { - - match &mut self.state { State::Idle(_) => match pos { SeekFrom::Start(n) => { @@ -420,7 +418,7 @@ mod tokio_io_adapter { mod stream_adapter { use super::*; - use crate::raw::{oio, BoxedStaticFuture}; + use crate::raw::*; use bytes::Bytes; use futures::Stream; use std::io; diff --git a/core/tests/behavior/async_read.rs b/core/tests/behavior/async_read.rs index aa8da5c3d611..f8b75885d2ab 100644 --- a/core/tests/behavior/async_read.rs +++ b/core/tests/behavior/async_read.rs @@ -35,12 +35,6 @@ pub fn tests(op: &Operator, tests: &mut Vec) { test_read_full, test_read_range, test_read_large_range, - test_reader_range, - test_reader_range_with_buffer, - test_reader_from, - test_reader_from_with_buffer, - test_reader_tail, - test_reader_tail_with_buffer, test_read_not_exist, test_read_with_if_match, test_read_with_if_none_match, @@ -58,9 +52,6 @@ pub fn tests(op: &Operator, tests: &mut Vec) { test_read_only_read_full, test_read_only_read_full_with_special_chars, test_read_only_read_with_range, - test_read_only_reader_with_range, - test_read_only_reader_from, - test_read_only_reader_tail, test_read_only_read_not_exist, test_read_only_read_with_dir_path, test_read_only_read_with_if_match, @@ -143,198 +134,6 @@ pub async fn test_read_large_range(op: Operator) -> anyhow::Result<()> { Ok(()) } -/// Read range content should match. -pub async fn test_reader_range(op: Operator) -> anyhow::Result<()> { - if !op.info().full_capability().read_with_range { - return Ok(()); - } - - let (path, content, size) = TEST_FIXTURE.new_file(op.clone()); - let (offset, length) = gen_offset_length(size); - - op.write(&path, content.clone()) - .await - .expect("write must succeed"); - - let r = op.reader_with(&path).range(offset..offset + length).await?; - - let mut bs = Vec::new(); - r.read_to_end(&mut bs).await?; - - assert_eq!( - format!("{:x}", Sha256::digest(&bs)), - format!( - "{:x}", - Sha256::digest(&content[offset as usize..(offset + length) as usize]) - ), - "read content" - ); - - Ok(()) -} - -/// Read range content should match. -pub async fn test_reader_range_with_buffer(op: Operator) -> anyhow::Result<()> { - if !op.info().full_capability().read_with_range { - return Ok(()); - } - - let (path, content, size) = TEST_FIXTURE.new_file(op.clone()); - let (offset, length) = gen_offset_length(size); - - op.write(&path, content.clone()) - .await - .expect("write must succeed"); - - let r = op - .reader_with(&path) - .range(offset..offset + length) - .buffer(4096) - .await?; - - let mut bs = Vec::new(); - r.read_to_end(&mut bs).await?; - - assert_eq!( - format!("{:x}", Sha256::digest(&bs)), - format!( - "{:x}", - Sha256::digest(&content[offset as usize..(offset + length) as usize]) - ), - "read content" - ); - - Ok(()) -} - -/// Read range from should match. -pub async fn test_reader_from(op: Operator) -> anyhow::Result<()> { - if !op.info().full_capability().read_with_range { - return Ok(()); - } - - let (path, content, size) = TEST_FIXTURE.new_file(op.clone()); - let (offset, _) = gen_offset_length(size); - - op.write(&path, content.clone()) - .await - .expect("write must succeed"); - - let r = op.reader_with(&path).range(offset..).await?; - - let mut bs = Vec::new(); - r.read_to_end(&mut bs).await?; - - assert_eq!(bs.len(), size - offset as usize, "read size"); - assert_eq!( - format!("{:x}", Sha256::digest(&bs)), - format!("{:x}", Sha256::digest(&content[offset as usize..])), - "read content" - ); - - Ok(()) -} - -/// Read range from should match. -pub async fn test_reader_from_with_buffer(op: Operator) -> anyhow::Result<()> { - if !op.info().full_capability().read_with_range { - return Ok(()); - } - - let (path, content, size) = TEST_FIXTURE.new_file(op.clone()); - let (offset, _) = gen_offset_length(size); - - op.write(&path, content.clone()) - .await - .expect("write must succeed"); - - let r = op.reader_with(&path).range(offset..).buffer(4096).await?; - - let mut bs = Vec::new(); - r.read_to_end(&mut bs).await?; - - assert_eq!(bs.len(), size - offset as usize, "read size"); - assert_eq!( - format!("{:x}", Sha256::digest(&bs)), - format!("{:x}", Sha256::digest(&content[offset as usize..])), - "read content" - ); - - Ok(()) -} - -/// Read range tail should match. -pub async fn test_reader_tail(op: Operator) -> anyhow::Result<()> { - if !op.info().full_capability().read_with_range { - return Ok(()); - } - - let (path, content, size) = TEST_FIXTURE.new_file(op.clone()); - let (_, length) = gen_offset_length(size); - - op.write(&path, content.clone()) - .await - .expect("write must succeed"); - - let r = match op.reader_with(&path).range(..length).await { - Ok(r) => r, - // Not all services support range with tail range, let's tolerate this. - Err(err) if err.kind() == ErrorKind::Unsupported => { - warn!("service doesn't support range with tail"); - return Ok(()); - } - Err(err) => return Err(err.into()), - }; - - let mut bs = Vec::new(); - r.read_to_end(&mut bs).await?; - - assert_eq!(bs.len(), length as usize, "read size"); - assert_eq!( - format!("{:x}", Sha256::digest(&bs)), - format!("{:x}", Sha256::digest(&content[size - length as usize..])), - "read content" - ); - - Ok(()) -} - -/// Read range tail should match. -pub async fn test_reader_tail_with_buffer(op: Operator) -> anyhow::Result<()> { - if !op.info().full_capability().read_with_range { - return Ok(()); - } - - let (path, content, size) = TEST_FIXTURE.new_file(op.clone()); - let (_, length) = gen_offset_length(size); - - op.write(&path, content.clone()) - .await - .expect("write must succeed"); - - let r = match op.reader_with(&path).range(..length).buffer(4096).await { - Ok(r) => r, - // Not all services support range with tail range, let's tolerate this. - Err(err) if err.kind() == ErrorKind::Unsupported => { - warn!("service doesn't support range with tail"); - return Ok(()); - } - Err(err) => return Err(err.into()), - }; - - let mut bs = Vec::new(); - r.read_to_end(&mut bs).await?; - - assert_eq!(bs.len(), length as usize, "read size"); - assert_eq!( - format!("{:x}", Sha256::digest(&bs)), - format!("{:x}", Sha256::digest(&content[size - length as usize..])), - "read content" - ); - - Ok(()) -} - /// Read not exist file should return NotFound pub async fn test_read_not_exist(op: Operator) -> anyhow::Result<()> { let path = uuid::Uuid::new_v4().to_string(); @@ -634,57 +433,6 @@ pub async fn test_read_only_read_with_range(op: Operator) -> anyhow::Result<()> Ok(()) } -/// Read range should match. -pub async fn test_read_only_reader_with_range(op: Operator) -> anyhow::Result<()> { - let r = op.reader_with("normal_file.txt").range(1024..2048).await?; - - let mut bs = Vec::new(); - r.read_to_end(&mut bs).await?; - - assert_eq!(bs.len(), 1024, "read size"); - assert_eq!( - format!("{:x}", Sha256::digest(&bs)), - "330c6d57fdc1119d6021b37714ca5ad0ede12edd484f66be799a5cff59667034", - "read content" - ); - - Ok(()) -} - -/// Read from should match. -pub async fn test_read_only_reader_from(op: Operator) -> anyhow::Result<()> { - let r = op.reader_with("normal_file.txt").range(29458..).await?; - - let mut bs = Vec::new(); - r.read_to_end(&mut bs).await?; - - assert_eq!(bs.len(), 1024, "read size"); - assert_eq!( - format!("{:x}", Sha256::digest(&bs)), - "cc9312c869238ea9410b6716e0fc3f48056f2bfb2fe06ccf5f96f2c3bf39e71b", - "read content" - ); - - Ok(()) -} - -/// Read tail should match. -pub async fn test_read_only_reader_tail(op: Operator) -> anyhow::Result<()> { - let r = op.reader_with("normal_file.txt").range(..1024).await?; - - let mut bs = Vec::new(); - r.read_to_end(&mut bs).await?; - - assert_eq!(bs.len(), 1024, "read size"); - assert_eq!( - format!("{:x}", Sha256::digest(&bs)), - "cc9312c869238ea9410b6716e0fc3f48056f2bfb2fe06ccf5f96f2c3bf39e71b", - "read content" - ); - - Ok(()) -} - /// Read not exist file should return NotFound pub async fn test_read_only_read_not_exist(op: Operator) -> anyhow::Result<()> { let path = uuid::Uuid::new_v4().to_string(); From 197ac27e48c4aa8ee6aa1403e09a4fd80957a8dd Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Thu, 21 Mar 2024 16:13:39 +0800 Subject: [PATCH 058/111] Save work Signed-off-by: Xuanwo --- core/src/layers/metrics.rs | 25 +++++++++++++++++++------ core/src/layers/retry.rs | 4 +--- core/src/layers/timeout.rs | 5 +---- core/src/raw/oio/buf/buffer.rs | 12 ++++++++++-- core/src/raw/oio/read/api.rs | 22 ++++++++++++---------- core/src/raw/ops.rs | 17 ----------------- core/src/services/alluxio/error.rs | 1 - core/src/services/alluxio/reader.rs | 4 ++-- core/src/services/azdls/reader.rs | 4 ++-- core/src/services/azfile/reader.rs | 4 ++-- core/src/services/b2/error.rs | 1 - core/src/services/chainsafe/error.rs | 1 - core/src/services/chainsafe/reader.rs | 4 ++-- core/src/services/ftp/backend.rs | 1 - core/src/services/ftp/reader.rs | 4 ++-- core/src/services/gdrive/reader.rs | 4 ++-- core/src/services/ghac/reader.rs | 4 ++-- core/src/services/github/error.rs | 1 - core/src/services/github/reader.rs | 4 ++-- core/src/services/ipfs/reader.rs | 4 ++-- core/src/services/ipmfs/reader.rs | 4 ++-- core/src/services/koofr/error.rs | 1 - core/src/services/koofr/reader.rs | 4 ++-- core/src/services/onedrive/reader.rs | 4 ++-- core/src/services/pcloud/error.rs | 1 - core/src/services/pcloud/reader.rs | 4 ++-- core/src/services/seafile/error.rs | 1 - core/src/services/seafile/reader.rs | 4 ++-- core/src/services/supabase/reader.rs | 4 ++-- core/src/services/upyun/error.rs | 1 - core/src/services/upyun/reader.rs | 4 ++-- core/src/services/vercel_blob/error.rs | 1 - core/src/services/webhdfs/error.rs | 3 +-- core/src/services/webhdfs/reader.rs | 4 ++-- core/src/services/yandex_disk/error.rs | 1 - core/src/services/yandex_disk/reader.rs | 4 ++-- core/src/types/blocking_reader.rs | 22 +++++++++------------- core/src/types/reader.rs | 15 ++++++++++----- 38 files changed, 99 insertions(+), 109 deletions(-) diff --git a/core/src/layers/metrics.rs b/core/src/layers/metrics.rs index 28890a5f9e67..e72a3aaae55b 100644 --- a/core/src/layers/metrics.rs +++ b/core/src/layers/metrics.rs @@ -754,9 +754,13 @@ impl MetricWrapper { impl oio::Read for MetricWrapper { async fn read_at(&self, offset: u64, limit: usize) -> Result { + let start = Instant::now(); + match self.inner.read_at(offset, limit).await { Ok(bs) => { self.bytes_counter.increment(bs.remaining() as u64); + self.requests_duration_seconds + .record(start.elapsed().as_secs_f64()); Ok(bs) } Err(e) => { @@ -769,10 +773,14 @@ impl oio::Read for MetricWrapper { impl oio::BlockingRead for MetricWrapper { fn read_at(&self, offset: u64, limit: usize) -> Result { + let start = Instant::now(); + self.inner .read_at(offset, limit) .map(|bs| { self.bytes_counter.increment(bs.remaining() as u64); + self.requests_duration_seconds + .record(start.elapsed().as_secs_f64()); bs }) .map_err(|e| { @@ -783,11 +791,16 @@ impl oio::BlockingRead for MetricWrapper { } impl oio::Write for MetricWrapper { - fn write(&mut self, bs: Bytes) -> impl Future> + Send { + async fn write(&mut self, bs: Bytes) -> Result { + let start = Instant::now(); + self.inner .write(bs) - .map_ok(|n| { + .await + .map(|n| { self.bytes_counter.increment(n as u64); + self.requests_duration_seconds + .record(start.elapsed().as_secs_f64()); n }) .map_err(|err| { @@ -796,15 +809,15 @@ impl oio::Write for MetricWrapper { }) } - fn abort(&mut self) -> impl Future> + Send { - self.inner.abort().map_err(|err| { + async fn abort(&mut self) -> Result<()> { + self.inner.abort().await.map_err(|err| { self.handle.increment_errors_total(self.op, err.kind()); err }) } - fn close(&mut self) -> impl Future> + Send { - self.inner.close().map_err(|err| { + async fn close(&mut self) -> Result<()> { + self.inner.close().await.map_err(|err| { self.handle.increment_errors_total(self.op, err.kind()); err }) diff --git a/core/src/layers/retry.rs b/core/src/layers/retry.rs index c9ea25b51c45..a9736199866b 100644 --- a/core/src/layers/retry.rs +++ b/core/src/layers/retry.rs @@ -17,7 +17,6 @@ use std::fmt::Debug; use std::fmt::Formatter; -use std::future::Future; use std::sync::Arc; @@ -903,7 +902,6 @@ impl oio::BlockingList for RetryWrapp #[cfg(test)] mod tests { use std::collections::HashMap; - use std::io; use std::sync::Arc; use std::sync::Mutex; @@ -1112,7 +1110,7 @@ mod tests { .layer(RetryLayer::new()) .finish(); - let mut r = op.reader("retryable_error").await.unwrap(); + let r = op.reader("retryable_error").await.unwrap(); let mut content = Vec::new(); let size = r .read_to_end(&mut content) diff --git a/core/src/layers/timeout.rs b/core/src/layers/timeout.rs index d917095c6ca8..d6b99e7fa707 100644 --- a/core/src/layers/timeout.rs +++ b/core/src/layers/timeout.rs @@ -17,7 +17,6 @@ use std::future::Future; - use std::time::Duration; use async_trait::async_trait; @@ -327,13 +326,11 @@ impl oio::List for TimeoutWrapper { #[cfg(test)] mod tests { use std::future::{pending, Future}; - use std::io::SeekFrom; use std::sync::Arc; use std::time::Duration; use async_trait::async_trait; - use bytes::Bytes; use futures::StreamExt; use tokio::time::sleep; use tokio::time::timeout; @@ -427,7 +424,7 @@ mod tests { let op = Operator::from_inner(acc) .layer(TimeoutLayer::new().with_io_timeout(Duration::from_secs(1))); - let mut reader = op.reader("test").await.unwrap(); + let reader = op.reader("test").await.unwrap(); let res = reader.read(&mut Vec::default(), 0, 4).await; assert!(res.is_err()); diff --git a/core/src/raw/oio/buf/buffer.rs b/core/src/raw/oio/buf/buffer.rs index 59dc2c8c92b9..ca754fa91c54 100644 --- a/core/src/raw/oio/buf/buffer.rs +++ b/core/src/raw/oio/buf/buffer.rs @@ -15,10 +15,14 @@ // specific language governing permissions and limitations // under the License. - use bytes::{Buf, BufMut, Bytes, BytesMut}; use std::collections::VecDeque; +/// Buffer is a wrapper of `Bytes` and `VecDeque`. +/// +/// We designed buffer to allow underlying storage to return non-contiguous bytes. +/// +/// For example, http based storage like s3 could generate non-contiguous bytes by stream. #[derive(Clone)] pub struct Buffer(Inner); @@ -29,11 +33,15 @@ enum Inner { } impl Buffer { + /// Create a new empty buffer. + /// + /// This operation is const and no allocation will be performed. #[inline] pub const fn new() -> Self { Self(Inner::NonContiguous(VecDeque::new())) } + /// Clone internal bytes to a new `Bytes`. #[inline] pub fn to_bytes(&self) -> Bytes { let mut bs = self.clone(); @@ -88,7 +96,7 @@ impl From> for Buffer { } } -impl bytes::Buf for Buffer { +impl Buf for Buffer { #[inline] fn remaining(&self) -> usize { match &self.0 { diff --git a/core/src/raw/oio/read/api.rs b/core/src/raw/oio/read/api.rs index df31650a0b7e..dc02b34944ff 100644 --- a/core/src/raw/oio/read/api.rs +++ b/core/src/raw/oio/read/api.rs @@ -86,6 +86,12 @@ pub type Reader = Box; /// an additional layer of indirection and an extra allocation. Ideally, `ReadDyn` should occur only /// once, at the outermost level of our API. pub trait Read: Unpin + Send + Sync { + /// Read at the given offset with the given limit. + /// + /// # Notes + /// + /// Storage services should try to read as much as possible, only return bytes less than the + /// limit while reaching the end of the file. #[cfg(not(target_arch = "wasm32"))] fn read_at( &self, @@ -143,17 +149,13 @@ impl Read for Box { pub type BlockingReader = Box; /// Read is the trait that OpenDAL returns to callers. -/// -/// Read is compose of the following trait -/// -/// - `Read` -/// - `Seek` -/// - `Iterator>` -/// -/// `Read` is required to be implemented, `Seek` and `Iterator` -/// is optional. We use `Read` to make users life easier. pub trait BlockingRead: Send + Sync { - /// Read synchronously. + /// Read data from the reader at the given offset with the given limit. + /// + /// # Notes + /// + /// Storage services should try to read as much as possible, only return bytes less than the + /// limit while reaching the end of the file. fn read_at(&self, offset: u64, limit: usize) -> Result; } diff --git a/core/src/raw/ops.rs b/core/src/raw/ops.rs index a87e2ce2d4af..32788aa4db55 100644 --- a/core/src/raw/ops.rs +++ b/core/src/raw/ops.rs @@ -316,23 +316,6 @@ impl OpRead { Self::default() } - /// The into_deterministic function transforms the OpRead into a deterministic version. - /// - /// This API is utilized because it allows for internal optimizations such as dividing read - /// ranges or retrying the read request from where it failed. In these scenarios, the expected - /// `ETag` value differs from what users specify in `If-Match` or `If-None-Match`.Therefore, - /// we need to eliminate these conditional headers to ensure that the read operation is - /// deterministic. - /// - /// This API is not intended to be used by users and should never be exposed. - pub(crate) fn into_deterministic(self) -> Self { - Self { - if_match: None, - if_none_match: None, - ..self - } - } - /// Create a new OpRead with range. pub fn with_range(mut self, range: BytesRange) -> Self { self.range = range; diff --git a/core/src/services/alluxio/error.rs b/core/src/services/alluxio/error.rs index bce4de9a7624..2b5c6b260fac 100644 --- a/core/src/services/alluxio/error.rs +++ b/core/src/services/alluxio/error.rs @@ -62,7 +62,6 @@ pub async fn parse_error(resp: Response) -> Result { #[cfg(test)] mod tests { - use futures::stream; use http::StatusCode; use super::*; diff --git a/core/src/services/alluxio/reader.rs b/core/src/services/alluxio/reader.rs index 804227800a27..7462cd234672 100644 --- a/core/src/services/alluxio/reader.rs +++ b/core/src/services/alluxio/reader.rs @@ -26,7 +26,7 @@ pub struct AlluxioReader { core: Arc, stream_id: u64, - op: OpRead, + _op: OpRead, } impl AlluxioReader { @@ -34,7 +34,7 @@ impl AlluxioReader { AlluxioReader { core, stream_id, - op, + _op: op, } } } diff --git a/core/src/services/azdls/reader.rs b/core/src/services/azdls/reader.rs index 1c726e2c37d2..3e3fc88fcaaf 100644 --- a/core/src/services/azdls/reader.rs +++ b/core/src/services/azdls/reader.rs @@ -26,7 +26,7 @@ pub struct AzdlsReader { core: Arc, path: String, - op: OpRead, + _op: OpRead, } impl AzdlsReader { @@ -34,7 +34,7 @@ impl AzdlsReader { AzdlsReader { core, path: path.to_string(), - op: op, + _op: op, } } } diff --git a/core/src/services/azfile/reader.rs b/core/src/services/azfile/reader.rs index 93292ebe98df..c92023bff734 100644 --- a/core/src/services/azfile/reader.rs +++ b/core/src/services/azfile/reader.rs @@ -26,7 +26,7 @@ pub struct AzfileReader { core: Arc, path: String, - op: OpRead, + _op: OpRead, } impl AzfileReader { @@ -34,7 +34,7 @@ impl AzfileReader { Self { core, path: path.to_string(), - op, + _op: op, } } } diff --git a/core/src/services/b2/error.rs b/core/src/services/b2/error.rs index 66bfe44e0907..9b4705891967 100644 --- a/core/src/services/b2/error.rs +++ b/core/src/services/b2/error.rs @@ -79,7 +79,6 @@ pub fn parse_b2_error_code(code: &str) -> Option<(ErrorKind, bool)> { #[cfg(test)] mod test { - use futures::stream; use http::StatusCode; use super::*; diff --git a/core/src/services/chainsafe/error.rs b/core/src/services/chainsafe/error.rs index dbc1ae61b44e..b7e3cf2d889f 100644 --- a/core/src/services/chainsafe/error.rs +++ b/core/src/services/chainsafe/error.rs @@ -75,7 +75,6 @@ pub async fn parse_error(resp: Response) -> Result { #[cfg(test)] mod test { - use futures::stream; use http::StatusCode; use super::*; diff --git a/core/src/services/chainsafe/reader.rs b/core/src/services/chainsafe/reader.rs index c4b363ae2309..79a6f9f2995c 100644 --- a/core/src/services/chainsafe/reader.rs +++ b/core/src/services/chainsafe/reader.rs @@ -26,7 +26,7 @@ pub struct ChainsafeReader { core: Arc, path: String, - op: OpRead, + _op: OpRead, } impl ChainsafeReader { @@ -34,7 +34,7 @@ impl ChainsafeReader { ChainsafeReader { core, path: path.to_string(), - op, + _op: op, } } } diff --git a/core/src/services/ftp/backend.rs b/core/src/services/ftp/backend.rs index 4157482d1df0..5a0a6e6ea30e 100644 --- a/core/src/services/ftp/backend.rs +++ b/core/src/services/ftp/backend.rs @@ -26,7 +26,6 @@ use async_trait::async_trait; use bb8::PooledConnection; use bb8::RunError; -use futures::AsyncReadExt; use http::Uri; use log::debug; use serde::Deserialize; diff --git a/core/src/services/ftp/reader.rs b/core/src/services/ftp/reader.rs index 25eb90ffe169..983e2d55876f 100644 --- a/core/src/services/ftp/reader.rs +++ b/core/src/services/ftp/reader.rs @@ -25,7 +25,7 @@ pub struct FtpReader { core: FtpBackend, path: String, - op: OpRead, + _op: OpRead, } impl FtpReader { @@ -33,7 +33,7 @@ impl FtpReader { FtpReader { core, path: path.to_string(), - op: op, + _op: op, } } } diff --git a/core/src/services/gdrive/reader.rs b/core/src/services/gdrive/reader.rs index cccc515003a7..dba874cd743a 100644 --- a/core/src/services/gdrive/reader.rs +++ b/core/src/services/gdrive/reader.rs @@ -26,7 +26,7 @@ pub struct GdriveReader { core: Arc, path: String, - op: OpRead, + _op: OpRead, } impl GdriveReader { @@ -34,7 +34,7 @@ impl GdriveReader { GdriveReader { core, path: path.to_string(), - op: op, + _op: op, } } } diff --git a/core/src/services/ghac/reader.rs b/core/src/services/ghac/reader.rs index 01b1682bbf74..7cfcc54f5750 100644 --- a/core/src/services/ghac/reader.rs +++ b/core/src/services/ghac/reader.rs @@ -24,7 +24,7 @@ pub struct GhacReader { core: GhacBackend, location: String, - op: OpRead, + _op: OpRead, } impl GhacReader { @@ -32,7 +32,7 @@ impl GhacReader { GhacReader { core, location: location.to_string(), - op: op, + _op: op, } } } diff --git a/core/src/services/github/error.rs b/core/src/services/github/error.rs index df295c7d5ef8..b438a82522c1 100644 --- a/core/src/services/github/error.rs +++ b/core/src/services/github/error.rs @@ -75,7 +75,6 @@ pub async fn parse_error(resp: Response) -> Result { #[cfg(test)] mod test { - use futures::stream; use http::StatusCode; use super::*; diff --git a/core/src/services/github/reader.rs b/core/src/services/github/reader.rs index 580599c53659..055c296972ea 100644 --- a/core/src/services/github/reader.rs +++ b/core/src/services/github/reader.rs @@ -26,7 +26,7 @@ pub struct GithubReader { core: Arc, path: String, - op: OpRead, + _op: OpRead, } impl GithubReader { @@ -34,7 +34,7 @@ impl GithubReader { GithubReader { core, path: path.to_string(), - op: op, + _op: op, } } } diff --git a/core/src/services/ipfs/reader.rs b/core/src/services/ipfs/reader.rs index fd26e0913885..1af91f145718 100644 --- a/core/src/services/ipfs/reader.rs +++ b/core/src/services/ipfs/reader.rs @@ -24,7 +24,7 @@ pub struct IpfsReader { core: IpfsBackend, path: String, - op: OpRead, + _op: OpRead, } impl IpfsReader { @@ -32,7 +32,7 @@ impl IpfsReader { IpfsReader { core, path: path.to_string(), - op: op, + _op: op, } } } diff --git a/core/src/services/ipmfs/reader.rs b/core/src/services/ipmfs/reader.rs index 6727b8273900..5e09d4be67cb 100644 --- a/core/src/services/ipmfs/reader.rs +++ b/core/src/services/ipmfs/reader.rs @@ -24,7 +24,7 @@ pub struct IpmfsReader { core: IpmfsBackend, path: String, - op: OpRead, + _op: OpRead, } impl IpmfsReader { @@ -32,7 +32,7 @@ impl IpmfsReader { IpmfsReader { core, path: path.to_string(), - op: op, + _op: op, } } } diff --git a/core/src/services/koofr/error.rs b/core/src/services/koofr/error.rs index 22543e2518e8..d3ee3ff3ae8c 100644 --- a/core/src/services/koofr/error.rs +++ b/core/src/services/koofr/error.rs @@ -54,7 +54,6 @@ pub async fn parse_error(resp: Response) -> Result { #[cfg(test)] mod test { - use futures::stream; use http::StatusCode; use super::*; diff --git a/core/src/services/koofr/reader.rs b/core/src/services/koofr/reader.rs index 78a1b90b9c6b..532fe2d9e03b 100644 --- a/core/src/services/koofr/reader.rs +++ b/core/src/services/koofr/reader.rs @@ -26,7 +26,7 @@ pub struct KoofrReader { core: Arc, path: String, - op: OpRead, + _op: OpRead, } impl KoofrReader { @@ -34,7 +34,7 @@ impl KoofrReader { KoofrReader { core, path: path.to_string(), - op: op, + _op: op, } } } diff --git a/core/src/services/onedrive/reader.rs b/core/src/services/onedrive/reader.rs index a9b4f2cf97c8..857919dc732d 100644 --- a/core/src/services/onedrive/reader.rs +++ b/core/src/services/onedrive/reader.rs @@ -24,7 +24,7 @@ pub struct OnedriveReader { core: OnedriveBackend, path: String, - op: OpRead, + _op: OpRead, } impl OnedriveReader { @@ -32,7 +32,7 @@ impl OnedriveReader { OnedriveReader { core, path: path.to_string(), - op: op, + _op: op, } } } diff --git a/core/src/services/pcloud/error.rs b/core/src/services/pcloud/error.rs index 03a1bf919cf9..59f0bd47ad80 100644 --- a/core/src/services/pcloud/error.rs +++ b/core/src/services/pcloud/error.rs @@ -58,7 +58,6 @@ pub async fn parse_error(resp: Response) -> Result { #[cfg(test)] mod test { - use futures::stream; use http::StatusCode; use super::*; diff --git a/core/src/services/pcloud/reader.rs b/core/src/services/pcloud/reader.rs index 1dad52ff0a71..93d2c864b847 100644 --- a/core/src/services/pcloud/reader.rs +++ b/core/src/services/pcloud/reader.rs @@ -26,7 +26,7 @@ pub struct PcloudReader { core: Arc, link: String, - op: OpRead, + _op: OpRead, } impl PcloudReader { @@ -34,7 +34,7 @@ impl PcloudReader { PcloudReader { core, link: link.to_string(), - op: op, + _op: op, } } } diff --git a/core/src/services/seafile/error.rs b/core/src/services/seafile/error.rs index f7982e841cee..1a9a4d5537cb 100644 --- a/core/src/services/seafile/error.rs +++ b/core/src/services/seafile/error.rs @@ -57,7 +57,6 @@ pub async fn parse_error(resp: Response) -> Result { #[cfg(test)] mod test { - use futures::stream; use http::StatusCode; use super::*; diff --git a/core/src/services/seafile/reader.rs b/core/src/services/seafile/reader.rs index 1546b6a04db3..ba9855969cd7 100644 --- a/core/src/services/seafile/reader.rs +++ b/core/src/services/seafile/reader.rs @@ -26,7 +26,7 @@ pub struct SeafileReader { core: Arc, path: String, - op: OpRead, + _op: OpRead, } impl SeafileReader { @@ -34,7 +34,7 @@ impl SeafileReader { SeafileReader { core, path: path.to_string(), - op: op, + _op: op, } } } diff --git a/core/src/services/supabase/reader.rs b/core/src/services/supabase/reader.rs index 9293ff0e9961..2ea7896acaed 100644 --- a/core/src/services/supabase/reader.rs +++ b/core/src/services/supabase/reader.rs @@ -26,7 +26,7 @@ pub struct SupabaseReader { core: Arc, path: String, - op: OpRead, + _op: OpRead, } impl SupabaseReader { @@ -34,7 +34,7 @@ impl SupabaseReader { SupabaseReader { core, path: path.to_string(), - op: op, + _op: op, } } } diff --git a/core/src/services/upyun/error.rs b/core/src/services/upyun/error.rs index 30678db4e2f7..d3eec02d7b9b 100644 --- a/core/src/services/upyun/error.rs +++ b/core/src/services/upyun/error.rs @@ -67,7 +67,6 @@ pub async fn parse_error(resp: Response) -> Result { #[cfg(test)] mod test { - use futures::stream; use http::StatusCode; use super::*; diff --git a/core/src/services/upyun/reader.rs b/core/src/services/upyun/reader.rs index 9846eb65cf0e..3a04e2c5b151 100644 --- a/core/src/services/upyun/reader.rs +++ b/core/src/services/upyun/reader.rs @@ -26,7 +26,7 @@ pub struct UpyunReader { core: Arc, path: String, - op: OpRead, + _op: OpRead, } impl UpyunReader { @@ -34,7 +34,7 @@ impl UpyunReader { UpyunReader { core, path: path.to_string(), - op: op, + _op: op, } } } diff --git a/core/src/services/vercel_blob/error.rs b/core/src/services/vercel_blob/error.rs index badc56714ef3..842b919c1629 100644 --- a/core/src/services/vercel_blob/error.rs +++ b/core/src/services/vercel_blob/error.rs @@ -68,7 +68,6 @@ pub async fn parse_error(resp: Response) -> Result { #[cfg(test)] mod test { - use futures::stream; use http::StatusCode; use super::*; diff --git a/core/src/services/webhdfs/error.rs b/core/src/services/webhdfs/error.rs index a2bbca026562..971997a7be8b 100644 --- a/core/src/services/webhdfs/error.rs +++ b/core/src/services/webhdfs/error.rs @@ -80,7 +80,6 @@ pub(super) fn parse_error_msg(parts: Parts, body: &str) -> Result { #[cfg(test)] mod tests { use bytes::Buf; - use futures::stream; use serde_json::from_reader; use super::*; @@ -96,7 +95,7 @@ mod tests { "exception" : "IllegalArgumentException", "javaClassName": "java.lang.IllegalArgumentException", "message" : "Invalid value for webhdfs parameter \"permission\": ..." - } + } } "#, ); diff --git a/core/src/services/webhdfs/reader.rs b/core/src/services/webhdfs/reader.rs index 19474b188e35..20f120142945 100644 --- a/core/src/services/webhdfs/reader.rs +++ b/core/src/services/webhdfs/reader.rs @@ -25,7 +25,7 @@ pub struct WebhdfsReader { core: WebhdfsBackend, path: String, - op: OpRead, + _op: OpRead, } impl WebhdfsReader { @@ -33,7 +33,7 @@ impl WebhdfsReader { WebhdfsReader { core, path: path.to_string(), - op: op, + _op: op, } } } diff --git a/core/src/services/yandex_disk/error.rs b/core/src/services/yandex_disk/error.rs index 492e81131645..38e6e3c72f8b 100644 --- a/core/src/services/yandex_disk/error.rs +++ b/core/src/services/yandex_disk/error.rs @@ -67,7 +67,6 @@ pub async fn parse_error(resp: Response) -> Result { #[cfg(test)] mod test { - use futures::stream; use http::StatusCode; use super::*; diff --git a/core/src/services/yandex_disk/reader.rs b/core/src/services/yandex_disk/reader.rs index e3c6eddce394..4dafffcd0ecf 100644 --- a/core/src/services/yandex_disk/reader.rs +++ b/core/src/services/yandex_disk/reader.rs @@ -26,7 +26,7 @@ pub struct YandexDiskReader { core: Arc, path: String, - op: OpRead, + _op: OpRead, } impl YandexDiskReader { @@ -34,7 +34,7 @@ impl YandexDiskReader { YandexDiskReader { core, path: path.to_string(), - op: op, + _op: op, } } } diff --git a/core/src/types/blocking_reader.rs b/core/src/types/blocking_reader.rs index 33f4d823ebab..890483267b57 100644 --- a/core/src/types/blocking_reader.rs +++ b/core/src/types/blocking_reader.rs @@ -17,17 +17,10 @@ use std::collections::Bound; - -use std::ops::{RangeBounds}; - - - - +use std::ops::RangeBounds; use bytes::{Buf, BufMut}; - - use crate::raw::oio::BlockingRead; use crate::raw::*; use crate::*; @@ -52,11 +45,11 @@ impl BlockingReader { Ok(BlockingReader { inner: r }) } - /// Create a new reader from an `oio::BlockingReader`. - pub(crate) fn new(r: oio::BlockingReader) -> Self { - BlockingReader { inner: r } - } - + /// Read from underlying storage and write data into the specified buffer, starting at + /// the given offset and up to the limit. + /// + /// A return value of `n` signifies that `n` bytes of data have been read into `buf`. + /// If `n < limit`, it indicates that the reader has reached EOF (End of File). #[inline] pub fn read(&self, buf: &mut impl BufMut, offset: u64, limit: usize) -> Result { let bs = self.inner.read_at(offset, limit)?; @@ -114,6 +107,9 @@ impl BlockingReader { } } + /// Read all data from reader. + /// + /// This API is exactly the same with `BlockingReader::read_range(buf, ..)`. #[inline] pub fn read_to_end(&self, buf: &mut impl BufMut) -> Result { self.read_range(buf, ..) diff --git a/core/src/types/reader.rs b/core/src/types/reader.rs index e9942f2105bc..87bf653ec066 100644 --- a/core/src/types/reader.rs +++ b/core/src/types/reader.rs @@ -24,10 +24,8 @@ use std::task::Context; use std::task::Poll; use bytes::{Buf, BufMut}; -use futures::Stream; use tokio::io::ReadBuf; -use crate::raw::oio::BlockingRead; use crate::raw::*; use crate::*; @@ -122,6 +120,11 @@ impl Reader { tokio_io_adapter::TokioReader::new(self.acc, self.path, self.op) } + /// Read from underlying storage and write data into the specified buffer, starting at + /// the given offset and up to the limit. + /// + /// A return value of `n` signifies that `n` bytes of data have been read into `buf`. + /// If `n < limit`, it indicates that the reader has reached EOF (End of File). #[inline] pub async fn read(&self, buf: &mut impl BufMut, offset: u64, limit: usize) -> Result { let bs = self.inner.read_at_dyn(offset, limit).await?; @@ -184,6 +187,9 @@ impl Reader { } } + /// Read all data from reader. + /// + /// This API is exactly the same with `Reader::read_range(buf, ..)`. #[inline] pub async fn read_to_end(&self, buf: &mut impl BufMut) -> Result { self.read_range(buf, ..).await @@ -418,7 +424,6 @@ mod tokio_io_adapter { mod stream_adapter { use super::*; - use crate::raw::*; use bytes::Bytes; use futures::Stream; use std::io; @@ -504,7 +509,7 @@ mod tests { .await .expect("write must succeed"); - let mut reader = op.reader(path).await.unwrap(); + let reader = op.reader(path).await.unwrap(); let mut buf = Vec::new(); reader .read_to_end(&mut buf) @@ -524,7 +529,7 @@ mod tests { .await .expect("write must succeed"); - let mut reader = op.reader(path).await.unwrap(); + let reader = op.reader(path).await.unwrap(); let mut buf = Vec::new(); reader .read_to_end(&mut buf) From d9707612c45b9395af2628b716f0ff6b43bcfe47 Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Thu, 21 Mar 2024 16:26:02 +0800 Subject: [PATCH 059/111] Save work Signed-off-by: Xuanwo --- core/benches/ops/read.rs | 1 + core/src/types/reader.rs | 340 +-------------------------------------- 2 files changed, 4 insertions(+), 337 deletions(-) diff --git a/core/benches/ops/read.rs b/core/benches/ops/read.rs index 795c05f1fb75..6f6f9eda5796 100644 --- a/core/benches/ops/read.rs +++ b/core/benches/ops/read.rs @@ -51,6 +51,7 @@ fn bench_read_full(c: &mut Criterion, name: &str, op: Operator) { group.bench_with_input(size.to_string(), &(op.clone(), &path), |b, (op, path)| { b.to_async(&*TEST_RUNTIME).iter(|| async { let r = op.reader_with(path).await.unwrap(); + let r = r.into_futures_read(); io::copy(r, &mut io::sink()).await.unwrap(); }) diff --git a/core/src/types/reader.rs b/core/src/types/reader.rs index 87bf653ec066..21b043f7ab5a 100644 --- a/core/src/types/reader.rs +++ b/core/src/types/reader.rs @@ -39,30 +39,11 @@ use crate::*; /// /// ## Direct /// -/// [`Reader`] provides public API including [`Reader::read`], [`Reader::seek`] and -/// [`Reader::read_to_end`]. You can use those APIs directly without extra copy. +/// [`Reader`] provides public API including [`Reader::read`], [`Reader:read_range`], and [`Reader::read_to_end`]. You can use those APIs directly without extra copy. /// -/// ## Bytes Stream +/// # TODO /// -/// [`Reader`] can be used as `Stream>>`. -/// -/// It also implements [`Send`], [`Sync`] and [`Unpin`]. -/// -/// ## Futures AsyncRead -/// -/// [`Reader`] can be used as [`futures::AsyncRead`] and [`futures::AsyncSeek`]. -/// -/// It also implements [`Send`], [`Sync`] and [`Unpin`]. -/// -/// [`Reader`] provides [`Reader::into_futures_read`] to remove extra APIs upon self. -/// -/// ## Tokio AsyncRead -/// -/// [`Reader`] can be used as [`tokio::io::AsyncRead`] and [`tokio::io::AsyncSeek`]. -/// -/// It also implements [`Send`], [`Sync`] and [`Unpin`]. -/// -/// [`Reader`] provides [`Reader::into_tokio_read`] to remove extra APIs upon self. +/// Implement `into_async_read` and `into_stream`. pub struct Reader { acc: FusedAccessor, path: String, @@ -90,36 +71,6 @@ impl Reader { }) } - /// Convert [`Reader`] into an [`futures::AsyncRead`] and [`futures::AsyncSeek`] - /// - /// `Reader` itself implements [`futures::AsyncRead`], this function is used to - /// make sure that `Reader` is used as an `AsyncRead` only. - /// - /// The returning type also implements `Send`, `Sync` and `Unpin`, so users can use it - /// as `Box` and calling `poll_read_unpin` on it. - #[inline] - #[cfg(not(target_arch = "wasm32"))] - pub fn into_futures_read( - self, - ) -> impl futures::AsyncRead + futures::AsyncSeek + Send + Sync + Unpin { - futures_io_adapter::FuturesReader::new(self.acc, self.path, self.op) - } - - /// Convert [`Reader`] into an [`tokio::io::AsyncRead`] and [`tokio::io::AsyncSeek`] - /// - /// `Reader` itself implements [`tokio::io::AsyncRead`], this function is used to - /// make sure that `Reader` is used as an [`tokio::io::AsyncRead`] only. - /// - /// The returning type also implements `Send`, `Sync` and `Unpin`, so users can use it - /// as `Box` and calling `poll_read_unpin` on it. - #[inline] - #[cfg(not(target_arch = "wasm32"))] - pub fn into_tokio_read( - self, - ) -> impl tokio::io::AsyncRead + tokio::io::AsyncSeek + Send + Sync + Unpin { - tokio_io_adapter::TokioReader::new(self.acc, self.path, self.op) - } - /// Read from underlying storage and write data into the specified buffer, starting at /// the given offset and up to the limit. /// @@ -196,291 +147,6 @@ impl Reader { } } -mod futures_io_adapter { - use super::*; - use futures::io::{AsyncRead, AsyncSeek}; - use std::pin::Pin; - use std::task::{Context, Poll}; - - /// TODO: we can implement async buf read. - pub struct FuturesReader { - pub acc: FusedAccessor, - pub path: String, - pub op: OpRead, - - pub state: State, - pub offset: u64, - } - - enum State { - Idle(Option), - Stating(BoxedStaticFuture>), - Reading(BoxedStaticFuture<(oio::Reader, Result)>), - } - - /// # Safety - /// - /// Reader will only be used with `&mut self`. - unsafe impl Sync for State {} - - impl FuturesReader { - pub fn new(acc: FusedAccessor, path: String, op: OpRead) -> Self { - Self { - acc, - path, - op, - state: State::Idle(None), - offset: 0, - } - } - } - - impl AsyncRead for FuturesReader { - fn poll_read( - mut self: Pin<&mut Self>, - cx: &mut Context<'_>, - mut buf: &mut [u8], - ) -> Poll> { - use oio::Read; - - match &mut self.state { - State::Idle(r) => { - let r = r.take().expect("reader must be valid"); - let size = buf.len(); - let offset = self.offset; - let fut = async move { - let res = r.read_at(offset, size).await; - (r, res) - }; - self.state = State::Reading(Box::pin(fut)); - self.poll_read(cx, buf) - } - State::Reading(fut) => { - let (r, res) = ready!(fut.as_mut().poll(cx)); - self.state = State::Idle(Some(r)); - let bs = res.map_err(format_std_io_error)?; - let n = bs.remaining(); - buf.put(bs); - self.offset += n as u64; - Poll::Ready(Ok(n)) - } - State::Stating(_) => Poll::Ready(Err(io::Error::new( - io::ErrorKind::Interrupted, - "another io operation is in progress", - ))), - } - } - } - - impl AsyncSeek for FuturesReader { - fn poll_seek( - mut self: Pin<&mut Self>, - _cx: &mut Context<'_>, - pos: io::SeekFrom, - ) -> Poll> { - match &mut self.state { - State::Idle(_) => match pos { - SeekFrom::Start(n) => { - self.offset = n; - Poll::Ready(Ok(n)) - } - SeekFrom::End(_) => todo!(), - SeekFrom::Current(amt) => { - let offset = self.offset as i64 + amt; - if offset < 0 { - return Poll::Ready(Err(io::Error::new( - io::ErrorKind::InvalidInput, - "invalid seek to a negative position", - ))); - } - self.offset = offset as u64; - Poll::Ready(Ok(self.offset)) - } - }, - State::Stating(_fut) => { - todo!() - } - State::Reading(_) => Poll::Ready(Err(io::Error::new( - io::ErrorKind::Interrupted, - "another io operation is in progress", - ))), - } - } - } -} - -mod tokio_io_adapter { - use super::*; - use std::pin::Pin; - use std::task::{Context, Poll}; - use tokio::io::{AsyncRead, AsyncSeek}; - - /// TODO: we can implement async buf read. - pub struct TokioReader { - acc: FusedAccessor, - path: String, - op: OpRead, - - state: State, - offset: u64, - } - - enum State { - Idle(Option), - Stating(BoxedStaticFuture>), - Reading(BoxedStaticFuture<(oio::Reader, Result)>), - } - - /// # Safety - /// - /// Reader will only be used with `&mut self`. - unsafe impl Sync for State {} - - impl TokioReader { - pub fn new(acc: FusedAccessor, path: String, op: OpRead) -> Self { - Self { - acc, - path, - op, - state: State::Idle(None), - offset: 0, - } - } - } - - impl AsyncRead for TokioReader { - fn poll_read( - mut self: Pin<&mut Self>, - cx: &mut Context<'_>, - buf: &mut ReadBuf<'_>, - ) -> Poll> { - use oio::Read; - - match &mut self.state { - State::Idle(r) => { - let r = r.take().expect("reader must be valid"); - let size = buf.remaining_mut(); - let offset = self.offset; - let fut = async move { - let res = r.read_at(offset, size).await; - (r, res) - }; - self.state = State::Reading(Box::pin(fut)); - self.poll_read(cx, buf) - } - State::Reading(fut) => { - let (r, res) = ready!(fut.as_mut().poll(cx)); - self.state = State::Idle(Some(r)); - let bs = res.map_err(format_std_io_error)?; - let n = bs.remaining(); - buf.put(bs); - self.offset += n as u64; - Poll::Ready(Ok(())) - } - State::Stating(_) => Poll::Ready(Err(io::Error::new( - io::ErrorKind::Interrupted, - "another io operation is in progress", - ))), - } - } - } - - impl AsyncSeek for TokioReader { - fn start_seek(mut self: Pin<&mut Self>, pos: io::SeekFrom) -> io::Result<()> { - match &mut self.state { - State::Idle(_) => match pos { - SeekFrom::Start(n) => { - self.offset = n; - Ok(()) - } - SeekFrom::End(_) => todo!(), - SeekFrom::Current(amt) => { - let offset = self.offset as i64 + amt; - if offset < 0 { - return Err(io::Error::new( - io::ErrorKind::InvalidInput, - "invalid seek to a negative position", - )); - } - self.offset = offset as u64; - Ok(()) - } - }, - State::Stating(_fut) => { - todo!() - } - State::Reading(_) => Err(io::Error::new( - io::ErrorKind::Interrupted, - "another io operation is in progress", - )), - } - } - - fn poll_complete(self: Pin<&mut Self>, _cx: &mut Context<'_>) -> Poll> { - Poll::Ready(Ok(self.offset)) - } - } -} - -mod stream_adapter { - use super::*; - use bytes::Bytes; - use futures::Stream; - use std::io; - use std::pin::Pin; - - pub struct StreamReader { - state: State, - - buffer: oio::Buffer, - offset: u64, - } - - enum State { - Idle(Option), - Reading(BoxedStaticFuture<(oio::Reader, crate::Result)>), - } - - impl Stream for StreamReader { - type Item = io::Result; - - fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - if self.buffer.has_remaining() { - let n = self.buffer.chunk().len(); - let bs = self.buffer.copy_to_bytes(n); - return Poll::Ready(Some(Ok(bs))); - } - - let offset = self.offset; - - match &mut self.state { - State::Idle(r) => { - let r = r.take().expect("reader must be valid"); - let fut = async move { - // TODO: should allow user to tune this value. - let res = r.read_at_dyn(offset, 4 * 1024 * 1024).await; - (r, res) - }; - self.state = State::Reading(Box::pin(fut)); - self.poll_next(cx) - } - State::Reading(fut) => { - let (r, res) = ready!(fut.as_mut().poll(cx)); - self.state = State::Idle(Some(r)); - let bs = res.map_err(format_std_io_error)?; - if bs.has_remaining() { - self.offset += bs.remaining() as u64; - self.buffer = bs; - self.poll_next(cx) - } else { - Poll::Ready(None) - } - } - } - } - } -} - #[cfg(test)] mod tests { use rand::rngs::ThreadRng; From a09efaf014391b20db52f745019b0d1580eb195e Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Thu, 21 Mar 2024 19:55:29 +0800 Subject: [PATCH 060/111] Build pass Signed-off-by: Xuanwo --- core/benches/ops/read.rs | 2 +- core/src/layers/metrics.rs | 1 - core/src/raw/oio/read/api.rs | 2 +- core/src/types/reader.rs | 266 ++++++++++++++++++++++++++++++++--- 4 files changed, 249 insertions(+), 22 deletions(-) diff --git a/core/benches/ops/read.rs b/core/benches/ops/read.rs index 6f6f9eda5796..4b8f40a0cc10 100644 --- a/core/benches/ops/read.rs +++ b/core/benches/ops/read.rs @@ -52,7 +52,7 @@ fn bench_read_full(c: &mut Criterion, name: &str, op: Operator) { b.to_async(&*TEST_RUNTIME).iter(|| async { let r = op.reader_with(path).await.unwrap(); - let r = r.into_futures_read(); + let r = r.into_async_read(0..size.bytes() as u64); io::copy(r, &mut io::sink()).await.unwrap(); }) }); diff --git a/core/src/layers/metrics.rs b/core/src/layers/metrics.rs index e72a3aaae55b..12a0aa4f8c53 100644 --- a/core/src/layers/metrics.rs +++ b/core/src/layers/metrics.rs @@ -17,7 +17,6 @@ use std::fmt::Debug; use std::fmt::Formatter; -use std::future::Future; use std::sync::Arc; diff --git a/core/src/raw/oio/read/api.rs b/core/src/raw/oio/read/api.rs index dc02b34944ff..435148348980 100644 --- a/core/src/raw/oio/read/api.rs +++ b/core/src/raw/oio/read/api.rs @@ -19,7 +19,7 @@ use std::fmt::Display; use std::fmt::Formatter; use std::ops::Deref; -use bytes::{Buf, Bytes}; +use bytes::Bytes; use futures::Future; use crate::raw::*; diff --git a/core/src/types/reader.rs b/core/src/types/reader.rs index 21b043f7ab5a..30c40cc54028 100644 --- a/core/src/types/reader.rs +++ b/core/src/types/reader.rs @@ -15,16 +15,9 @@ // specific language governing permissions and limitations // under the License. -use std::io; -use std::io::SeekFrom; -use std::ops::{Bound, RangeBounds}; - -use std::task::ready; -use std::task::Context; -use std::task::Poll; +use std::ops::{Bound, Range, RangeBounds}; use bytes::{Buf, BufMut}; -use tokio::io::ReadBuf; use crate::raw::*; use crate::*; @@ -45,10 +38,6 @@ use crate::*; /// /// Implement `into_async_read` and `into_stream`. pub struct Reader { - acc: FusedAccessor, - path: String, - op: OpRead, - inner: oio::Reader, } @@ -61,14 +50,9 @@ impl Reader { /// We don't want to expose those details to users so keep this function /// in crate only. pub(crate) async fn create(acc: FusedAccessor, path: &str, op: OpRead) -> Result { - let (_, r) = acc.read(path, op.clone()).await?; + let (_, r) = acc.read(path, op).await?; - Ok(Reader { - acc, - path: path.to_string(), - op, - inner: r, - }) + Ok(Reader { inner: r }) } /// Read from underlying storage and write data into the specified buffer, starting at @@ -145,6 +129,250 @@ impl Reader { pub async fn read_to_end(&self, buf: &mut impl BufMut) -> Result { self.read_range(buf, ..).await } + + /// Convert reader into async read. + #[inline] + pub fn into_async_read( + self, + range: Range, + ) -> impl futures::AsyncRead + futures::AsyncSeek + Send + Sync + Unpin { + // TODO: the capacity should be decided by services. + impl_futures_async_read::FuturesReader::new(self.inner, range, 4 * 1024 * 1024) + } + + /// Convert reader into async buf read. + #[inline] + pub fn into_async_buf_read( + self, + range: Range, + capacity: usize, + ) -> impl futures::AsyncBufRead + futures::AsyncSeek + Send + Sync + Unpin { + impl_futures_async_read::FuturesReader::new(self.inner, range, capacity) + } + + /// Convert reader into stream. + #[inline] + pub fn into_stream(self, range: Range) -> impl futures::Stream + Send + Sync + Unpin { + // TODO: the capacity should be decided by services. + impl_futures_stream::FuturesStream::new(self.inner, range, 4 * 1024 * 1024) + } +} + +mod impl_futures_async_read { + use crate::raw::*; + use crate::*; + use bytes::Buf; + use futures::{AsyncBufRead, AsyncRead, AsyncSeek}; + use std::io; + use std::io::SeekFrom; + use std::ops::Range; + use std::pin::Pin; + use std::task::{ready, Context, Poll}; + + pub struct FuturesReader { + state: State, + offset: u64, + size: u64, + cap: usize, + + cur: u64, + buf: oio::Buffer, + } + + enum State { + Idle(Option), + Fill(BoxedStaticFuture<(oio::Reader, Result)>), + } + + /// # Safety + /// + /// FuturesReader only exposes `&mut self` to the outside world, so it's safe to be `Sync`. + unsafe impl Sync for State {} + + impl FuturesReader { + #[inline] + pub fn new(r: oio::Reader, range: Range, cap: usize) -> Self { + FuturesReader { + state: State::Idle(Some(r)), + offset: range.start, + size: range.end - range.start, + cap, + + cur: 0, + buf: oio::Buffer::new(), + } + } + } + + impl AsyncBufRead for FuturesReader { + fn poll_fill_buf(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + let this = self.get_mut(); + loop { + if this.buf.has_remaining() { + return Poll::Ready(Ok(this.buf.chunk())); + } + + match &mut this.state { + State::Idle(r) => { + // Make sure cur didn't exceed size. + if this.cur >= this.size { + return Poll::Ready(Ok(&[])); + } + + let r = r.take().expect("reader must be present"); + let next_offset = this.offset + this.cur; + let next_size = (this.size - this.cur).min(this.cap as u64) as usize; + let fut = async move { + let res = r.read_at_dyn(next_offset, next_size).await; + (r, res) + }; + this.state = State::Fill(Box::pin(fut)); + } + State::Fill(fut) => { + let (r, res) = ready!(fut.as_mut().poll(cx)); + this.state = State::Idle(Some(r)); + this.buf = res?; + } + } + } + } + + fn consume(mut self: Pin<&mut Self>, amt: usize) { + self.buf.advance(amt); + self.cur += amt as u64; + } + } + + impl AsyncRead for FuturesReader { + fn poll_read( + mut self: Pin<&mut Self>, + cx: &mut Context<'_>, + buf: &mut [u8], + ) -> Poll> { + let bs = ready!(self.as_mut().poll_fill_buf(cx))?; + let n = bs.len().min(buf.len()); + buf[..n].copy_from_slice(&bs[..n]); + self.as_mut().consume(n); + Poll::Ready(Ok(n)) + } + } + + impl AsyncSeek for FuturesReader { + fn poll_seek( + mut self: Pin<&mut Self>, + _: &mut Context<'_>, + pos: SeekFrom, + ) -> Poll> { + let new_pos = match pos { + SeekFrom::Start(pos) => pos as i64, + SeekFrom::End(pos) => self.size as i64 + pos, + SeekFrom::Current(pos) => self.cur as i64 + pos, + }; + + if new_pos < 0 { + return Poll::Ready(Err(io::Error::new( + io::ErrorKind::InvalidInput, + "invalid seek to a negative position", + ))); + } + + let new_pos = new_pos as u64; + + if (self.cur..self.cur + self.buf.remaining() as u64).contains(&new_pos) { + let cnt = new_pos - self.cur; + self.buf.advance(cnt as _); + } else { + self.buf = oio::Buffer::new() + } + + self.cur = new_pos; + Poll::Ready(Ok(self.cur)) + } + } +} + +mod impl_futures_stream { + use crate::raw::*; + use crate::*; + use bytes::{Buf, Bytes}; + use futures::Stream; + use std::io; + use std::ops::Range; + use std::pin::Pin; + use std::task::{ready, Context, Poll}; + + pub struct FuturesStream { + state: State, + offset: u64, + size: u64, + cap: usize, + + cur: u64, + } + + enum State { + Idle(Option), + Next(BoxedStaticFuture<(oio::Reader, Result)>), + } + + /// # Safety + /// + /// FuturesReader only exposes `&mut self` to the outside world, so it's safe to be `Sync`. + unsafe impl Sync for State {} + + impl FuturesStream { + #[inline] + pub fn new(r: oio::Reader, range: Range, cap: usize) -> Self { + FuturesStream { + state: State::Idle(Some(r)), + offset: range.start, + size: range.end - range.start, + cap, + + cur: 0, + } + } + } + + impl Stream for FuturesStream { + type Item = io::Result; + + fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + let this = self.get_mut(); + + loop { + match &mut this.state { + State::Idle(r) => { + // Make sure cur didn't exceed size. + if this.cur >= this.size { + return Poll::Ready(None); + } + + let r = r.take().expect("reader must be present"); + let next_offset = this.offset + this.cur; + let next_size = (this.size - this.cur).min(this.cap as u64) as usize; + let fut = async move { + let res = r.read_at_dyn(next_offset, next_size).await; + (r, res) + }; + this.state = State::Next(Box::pin(fut)); + } + State::Next(fut) => { + let (r, res) = ready!(fut.as_mut().poll(cx)); + this.state = State::Idle(Some(r)); + return match res { + Ok(buf) if !buf.has_remaining() => Poll::Ready(None), + Ok(mut buf) => { + this.cur += buf.remaining() as u64; + Poll::Ready(Some(Ok(buf.copy_to_bytes(buf.remaining())))) + } + Err(err) => Poll::Ready(Some(Err(format_std_io_error(err)))), + }; + } + } + } + } + } } #[cfg(test)] From 9656d785718aab73bc6c2422a0f2a82dc950cbc8 Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Thu, 21 Mar 2024 20:06:31 +0800 Subject: [PATCH 061/111] Everyone is happy Signed-off-by: Xuanwo --- core/benches/vs_s3/src/main.rs | 4 +- core/fuzz/fuzz_reader.rs | 76 ++------------------ core/src/raw/tests/read.rs | 21 ++---- core/src/services/b2/reader.rs | 2 +- core/src/services/cos/reader.rs | 2 +- core/src/services/gcs/reader.rs | 2 +- core/src/services/gdrive/writer.rs | 1 - core/src/services/http/reader.rs | 2 +- core/src/services/huggingface/reader.rs | 2 +- core/src/services/icloud/core.rs | 6 ++ core/src/services/icloud/reader.rs | 2 +- core/src/services/obs/reader.rs | 2 +- core/src/services/oss/reader.rs | 2 +- core/src/services/s3/core.rs | 2 +- core/src/services/swift/reader.rs | 2 +- core/src/services/vercel_artifacts/reader.rs | 2 +- core/src/services/vercel_blob/reader.rs | 2 +- core/src/services/webdav/reader.rs | 2 +- core/src/types/blocking_reader.rs | 5 +- core/src/types/reader.rs | 5 +- 20 files changed, 37 insertions(+), 107 deletions(-) diff --git a/core/benches/vs_s3/src/main.rs b/core/benches/vs_s3/src/main.rs index b8fd9c3fb857..dd28226257df 100644 --- a/core/benches/vs_s3/src/main.rs +++ b/core/benches/vs_s3/src/main.rs @@ -73,7 +73,7 @@ fn bench_read(c: &mut Criterion, op: Operator, s3_client: aws_sdk_s3::Client, bu group.bench_function("opendal_s3_reader", |b| { b.to_async(&*TEST_RUNTIME).iter(|| async { - let mut r = op.reader("file").await.unwrap(); + let r = op.reader("file").await.unwrap(); let mut bs = Vec::new(); let _ = r.read_to_end(&mut bs).await.unwrap(); }); @@ -96,7 +96,7 @@ fn bench_read(c: &mut Criterion, op: Operator, s3_client: aws_sdk_s3::Client, bu group.bench_function("opendal_s3_reader_with_capacity", |b| { b.to_async(&*TEST_RUNTIME).iter(|| async { - let mut r = op.reader("file").await.unwrap(); + let r = op.reader("file").await.unwrap(); let mut bs = Vec::with_capacity(16 * 1024 * 1024); let _ = r.read_to_end(&mut bs).await.unwrap(); }); diff --git a/core/fuzz/fuzz_reader.rs b/core/fuzz/fuzz_reader.rs index c18a348cef84..fb026862c081 100644 --- a/core/fuzz/fuzz_reader.rs +++ b/core/fuzz/fuzz_reader.rs @@ -19,7 +19,6 @@ use std::fmt::Debug; use std::fmt::Formatter; -use std::io::SeekFrom; use libfuzzer_sys::arbitrary::Arbitrary; use libfuzzer_sys::arbitrary::Unstructured; @@ -28,7 +27,6 @@ use opendal::raw::tests::init_test_service; use opendal::raw::tests::ReadAction; use opendal::raw::tests::ReadChecker; use opendal::raw::tests::TEST_RUNTIME; -use opendal::raw::BytesRange; use opendal::Operator; use opendal::Result; use tracing::warn; @@ -39,8 +37,6 @@ const MAX_DATA_SIZE: usize = 16 * 1024 * 1024; struct FuzzInput { path: String, size: usize, - range: BytesRange, - buffer: Option, actions: Vec, } @@ -48,14 +44,12 @@ impl Debug for FuzzInput { fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { let mut actions = self.actions.clone(); // Remove all Read(0) entry. - let empty = ReadAction::Read(0); + let empty = ReadAction::Read(0, 0); actions.retain(|e| e != &empty); f.debug_struct("FuzzInput") .field("path", &self.path) .field("size", &self.size) - .field("range", &self.range.to_string()) - .field("buffer", &self.buffer) .field("actions", &actions) .finish() } @@ -65,85 +59,29 @@ impl Arbitrary<'_> for FuzzInput { fn arbitrary(u: &mut Unstructured<'_>) -> arbitrary::Result { let total_size = u.int_in_range(1..=MAX_DATA_SIZE)?; - // TODO: it's valid that size is larger than total_size. - let (offset, size) = match u.int_in_range(0..=3)? { - // Full range - 0 => (None, None), - 1 => { - let offset = u.int_in_range(0..=total_size as u64 - 1)?; - (Some(offset), None) - } - 2 => { - let size = u.int_in_range(1..=total_size as u64)?; - (None, Some(size)) - } - 3 => { - let offset = u.int_in_range(0..=total_size as u64 - 1)?; - let size = u.int_in_range(1..=total_size as u64 - offset)?; - (Some(offset), Some(size)) - } - _ => unreachable!("invalid int generated by arbitrary"), - }; - let range = BytesRange::new(offset, size); - - let buffer = if u.int_in_range(0..=1)? == 1 { - Some(u.int_in_range(1..=8 * 1024 * 1024)?) - } else { - None - }; - let count = u.int_in_range(1..=1024)?; let mut actions = vec![]; for _ in 0..count { - let action = match u.int_in_range(0..=3)? { - // Read - 0 => { - let size = u.int_in_range(0..=total_size * 2)?; - ReadAction::Read(size) - } - // Seek Start - 1 => { - // NOTE: seek out of the end of file is valid. - let offset = u.int_in_range(0..=total_size * 2)?; - ReadAction::Seek(SeekFrom::Start(offset as u64)) - } - // Seek Current - 2 => { - let offset = u.int_in_range(-(total_size as i64)..=(total_size as i64))?; - ReadAction::Seek(SeekFrom::Current(offset)) - } - // Seek End - 3 => { - let offset = u.int_in_range(-(total_size as i64)..=(total_size as i64))?; - ReadAction::Seek(SeekFrom::End(offset)) - } - _ => unreachable!("invalid int generated by arbitrary"), - }; - - actions.push(action); + let offset = u.int_in_range(0..=total_size)?; + let size = u.int_in_range(0..=total_size * 2)?; + + actions.push(ReadAction::Read(offset, size)); } Ok(FuzzInput { path: uuid::Uuid::new_v4().to_string(), size: total_size, - range, - buffer, actions, }) } } async fn fuzz_reader(op: Operator, input: FuzzInput) -> Result<()> { - let mut checker = ReadChecker::new(input.size, input.range); + let mut checker = ReadChecker::new(input.size); op.write(&input.path, checker.data()).await?; - let mut r = op.reader_with(&input.path); - r = r.range(input.range.to_range()); - if let Some(buffer) = input.buffer { - r = r.buffer(buffer); - } - let r = r.await?; + let r = op.reader(&input.path).await?; checker.check(r, &input.actions).await; diff --git a/core/src/raw/tests/read.rs b/core/src/raw/tests/read.rs index 404c8763e08a..326001a94fc0 100644 --- a/core/src/raw/tests/read.rs +++ b/core/src/raw/tests/read.rs @@ -15,13 +15,12 @@ // specific language governing permissions and limitations // under the License. -use bytes::{Bytes}; +use bytes::Bytes; use rand::thread_rng; use rand::RngCore; use sha2::Digest; use sha2::Sha256; -use crate::raw::*; use crate::*; /// ReadAction represents a read action. @@ -39,8 +38,6 @@ pub enum ReadAction { pub struct ReadChecker { /// Raw Data is the data we write to the storage. raw_data: Bytes, - /// Ranged Data is the data that we read from the storage. - ranged_data: Bytes, } impl ReadChecker { @@ -48,18 +45,14 @@ impl ReadChecker { /// /// It's by design that we use a random generator to generate the raw data. The content of data /// is not important, we only care about the correctness of the read process. - pub fn new(size: usize, range: impl Into) -> Self { + pub fn new(size: usize) -> Self { let mut rng = thread_rng(); let mut data = vec![0; size]; rng.fill_bytes(&mut data); let raw_data = Bytes::from(data); - let ranged_data = range.into().apply_on_bytes(raw_data.clone()); - Self { - raw_data, - ranged_data, - } + Self { raw_data } } /// Return the raw data of this read checker. @@ -83,18 +76,18 @@ impl ReadChecker { if size > 0 && output.is_empty() { assert!( - offset >= self.ranged_data.len(), + offset >= self.raw_data.len(), "check read failed: no data read means cur must outsides of ranged_data", ); return; } assert!( - offset + output.len() <= self.ranged_data.len(), - "check read failed: cur + output length must be less than ranged_data length, offset: {}, output: {}, ranged_data: {}", offset, output.len(), self.ranged_data.len(), + offset + output.len() <= self.raw_data.len(), + "check read failed: cur + output length must be less than ranged_data length, offset: {}, output: {}, ranged_data: {}", offset, output.len(), self.raw_data.len(), ); - let expected = &self.ranged_data[offset..offset + output.len()]; + let expected = &self.raw_data[offset..offset + output.len()]; // Check the read result assert_eq!( diff --git a/core/src/services/b2/reader.rs b/core/src/services/b2/reader.rs index 17455e1eed21..f412f95156a3 100644 --- a/core/src/services/b2/reader.rs +++ b/core/src/services/b2/reader.rs @@ -34,7 +34,7 @@ impl B2Reader { B2Reader { core, path: path.to_string(), - op: op, + op, } } } diff --git a/core/src/services/cos/reader.rs b/core/src/services/cos/reader.rs index 558349f425d0..3b0eadd7dbd9 100644 --- a/core/src/services/cos/reader.rs +++ b/core/src/services/cos/reader.rs @@ -33,7 +33,7 @@ impl CosReader { CosReader { core, path: path.to_string(), - op: op, + op, } } } diff --git a/core/src/services/gcs/reader.rs b/core/src/services/gcs/reader.rs index da42c3cb7d65..3ec36f3ea289 100644 --- a/core/src/services/gcs/reader.rs +++ b/core/src/services/gcs/reader.rs @@ -34,7 +34,7 @@ impl GcsReader { GcsReader { core, path: path.to_string(), - op: op, + op, } } } diff --git a/core/src/services/gdrive/writer.rs b/core/src/services/gdrive/writer.rs index ed4689cfcb9f..ebd91afbdd67 100644 --- a/core/src/services/gdrive/writer.rs +++ b/core/src/services/gdrive/writer.rs @@ -68,7 +68,6 @@ impl oio::OneShotWrite for GdriveWriter { let file: GdriveFile = serde_json::from_reader(bs.reader()).map_err(new_json_deserialize_error)?; self.core.path_cache.insert(&self.path, &file.id).await; - } else { } Ok(()) } diff --git a/core/src/services/http/reader.rs b/core/src/services/http/reader.rs index f981d324e726..1d0e8ae17926 100644 --- a/core/src/services/http/reader.rs +++ b/core/src/services/http/reader.rs @@ -32,7 +32,7 @@ impl HttpReader { HttpReader { core, path: path.to_string(), - op: op, + op, } } } diff --git a/core/src/services/huggingface/reader.rs b/core/src/services/huggingface/reader.rs index 6531a22f4ee7..dcf98c0afcd9 100644 --- a/core/src/services/huggingface/reader.rs +++ b/core/src/services/huggingface/reader.rs @@ -34,7 +34,7 @@ impl HuggingfaceReader { HuggingfaceReader { core, path: path.to_string(), - op: op, + op, } } } diff --git a/core/src/services/icloud/core.rs b/core/src/services/icloud/core.rs index 6cef03f9624d..fe1008c0b55d 100644 --- a/core/src/services/icloud/core.rs +++ b/core/src/services/icloud/core.rs @@ -78,6 +78,12 @@ pub struct SessionData { docws_url: String, } +impl Default for SessionData { + fn default() -> Self { + Self::new() + } +} + impl SessionData { pub fn new() -> SessionData { Self { diff --git a/core/src/services/icloud/reader.rs b/core/src/services/icloud/reader.rs index ee38c2324441..8cbbdc832ad4 100644 --- a/core/src/services/icloud/reader.rs +++ b/core/src/services/icloud/reader.rs @@ -33,7 +33,7 @@ impl IcloudReader { IcloudReader { core, path: path.to_string(), - op: op, + op, } } } diff --git a/core/src/services/obs/reader.rs b/core/src/services/obs/reader.rs index 0f8301c156f5..fa15062a061c 100644 --- a/core/src/services/obs/reader.rs +++ b/core/src/services/obs/reader.rs @@ -34,7 +34,7 @@ impl ObsReader { ObsReader { core, path: path.to_string(), - op: op, + op, } } } diff --git a/core/src/services/oss/reader.rs b/core/src/services/oss/reader.rs index fa2e692d3400..366b44d474cc 100644 --- a/core/src/services/oss/reader.rs +++ b/core/src/services/oss/reader.rs @@ -34,7 +34,7 @@ impl OssReader { OssReader { core, path: path.to_string(), - op: op, + op, } } } diff --git a/core/src/services/s3/core.rs b/core/src/services/s3/core.rs index 228a037026fd..4f93f08fea37 100644 --- a/core/src/services/s3/core.rs +++ b/core/src/services/s3/core.rs @@ -368,7 +368,7 @@ impl S3Core { range: BytesRange, args: &OpRead, ) -> Result> { - let mut req = self.s3_get_object_request(path, range, &args)?; + let mut req = self.s3_get_object_request(path, range, args)?; self.sign(&mut req).await?; diff --git a/core/src/services/swift/reader.rs b/core/src/services/swift/reader.rs index 777530f14005..b0dfbe046a4a 100644 --- a/core/src/services/swift/reader.rs +++ b/core/src/services/swift/reader.rs @@ -34,7 +34,7 @@ impl SwiftReader { SwiftReader { core, path: path.to_string(), - op: op, + op, } } } diff --git a/core/src/services/vercel_artifacts/reader.rs b/core/src/services/vercel_artifacts/reader.rs index a36812ebc7db..55d7e1562967 100644 --- a/core/src/services/vercel_artifacts/reader.rs +++ b/core/src/services/vercel_artifacts/reader.rs @@ -32,7 +32,7 @@ impl VercelArtifactsReader { VercelArtifactsReader { core, path: path.to_string(), - op: op, + op, } } } diff --git a/core/src/services/vercel_blob/reader.rs b/core/src/services/vercel_blob/reader.rs index b41866616ed0..f7e16e7b4dbe 100644 --- a/core/src/services/vercel_blob/reader.rs +++ b/core/src/services/vercel_blob/reader.rs @@ -34,7 +34,7 @@ impl VercelBlobReader { VercelBlobReader { core, path: path.to_string(), - op: op, + op, } } } diff --git a/core/src/services/webdav/reader.rs b/core/src/services/webdav/reader.rs index a87793a70153..8176ae239b47 100644 --- a/core/src/services/webdav/reader.rs +++ b/core/src/services/webdav/reader.rs @@ -34,7 +34,7 @@ impl WebdavReader { WebdavReader { core, path: path.to_string(), - op: op, + op, } } } diff --git a/core/src/types/blocking_reader.rs b/core/src/types/blocking_reader.rs index 890483267b57..62468f50ce9d 100644 --- a/core/src/types/blocking_reader.rs +++ b/core/src/types/blocking_reader.rs @@ -80,10 +80,7 @@ impl BlockingReader { } let mut offset = start; - let mut size = match end { - Some(end) => Some(end - start), - None => None, - }; + let mut size = end.map(|end| end - start); let mut read = 0; loop { diff --git a/core/src/types/reader.rs b/core/src/types/reader.rs index 30c40cc54028..9c3f6e7404f5 100644 --- a/core/src/types/reader.rs +++ b/core/src/types/reader.rs @@ -94,10 +94,7 @@ impl Reader { } let mut offset = start; - let mut size = match end { - Some(end) => Some(end - start), - None => None, - }; + let mut size = end.map(|end| end - start); let mut read = 0; loop { From 3c0dfd3716914617003a01df5efba17767478ff8 Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Thu, 21 Mar 2024 20:09:08 +0800 Subject: [PATCH 062/111] Format code Signed-off-by: Xuanwo --- core/src/layers/blocking.rs | 1 - core/src/layers/chaos.rs | 5 ++- core/src/layers/complete.rs | 1 - core/src/layers/concurrent_limit.rs | 3 -- core/src/layers/dtrace.rs | 2 -- core/src/layers/error_context.rs | 2 -- core/src/layers/immutable_index.rs | 1 - core/src/layers/logging.rs | 8 ++--- core/src/layers/metrics.rs | 5 ++- core/src/layers/minitrace.rs | 2 -- core/src/layers/oteltrace.rs | 1 - core/src/layers/prometheus.rs | 5 ++- core/src/layers/prometheus_client.rs | 6 ++-- core/src/layers/retry.rs | 6 ++-- core/src/layers/throttle.rs | 2 -- core/src/layers/timeout.rs | 5 ++- core/src/layers/tracing.rs | 2 -- core/src/raw/adapters/kv/backend.rs | 1 - core/src/raw/adapters/typed_kv/backend.rs | 1 - core/src/raw/enum_utils.rs | 3 -- core/src/raw/http_util/client.rs | 3 +- core/src/raw/http_util/multipart.rs | 1 - core/src/raw/oio/buf/buffer.rs | 6 +++- core/src/raw/oio/list/api.rs | 1 - core/src/raw/oio/stream/api.rs | 3 +- core/src/raw/oio/write/api.rs | 3 +- core/src/raw/oio/write/block_write.rs | 1 - core/src/raw/oio/write/exact_buf_write.rs | 5 ++- core/src/raw/oio/write/multipart_write.rs | 1 - core/src/raw/oio/write/range_write.rs | 1 - core/src/raw/std_io_util.rs | 3 +- core/src/services/alluxio/core.rs | 2 +- core/src/services/alluxio/writer.rs | 1 - core/src/services/atomicserver/backend.rs | 3 +- core/src/services/azdls/lister.rs | 2 +- core/src/services/azdls/reader.rs | 7 ++-- core/src/services/azdls/writer.rs | 1 - core/src/services/azfile/lister.rs | 6 ++-- core/src/services/azfile/reader.rs | 6 ++-- core/src/services/b2/core.rs | 2 +- core/src/services/b2/reader.rs | 6 ++-- core/src/services/b2/writer.rs | 2 +- core/src/services/chainsafe/lister.rs | 2 +- core/src/services/chainsafe/reader.rs | 7 ++-- core/src/services/cos/reader.rs | 6 ++-- core/src/services/dbfs/error.rs | 2 +- core/src/services/dbfs/lister.rs | 2 +- core/src/services/dbfs/writer.rs | 1 - core/src/services/dropbox/core.rs | 3 +- core/src/services/dropbox/reader.rs | 7 ++-- core/src/services/fs/backend.rs | 1 - core/src/services/fs/reader.rs | 3 +- core/src/services/fs/writer.rs | 3 +- core/src/services/ftp/backend.rs | 1 - core/src/services/ftp/lister.rs | 1 - core/src/services/ftp/reader.rs | 3 +- core/src/services/ftp/writer.rs | 1 - core/src/services/gcs/lister.rs | 2 +- core/src/services/gcs/reader.rs | 7 ++-- core/src/services/gdrive/backend.rs | 3 +- core/src/services/gdrive/core.rs | 3 +- core/src/services/gdrive/reader.rs | 7 ++-- core/src/services/gdrive/writer.rs | 3 +- core/src/services/ghac/backend.rs | 3 +- core/src/services/ghac/reader.rs | 3 +- core/src/services/github/core.rs | 6 ++-- core/src/services/github/reader.rs | 7 ++-- core/src/services/hdfs/backend.rs | 1 - core/src/services/hdfs/reader.rs | 10 +++--- core/src/services/hdfs/writer.rs | 2 -- core/src/services/hdfs_native/reader.rs | 4 --- core/src/services/hdfs_native/writer.rs | 2 -- core/src/services/http/reader.rs | 3 +- core/src/services/huggingface/error.rs | 2 +- core/src/services/huggingface/lister.rs | 3 +- core/src/services/huggingface/reader.rs | 7 ++-- core/src/services/icloud/backend.rs | 1 - core/src/services/icloud/core.rs | 3 +- core/src/services/icloud/reader.rs | 8 +++-- core/src/services/ipfs/reader.rs | 3 +- core/src/services/ipmfs/backend.rs | 3 +- core/src/services/ipmfs/lister.rs | 2 +- core/src/services/ipmfs/reader.rs | 3 +- core/src/services/ipmfs/writer.rs | 1 - core/src/services/koofr/core.rs | 3 +- core/src/services/koofr/lister.rs | 3 +- core/src/services/koofr/reader.rs | 7 ++-- core/src/services/libsql/backend.rs | 3 +- core/src/services/memory/backend.rs | 6 ++-- core/src/services/obs/reader.rs | 7 ++-- core/src/services/onedrive/backend.rs | 3 +- core/src/services/onedrive/lister.rs | 3 +- core/src/services/onedrive/reader.rs | 3 +- core/src/services/onedrive/writer.rs | 3 +- core/src/services/oss/reader.rs | 7 ++-- core/src/services/pcloud/core.rs | 6 ++-- core/src/services/pcloud/error.rs | 2 +- core/src/services/pcloud/lister.rs | 2 +- core/src/services/pcloud/reader.rs | 7 ++-- core/src/services/pcloud/writer.rs | 3 +- core/src/services/s3/reader.rs | 6 ++-- core/src/services/s3/writer.rs | 2 +- core/src/services/seafile/backend.rs | 2 -- core/src/services/seafile/core.rs | 3 +- core/src/services/seafile/lister.rs | 2 +- core/src/services/seafile/reader.rs | 7 ++-- core/src/services/sftp/backend.rs | 2 -- core/src/services/supabase/reader.rs | 7 ++-- core/src/services/supabase/writer.rs | 1 - core/src/services/swift/lister.rs | 3 +- core/src/services/swift/reader.rs | 7 ++-- core/src/services/swift/writer.rs | 1 - core/src/services/upyun/lister.rs | 3 +- core/src/services/upyun/reader.rs | 7 ++-- core/src/services/vercel_artifacts/reader.rs | 3 +- core/src/services/vercel_blob/core.rs | 3 +- core/src/services/vercel_blob/reader.rs | 7 ++-- core/src/services/vercel_blob/writer.rs | 2 +- core/src/services/webdav/reader.rs | 7 ++-- core/src/services/webdav/writer.rs | 1 - core/src/services/webhdfs/backend.rs | 1 - core/src/services/webhdfs/reader.rs | 8 +++-- core/src/services/yandex_disk/backend.rs | 1 - core/src/services/yandex_disk/core.rs | 2 +- core/src/services/yandex_disk/lister.rs | 3 +- core/src/services/yandex_disk/reader.rs | 9 +++-- core/src/types/blocking_reader.rs | 4 +-- core/src/types/operator/blocking_operator.rs | 3 +- core/src/types/reader.rs | 38 +++++++++++++------- core/src/types/writer.rs | 6 ++-- 130 files changed, 258 insertions(+), 234 deletions(-) diff --git a/core/src/layers/blocking.rs b/core/src/layers/blocking.rs index 93c8d0ed4882..37cc798109a2 100644 --- a/core/src/layers/blocking.rs +++ b/core/src/layers/blocking.rs @@ -18,7 +18,6 @@ use async_trait::async_trait; use bytes; use bytes::Bytes; - use tokio::runtime::Handle; use crate::raw::*; diff --git a/core/src/layers/chaos.rs b/core/src/layers/chaos.rs index b5f58352505e..063bc3f9f6bf 100644 --- a/core/src/layers/chaos.rs +++ b/core/src/layers/chaos.rs @@ -15,11 +15,10 @@ // specific language governing permissions and limitations // under the License. - -use std::sync::{Arc, Mutex}; +use std::sync::Arc; +use std::sync::Mutex; use async_trait::async_trait; - use futures::FutureExt; use rand::prelude::*; use rand::rngs::StdRng; diff --git a/core/src/layers/complete.rs b/core/src/layers/complete.rs index 2271e91e78c1..17f59e811614 100644 --- a/core/src/layers/complete.rs +++ b/core/src/layers/complete.rs @@ -18,7 +18,6 @@ use std::cmp; use std::fmt::Debug; use std::fmt::Formatter; - use std::sync::Arc; use async_trait::async_trait; diff --git a/core/src/layers/concurrent_limit.rs b/core/src/layers/concurrent_limit.rs index 9cf85355378c..44d968024f35 100644 --- a/core/src/layers/concurrent_limit.rs +++ b/core/src/layers/concurrent_limit.rs @@ -16,9 +16,6 @@ // under the License. use std::fmt::Debug; - - - use std::sync::Arc; use async_trait::async_trait; diff --git a/core/src/layers/dtrace.rs b/core/src/layers/dtrace.rs index 2713cda6c1e0..5787f845a481 100644 --- a/core/src/layers/dtrace.rs +++ b/core/src/layers/dtrace.rs @@ -19,8 +19,6 @@ use std::ffi::CString; use std::fmt::Debug; use std::fmt::Formatter; - - use async_trait::async_trait; use bytes::Buf; use bytes::Bytes; diff --git a/core/src/layers/error_context.rs b/core/src/layers/error_context.rs index 0e40252f961b..2db5b5b5ea39 100644 --- a/core/src/layers/error_context.rs +++ b/core/src/layers/error_context.rs @@ -18,8 +18,6 @@ use std::fmt::Debug; use std::fmt::Formatter; - - use async_trait::async_trait; use bytes::Bytes; use futures::TryFutureExt; diff --git a/core/src/layers/immutable_index.rs b/core/src/layers/immutable_index.rs index a6eb054063ae..cf7458cd6b4d 100644 --- a/core/src/layers/immutable_index.rs +++ b/core/src/layers/immutable_index.rs @@ -17,7 +17,6 @@ use std::collections::HashSet; use std::fmt::Debug; - use std::vec::IntoIter; use async_trait::async_trait; diff --git a/core/src/layers/logging.rs b/core/src/layers/logging.rs index 9825a3dc5af7..b74dc5f80c52 100644 --- a/core/src/layers/logging.rs +++ b/core/src/layers/logging.rs @@ -16,18 +16,18 @@ // under the License. use std::fmt::Debug; - - +use std::sync::atomic::AtomicU64; +use std::sync::atomic::Ordering; use async_trait::async_trait; -use bytes::{Buf, Bytes}; +use bytes::Buf; +use bytes::Bytes; use futures::FutureExt; use futures::TryFutureExt; use log::debug; use log::log; use log::trace; use log::Level; -use std::sync::atomic::{AtomicU64, Ordering}; use crate::raw::oio::ReadOperation; use crate::raw::oio::WriteOperation; diff --git a/core/src/layers/metrics.rs b/core/src/layers/metrics.rs index 12a0aa4f8c53..b1e1c615c817 100644 --- a/core/src/layers/metrics.rs +++ b/core/src/layers/metrics.rs @@ -17,13 +17,12 @@ use std::fmt::Debug; use std::fmt::Formatter; - use std::sync::Arc; - use std::time::Instant; use async_trait::async_trait; -use bytes::{Buf, Bytes}; +use bytes::Buf; +use bytes::Bytes; use futures::FutureExt; use futures::TryFutureExt; use metrics::increment_counter; diff --git a/core/src/layers/minitrace.rs b/core/src/layers/minitrace.rs index 9dc60d0bf82e..254fd11128a2 100644 --- a/core/src/layers/minitrace.rs +++ b/core/src/layers/minitrace.rs @@ -18,8 +18,6 @@ use std::fmt::Debug; use std::future::Future; - - use async_trait::async_trait; use bytes::Bytes; use futures::FutureExt; diff --git a/core/src/layers/oteltrace.rs b/core/src/layers/oteltrace.rs index fd2ba5bb4b94..574eb58c004c 100644 --- a/core/src/layers/oteltrace.rs +++ b/core/src/layers/oteltrace.rs @@ -17,7 +17,6 @@ use std::future::Future; - use async_trait::async_trait; use bytes::Bytes; use futures::FutureExt; diff --git a/core/src/layers/prometheus.rs b/core/src/layers/prometheus.rs index c8e96e0d5272..224db75c9814 100644 --- a/core/src/layers/prometheus.rs +++ b/core/src/layers/prometheus.rs @@ -17,12 +17,11 @@ use std::fmt::Debug; use std::fmt::Formatter; - - use std::sync::Arc; use async_trait::async_trait; -use bytes::{Buf, Bytes}; +use bytes::Buf; +use bytes::Bytes; use futures::FutureExt; use futures::TryFutureExt; use log::debug; diff --git a/core/src/layers/prometheus_client.rs b/core/src/layers/prometheus_client.rs index a703fbb690fb..967cb8800142 100644 --- a/core/src/layers/prometheus_client.rs +++ b/core/src/layers/prometheus_client.rs @@ -17,15 +17,13 @@ use std::fmt::Debug; use std::fmt::Formatter; - - use std::sync::Arc; - use std::time::Duration; use std::time::Instant; use async_trait::async_trait; -use bytes::{Buf, Bytes}; +use bytes::Buf; +use bytes::Bytes; use futures::FutureExt; use futures::TryFutureExt; use prometheus_client::metrics::counter::Counter; diff --git a/core/src/layers/retry.rs b/core/src/layers/retry.rs index a9736199866b..b2bd2d791fa4 100644 --- a/core/src/layers/retry.rs +++ b/core/src/layers/retry.rs @@ -17,23 +17,21 @@ use std::fmt::Debug; use std::fmt::Formatter; - use std::sync::Arc; - use std::time::Duration; use async_trait::async_trait; use backon::BlockingRetryable; - use backon::ExponentialBuilder; use backon::Retryable; use bytes::Bytes; use futures::FutureExt; use log::warn; +use crate::raw::oio::Buffer; +use crate::raw::oio::ListOperation; use crate::raw::oio::ReadOperation; use crate::raw::oio::WriteOperation; -use crate::raw::oio::{Buffer, ListOperation}; use crate::raw::*; use crate::*; diff --git a/core/src/layers/throttle.rs b/core/src/layers/throttle.rs index 9f7373f40fce..a21b2d94f1c2 100644 --- a/core/src/layers/throttle.rs +++ b/core/src/layers/throttle.rs @@ -15,10 +15,8 @@ // specific language governing permissions and limitations // under the License. - use std::num::NonZeroU32; use std::sync::Arc; - use std::thread; use async_trait::async_trait; diff --git a/core/src/layers/timeout.rs b/core/src/layers/timeout.rs index d6b99e7fa707..fdd7faf8af51 100644 --- a/core/src/layers/timeout.rs +++ b/core/src/layers/timeout.rs @@ -16,7 +16,6 @@ // under the License. use std::future::Future; - use std::time::Duration; use async_trait::async_trait; @@ -325,9 +324,9 @@ impl oio::List for TimeoutWrapper { #[cfg(test)] mod tests { - use std::future::{pending, Future}; + use std::future::pending; + use std::future::Future; use std::sync::Arc; - use std::time::Duration; use async_trait::async_trait; diff --git a/core/src/layers/tracing.rs b/core/src/layers/tracing.rs index 8e1e439ee320..bbdc6b204b40 100644 --- a/core/src/layers/tracing.rs +++ b/core/src/layers/tracing.rs @@ -18,8 +18,6 @@ use std::fmt::Debug; use std::future::Future; - - use async_trait::async_trait; use bytes::Bytes; use futures::FutureExt; diff --git a/core/src/raw/adapters/kv/backend.rs b/core/src/raw/adapters/kv/backend.rs index 05a02b76acb5..cea7c982c7fa 100644 --- a/core/src/raw/adapters/kv/backend.rs +++ b/core/src/raw/adapters/kv/backend.rs @@ -16,7 +16,6 @@ // under the License. use std::sync::Arc; - use std::vec::IntoIter; use async_trait::async_trait; diff --git a/core/src/raw/adapters/typed_kv/backend.rs b/core/src/raw/adapters/typed_kv/backend.rs index cf4e0136ed04..edc48285df67 100644 --- a/core/src/raw/adapters/typed_kv/backend.rs +++ b/core/src/raw/adapters/typed_kv/backend.rs @@ -16,7 +16,6 @@ // under the License. use std::sync::Arc; - use std::vec::IntoIter; use async_trait::async_trait; diff --git a/core/src/raw/enum_utils.rs b/core/src/raw/enum_utils.rs index 315aa5c11c85..475cbe98f2e4 100644 --- a/core/src/raw/enum_utils.rs +++ b/core/src/raw/enum_utils.rs @@ -38,9 +38,6 @@ //! This module is used to provide some enums for the above code. We should remove this module once //! type_alias_impl_trait has been stabilized. - - - use bytes::Bytes; use crate::raw::oio::Buffer; diff --git a/core/src/raw/http_util/client.rs b/core/src/raw/http_util/client.rs index a7f1f9291a03..6b772da36523 100644 --- a/core/src/raw/http_util/client.rs +++ b/core/src/raw/http_util/client.rs @@ -15,13 +15,14 @@ // specific language governing permissions and limitations // under the License. -use bytes::{Buf, Bytes}; use std::cmp::Ordering; use std::fmt::Debug; use std::fmt::Formatter; use std::mem; use std::str::FromStr; +use bytes::Buf; +use bytes::Bytes; use futures::TryStreamExt; use http::Request; use http::Response; diff --git a/core/src/raw/http_util/multipart.rs b/core/src/raw/http_util/multipart.rs index 81a9b2c6ae9d..1913add572e1 100644 --- a/core/src/raw/http_util/multipart.rs +++ b/core/src/raw/http_util/multipart.rs @@ -24,7 +24,6 @@ use std::task::Poll; use bytes::Bytes; use bytes::BytesMut; - use http::header::CONTENT_DISPOSITION; use http::header::CONTENT_LENGTH; use http::header::CONTENT_TYPE; diff --git a/core/src/raw/oio/buf/buffer.rs b/core/src/raw/oio/buf/buffer.rs index ca754fa91c54..e2e9a5260e81 100644 --- a/core/src/raw/oio/buf/buffer.rs +++ b/core/src/raw/oio/buf/buffer.rs @@ -15,9 +15,13 @@ // specific language governing permissions and limitations // under the License. -use bytes::{Buf, BufMut, Bytes, BytesMut}; use std::collections::VecDeque; +use bytes::Buf; +use bytes::BufMut; +use bytes::Bytes; +use bytes::BytesMut; + /// Buffer is a wrapper of `Bytes` and `VecDeque`. /// /// We designed buffer to allow underlying storage to return non-contiguous bytes. diff --git a/core/src/raw/oio/list/api.rs b/core/src/raw/oio/list/api.rs index e5c62054cd00..48368c7a93a4 100644 --- a/core/src/raw/oio/list/api.rs +++ b/core/src/raw/oio/list/api.rs @@ -18,7 +18,6 @@ use std::fmt::Display; use std::fmt::Formatter; use std::future::Future; - use std::ops::DerefMut; use crate::raw::oio::Entry; diff --git a/core/src/raw/oio/stream/api.rs b/core/src/raw/oio/stream/api.rs index 9837e308616d..4e912c303c1e 100644 --- a/core/src/raw/oio/stream/api.rs +++ b/core/src/raw/oio/stream/api.rs @@ -21,8 +21,9 @@ use std::task::ready; use std::task::Context; use std::task::Poll; +use bytes::Buf; +use bytes::Bytes; use bytes::BytesMut; -use bytes::{Buf, Bytes}; use crate::*; diff --git a/core/src/raw/oio/write/api.rs b/core/src/raw/oio/write/api.rs index cb6707df7314..9d250d777bb0 100644 --- a/core/src/raw/oio/write/api.rs +++ b/core/src/raw/oio/write/api.rs @@ -15,12 +15,13 @@ // specific language governing permissions and limitations // under the License. -use bytes::Bytes; use std::fmt::Display; use std::fmt::Formatter; use std::future::Future; use std::ops::DerefMut; +use bytes::Bytes; + use crate::raw::*; use crate::*; diff --git a/core/src/raw/oio/write/block_write.rs b/core/src/raw/oio/write/block_write.rs index 3b0266ffbdb6..1c867961a679 100644 --- a/core/src/raw/oio/write/block_write.rs +++ b/core/src/raw/oio/write/block_write.rs @@ -17,7 +17,6 @@ use std::pin::Pin; use std::sync::Arc; - use std::task::Context; use std::task::Poll; diff --git a/core/src/raw/oio/write/exact_buf_write.rs b/core/src/raw/oio/write/exact_buf_write.rs index fba3d8dbaa58..3947bcf03341 100644 --- a/core/src/raw/oio/write/exact_buf_write.rs +++ b/core/src/raw/oio/write/exact_buf_write.rs @@ -15,9 +15,12 @@ // specific language governing permissions and limitations // under the License. -use bytes::{Buf, BufMut, Bytes}; use std::mem; +use bytes::Buf; +use bytes::BufMut; +use bytes::Bytes; + use crate::raw::*; use crate::*; diff --git a/core/src/raw/oio/write/multipart_write.rs b/core/src/raw/oio/write/multipart_write.rs index 2f4339329000..726f9bd3bfc6 100644 --- a/core/src/raw/oio/write/multipart_write.rs +++ b/core/src/raw/oio/write/multipart_write.rs @@ -17,7 +17,6 @@ use std::pin::Pin; use std::sync::Arc; - use std::task::Context; use std::task::Poll; diff --git a/core/src/raw/oio/write/range_write.rs b/core/src/raw/oio/write/range_write.rs index a8de1366fe4e..c858f14e9337 100644 --- a/core/src/raw/oio/write/range_write.rs +++ b/core/src/raw/oio/write/range_write.rs @@ -17,7 +17,6 @@ use std::pin::Pin; use std::sync::Arc; - use std::task::Context; use std::task::Poll; diff --git a/core/src/raw/std_io_util.rs b/core/src/raw/std_io_util.rs index 1efe5d98c095..9aac5444b220 100644 --- a/core/src/raw/std_io_util.rs +++ b/core/src/raw/std_io_util.rs @@ -15,9 +15,10 @@ // specific language governing permissions and limitations // under the License. -use crate::*; use std::io; +use crate::*; + /// Parse std io error into opendal::Error. /// /// # TODO diff --git a/core/src/services/alluxio/core.rs b/core/src/services/alluxio/core.rs index 718e7a5315fa..91dc086579d9 100644 --- a/core/src/services/alluxio/core.rs +++ b/core/src/services/alluxio/core.rs @@ -15,10 +15,10 @@ // specific language governing permissions and limitations // under the License. -use bytes::Buf; use std::fmt::Debug; use std::fmt::Formatter; +use bytes::Buf; use http::header::RANGE; use http::Request; use http::Response; diff --git a/core/src/services/alluxio/writer.rs b/core/src/services/alluxio/writer.rs index eed45f2cab08..6232799eea74 100644 --- a/core/src/services/alluxio/writer.rs +++ b/core/src/services/alluxio/writer.rs @@ -20,7 +20,6 @@ use std::sync::Arc; use bytes::Bytes; use super::core::AlluxioCore; - use crate::raw::*; use crate::*; diff --git a/core/src/services/atomicserver/backend.rs b/core/src/services/atomicserver/backend.rs index db2d54ed6fa2..bc8742f58719 100644 --- a/core/src/services/atomicserver/backend.rs +++ b/core/src/services/atomicserver/backend.rs @@ -23,7 +23,8 @@ use async_trait::async_trait; use atomic_lib::agents::Agent; use atomic_lib::client::get_authentication_headers; use atomic_lib::commit::sign_message; -use bytes::{Buf, Bytes}; +use bytes::Buf; +use bytes::Bytes; use http::header::CONTENT_DISPOSITION; use http::header::CONTENT_TYPE; use http::Request; diff --git a/core/src/services/azdls/lister.rs b/core/src/services/azdls/lister.rs index a10b41af1b88..27d51e266765 100644 --- a/core/src/services/azdls/lister.rs +++ b/core/src/services/azdls/lister.rs @@ -15,9 +15,9 @@ // specific language governing permissions and limitations // under the License. -use bytes::Buf; use std::sync::Arc; +use bytes::Buf; use serde::Deserialize; use serde_json::de; diff --git a/core/src/services/azdls/reader.rs b/core/src/services/azdls/reader.rs index 3e3fc88fcaaf..914c8a62fec8 100644 --- a/core/src/services/azdls/reader.rs +++ b/core/src/services/azdls/reader.rs @@ -15,12 +15,13 @@ // specific language governing permissions and limitations // under the License. +use std::sync::Arc; + +use http::StatusCode; + use super::core::AzdlsCore; use super::error::parse_error; use crate::raw::*; -use http::StatusCode; - -use std::sync::Arc; pub struct AzdlsReader { core: Arc, diff --git a/core/src/services/azdls/writer.rs b/core/src/services/azdls/writer.rs index 7dd3638b9f03..aabd8133f1b5 100644 --- a/core/src/services/azdls/writer.rs +++ b/core/src/services/azdls/writer.rs @@ -22,7 +22,6 @@ use http::StatusCode; use super::core::AzdlsCore; use super::error::parse_error; - use crate::raw::*; use crate::*; diff --git a/core/src/services/azfile/lister.rs b/core/src/services/azfile/lister.rs index 11cbe99a10df..913af07d838f 100644 --- a/core/src/services/azfile/lister.rs +++ b/core/src/services/azfile/lister.rs @@ -15,12 +15,11 @@ // specific language governing permissions and limitations // under the License. -use bytes::Buf; use std::sync::Arc; +use bytes::Buf; use http::StatusCode; use quick_xml::de; - use serde::Deserialize; use super::core::AzfileCore; @@ -149,9 +148,10 @@ struct Properties { #[cfg(test)] mod tests { - use super::*; use quick_xml::de::from_str; + use super::*; + #[test] fn test_parse_list_result() { let xml = r#" diff --git a/core/src/services/azfile/reader.rs b/core/src/services/azfile/reader.rs index c92023bff734..651188b8ef8f 100644 --- a/core/src/services/azfile/reader.rs +++ b/core/src/services/azfile/reader.rs @@ -15,12 +15,14 @@ // specific language governing permissions and limitations // under the License. +use std::sync::Arc; + +use http::StatusCode; + use super::core::AzfileCore; use super::error::parse_error; use crate::raw::*; use crate::*; -use http::StatusCode; -use std::sync::Arc; pub struct AzfileReader { core: Arc, diff --git a/core/src/services/b2/core.rs b/core/src/services/b2/core.rs index c2b3a2e3feed..461d1a7b2fc6 100644 --- a/core/src/services/b2/core.rs +++ b/core/src/services/b2/core.rs @@ -15,12 +15,12 @@ // specific language governing permissions and limitations // under the License. -use bytes::Buf; use std::fmt::Debug; use std::fmt::Formatter; use std::sync::Arc; use std::time::Duration; +use bytes::Buf; use chrono::DateTime; use chrono::Utc; use http::header; diff --git a/core/src/services/b2/reader.rs b/core/src/services/b2/reader.rs index f412f95156a3..63cb47b6b78c 100644 --- a/core/src/services/b2/reader.rs +++ b/core/src/services/b2/reader.rs @@ -15,12 +15,14 @@ // specific language governing permissions and limitations // under the License. +use std::sync::Arc; + +use http::StatusCode; + use super::core::*; use super::error::*; use crate::raw::*; use crate::*; -use http::StatusCode; -use std::sync::Arc; pub struct B2Reader { core: Arc, diff --git a/core/src/services/b2/writer.rs b/core/src/services/b2/writer.rs index 66c7beabefc0..51df2623fa84 100644 --- a/core/src/services/b2/writer.rs +++ b/core/src/services/b2/writer.rs @@ -15,9 +15,9 @@ // specific language governing permissions and limitations // under the License. -use bytes::Buf; use std::sync::Arc; +use bytes::Buf; use http::StatusCode; use super::core::B2Core; diff --git a/core/src/services/chainsafe/lister.rs b/core/src/services/chainsafe/lister.rs index 95d8beb6afd2..4d419cd3d5dc 100644 --- a/core/src/services/chainsafe/lister.rs +++ b/core/src/services/chainsafe/lister.rs @@ -15,9 +15,9 @@ // specific language governing permissions and limitations // under the License. -use bytes::Buf; use std::sync::Arc; +use bytes::Buf; use http::StatusCode; use super::core::parse_info; diff --git a/core/src/services/chainsafe/reader.rs b/core/src/services/chainsafe/reader.rs index 79a6f9f2995c..b6813660dd67 100644 --- a/core/src/services/chainsafe/reader.rs +++ b/core/src/services/chainsafe/reader.rs @@ -15,12 +15,13 @@ // specific language governing permissions and limitations // under the License. +use std::sync::Arc; + +use http::StatusCode; + use super::core::ChainsafeCore; use super::error::parse_error; use crate::raw::*; -use http::StatusCode; - -use std::sync::Arc; pub struct ChainsafeReader { core: Arc, diff --git a/core/src/services/cos/reader.rs b/core/src/services/cos/reader.rs index 3b0eadd7dbd9..57af3a1607e4 100644 --- a/core/src/services/cos/reader.rs +++ b/core/src/services/cos/reader.rs @@ -15,11 +15,13 @@ // specific language governing permissions and limitations // under the License. +use std::sync::Arc; + +use http::StatusCode; + use super::core::CosCore; use super::error::parse_error; use crate::raw::*; -use http::StatusCode; -use std::sync::Arc; pub struct CosReader { core: Arc, diff --git a/core/src/services/dbfs/error.rs b/core/src/services/dbfs/error.rs index 53a5e4d4eba2..c65612fe3ddc 100644 --- a/core/src/services/dbfs/error.rs +++ b/core/src/services/dbfs/error.rs @@ -15,9 +15,9 @@ // specific language governing permissions and limitations // under the License. -use bytes::Buf; use std::fmt::Debug; +use bytes::Buf; use http::Response; use http::StatusCode; use serde::Deserialize; diff --git a/core/src/services/dbfs/lister.rs b/core/src/services/dbfs/lister.rs index f6b31091d6f0..6a48066a584b 100644 --- a/core/src/services/dbfs/lister.rs +++ b/core/src/services/dbfs/lister.rs @@ -15,9 +15,9 @@ // specific language governing permissions and limitations // under the License. -use bytes::Buf; use std::sync::Arc; +use bytes::Buf; use http::StatusCode; use serde::Deserialize; diff --git a/core/src/services/dbfs/writer.rs b/core/src/services/dbfs/writer.rs index 625dc552e269..390f973269b4 100644 --- a/core/src/services/dbfs/writer.rs +++ b/core/src/services/dbfs/writer.rs @@ -21,7 +21,6 @@ use bytes::Bytes; use http::StatusCode; use super::error::parse_error; - use crate::raw::*; use crate::services::dbfs::core::DbfsCore; use crate::*; diff --git a/core/src/services/dropbox/core.rs b/core/src/services/dropbox/core.rs index 9301d37de580..71ed3a085862 100644 --- a/core/src/services/dropbox/core.rs +++ b/core/src/services/dropbox/core.rs @@ -23,7 +23,8 @@ use std::sync::Arc; use std::time::Duration; use backon::ExponentialBuilder; -use bytes::{Buf, Bytes}; +use bytes::Buf; +use bytes::Bytes; use chrono::DateTime; use chrono::Utc; use http::header; diff --git a/core/src/services/dropbox/reader.rs b/core/src/services/dropbox/reader.rs index 89547506bcba..dd22b7ab0427 100644 --- a/core/src/services/dropbox/reader.rs +++ b/core/src/services/dropbox/reader.rs @@ -15,12 +15,13 @@ // specific language governing permissions and limitations // under the License. +use std::sync::Arc; + +use http::StatusCode; + use super::core::DropboxCore; use super::error::parse_error; use crate::raw::*; -use http::StatusCode; - -use std::sync::Arc; pub struct DropboxReader { core: Arc, diff --git a/core/src/services/fs/backend.rs b/core/src/services/fs/backend.rs index 92079d4c9bf5..9128f5824f5e 100644 --- a/core/src/services/fs/backend.rs +++ b/core/src/services/fs/backend.rs @@ -20,7 +20,6 @@ use std::path::Path; use std::path::PathBuf; use async_trait::async_trait; - use chrono::DateTime; use log::debug; use uuid::Uuid; diff --git a/core/src/services/fs/reader.rs b/core/src/services/fs/reader.rs index ad6ba8d416e4..af03e80e00d3 100644 --- a/core/src/services/fs/reader.rs +++ b/core/src/services/fs/reader.rs @@ -15,9 +15,10 @@ // specific language governing permissions and limitations // under the License. +use tokio::io::ReadBuf; + use crate::raw::*; use crate::*; -use tokio::io::ReadBuf; pub struct FsReader { f: std::fs::File, diff --git a/core/src/services/fs/writer.rs b/core/src/services/fs/writer.rs index c035ff1bf94b..7cec4bd27332 100644 --- a/core/src/services/fs/writer.rs +++ b/core/src/services/fs/writer.rs @@ -15,11 +15,10 @@ // specific language governing permissions and limitations // under the License. -use bytes::Bytes; - use std::io::Write; use std::path::PathBuf; +use bytes::Bytes; use tokio::io::AsyncWriteExt; use crate::raw::*; diff --git a/core/src/services/ftp/backend.rs b/core/src/services/ftp/backend.rs index 5a0a6e6ea30e..3f32e9f4f924 100644 --- a/core/src/services/ftp/backend.rs +++ b/core/src/services/ftp/backend.rs @@ -25,7 +25,6 @@ use async_tls::TlsConnector; use async_trait::async_trait; use bb8::PooledConnection; use bb8::RunError; - use http::Uri; use log::debug; use serde::Deserialize; diff --git a/core/src/services/ftp/lister.rs b/core/src/services/ftp/lister.rs index 3d531815a201..c3625a2eed0d 100644 --- a/core/src/services/ftp/lister.rs +++ b/core/src/services/ftp/lister.rs @@ -17,7 +17,6 @@ use std::str; use std::str::FromStr; - use std::vec::IntoIter; use suppaftp::list::File; diff --git a/core/src/services/ftp/reader.rs b/core/src/services/ftp/reader.rs index 983e2d55876f..f85f7074c4fd 100644 --- a/core/src/services/ftp/reader.rs +++ b/core/src/services/ftp/reader.rs @@ -15,11 +15,12 @@ // specific language governing permissions and limitations // under the License. +use futures::AsyncReadExt; + use super::backend::FtpBackend; use super::err::parse_error; use crate::raw::*; use crate::*; -use futures::AsyncReadExt; pub struct FtpReader { core: FtpBackend, diff --git a/core/src/services/ftp/writer.rs b/core/src/services/ftp/writer.rs index 505176a46d27..fa9f959e2b00 100644 --- a/core/src/services/ftp/writer.rs +++ b/core/src/services/ftp/writer.rs @@ -19,7 +19,6 @@ use bytes::Bytes; use futures::AsyncWriteExt; use super::backend::FtpBackend; - use crate::raw::*; use crate::services::ftp::err::parse_error; use crate::*; diff --git a/core/src/services/gcs/lister.rs b/core/src/services/gcs/lister.rs index 79ef5c98a4d6..17c98452edec 100644 --- a/core/src/services/gcs/lister.rs +++ b/core/src/services/gcs/lister.rs @@ -15,9 +15,9 @@ // specific language governing permissions and limitations // under the License. -use bytes::Buf; use std::sync::Arc; +use bytes::Buf; use serde_json; use super::core::*; diff --git a/core/src/services/gcs/reader.rs b/core/src/services/gcs/reader.rs index 3ec36f3ea289..bc10981cb2c7 100644 --- a/core/src/services/gcs/reader.rs +++ b/core/src/services/gcs/reader.rs @@ -15,12 +15,13 @@ // specific language governing permissions and limitations // under the License. +use std::sync::Arc; + +use http::StatusCode; + use super::core::GcsCore; use super::error::parse_error; use crate::raw::*; -use http::StatusCode; - -use std::sync::Arc; pub struct GcsReader { core: Arc, diff --git a/core/src/services/gdrive/backend.rs b/core/src/services/gdrive/backend.rs index 247683b862fb..c6233359f610 100644 --- a/core/src/services/gdrive/backend.rs +++ b/core/src/services/gdrive/backend.rs @@ -19,7 +19,8 @@ use std::fmt::Debug; use std::sync::Arc; use async_trait::async_trait; -use bytes::{Buf, Bytes}; +use bytes::Buf; +use bytes::Bytes; use chrono::Utc; use http::Request; use http::StatusCode; diff --git a/core/src/services/gdrive/core.rs b/core/src/services/gdrive/core.rs index 3df9e05f453e..8b062b6c3450 100644 --- a/core/src/services/gdrive/core.rs +++ b/core/src/services/gdrive/core.rs @@ -21,7 +21,8 @@ use std::sync::Arc; use async_trait::async_trait; use bytes; -use bytes::{Buf, Bytes}; +use bytes::Buf; +use bytes::Bytes; use chrono::DateTime; use chrono::Utc; use http::header; diff --git a/core/src/services/gdrive/reader.rs b/core/src/services/gdrive/reader.rs index dba874cd743a..267777753a84 100644 --- a/core/src/services/gdrive/reader.rs +++ b/core/src/services/gdrive/reader.rs @@ -15,12 +15,13 @@ // specific language governing permissions and limitations // under the License. +use std::sync::Arc; + +use http::StatusCode; + use super::core::GdriveCore; use super::error::parse_error; use crate::raw::*; -use http::StatusCode; - -use std::sync::Arc; pub struct GdriveReader { core: Arc, diff --git a/core/src/services/gdrive/writer.rs b/core/src/services/gdrive/writer.rs index ebd91afbdd67..9647b55b6dad 100644 --- a/core/src/services/gdrive/writer.rs +++ b/core/src/services/gdrive/writer.rs @@ -17,7 +17,8 @@ use std::sync::Arc; -use bytes::{Buf, Bytes}; +use bytes::Buf; +use bytes::Bytes; use http::StatusCode; use super::core::GdriveCore; diff --git a/core/src/services/ghac/backend.rs b/core/src/services/ghac/backend.rs index 057ec753299a..815133e475e1 100644 --- a/core/src/services/ghac/backend.rs +++ b/core/src/services/ghac/backend.rs @@ -19,7 +19,8 @@ use std::collections::HashMap; use std::env; use async_trait::async_trait; -use bytes::{Buf, Bytes}; +use bytes::Buf; +use bytes::Bytes; use http::header; use http::header::ACCEPT; use http::header::AUTHORIZATION; diff --git a/core/src/services/ghac/reader.rs b/core/src/services/ghac/reader.rs index 7cfcc54f5750..c6fcfea79aeb 100644 --- a/core/src/services/ghac/reader.rs +++ b/core/src/services/ghac/reader.rs @@ -15,10 +15,11 @@ // specific language governing permissions and limitations // under the License. +use http::StatusCode; + use super::error::parse_error; use crate::raw::*; use crate::services::ghac::backend::GhacBackend; -use http::StatusCode; pub struct GhacReader { core: GhacBackend, diff --git a/core/src/services/github/core.rs b/core/src/services/github/core.rs index a5419da9bb23..0c93c83aa5ed 100644 --- a/core/src/services/github/core.rs +++ b/core/src/services/github/core.rs @@ -19,7 +19,8 @@ use std::fmt::Debug; use std::fmt::Formatter; use base64::Engine; -use bytes::{Buf, Bytes}; +use bytes::Buf; +use bytes::Bytes; use http::header; use http::request; use http::Request; @@ -28,11 +29,10 @@ use http::StatusCode; use serde::Deserialize; use serde::Serialize; +use super::error::parse_error; use crate::raw::*; use crate::*; -use super::error::parse_error; - /// Core of [github contents](https://docs.github.com/en/rest/repos/contents?apiVersion=2022-11-28#create-or-update-file-contents) services support. #[derive(Clone)] pub struct GithubCore { diff --git a/core/src/services/github/reader.rs b/core/src/services/github/reader.rs index 055c296972ea..64fd0ded58a6 100644 --- a/core/src/services/github/reader.rs +++ b/core/src/services/github/reader.rs @@ -15,12 +15,13 @@ // specific language governing permissions and limitations // under the License. +use std::sync::Arc; + +use http::StatusCode; + use super::core::GithubCore; use super::error::parse_error; use crate::raw::*; -use http::StatusCode; - -use std::sync::Arc; pub struct GithubReader { core: Arc, diff --git a/core/src/services/hdfs/backend.rs b/core/src/services/hdfs/backend.rs index 2a55bb38f366..129b0c7e9200 100644 --- a/core/src/services/hdfs/backend.rs +++ b/core/src/services/hdfs/backend.rs @@ -23,7 +23,6 @@ use std::path::PathBuf; use std::sync::Arc; use async_trait::async_trait; - use futures::AsyncWriteExt; use log::debug; use serde::Deserialize; diff --git a/core/src/services/hdfs/reader.rs b/core/src/services/hdfs/reader.rs index c1e6392b91d2..63b17a05587c 100644 --- a/core/src/services/hdfs/reader.rs +++ b/core/src/services/hdfs/reader.rs @@ -15,13 +15,15 @@ // specific language governing permissions and limitations // under the License. -use crate::raw::*; -use crate::{Error, ErrorKind}; -use hdrs::File; - use std::sync::Arc; + +use hdrs::File; use tokio::io::ReadBuf; +use crate::raw::*; +use crate::Error; +use crate::ErrorKind; + pub struct HdfsReader { f: Arc, } diff --git a/core/src/services/hdfs/writer.rs b/core/src/services/hdfs/writer.rs index be1fc78e14d8..4fbc55bc84f2 100644 --- a/core/src/services/hdfs/writer.rs +++ b/core/src/services/hdfs/writer.rs @@ -16,11 +16,9 @@ // under the License. use std::io::Write; - use std::sync::Arc; use bytes::Bytes; - use futures::AsyncWriteExt; use crate::raw::*; diff --git a/core/src/services/hdfs_native/reader.rs b/core/src/services/hdfs_native/reader.rs index b6864b04b0fa..f54697236604 100644 --- a/core/src/services/hdfs_native/reader.rs +++ b/core/src/services/hdfs_native/reader.rs @@ -15,14 +15,10 @@ // specific language governing permissions and limitations // under the License. - - - use hdfs_native::file::FileReader; use crate::raw::oio; - pub struct HdfsNativeReader { _f: FileReader, } diff --git a/core/src/services/hdfs_native/writer.rs b/core/src/services/hdfs_native/writer.rs index 75ad0aeaf620..36648097cc99 100644 --- a/core/src/services/hdfs_native/writer.rs +++ b/core/src/services/hdfs_native/writer.rs @@ -16,11 +16,9 @@ // under the License. use bytes::Bytes; - use hdfs_native::file::FileWriter; use crate::raw::oio; - use crate::*; pub struct HdfsNativeWriter { diff --git a/core/src/services/http/reader.rs b/core/src/services/http/reader.rs index 1d0e8ae17926..d38dc4b2a2cf 100644 --- a/core/src/services/http/reader.rs +++ b/core/src/services/http/reader.rs @@ -15,10 +15,11 @@ // specific language governing permissions and limitations // under the License. +use http::StatusCode; + use super::error::parse_error; use crate::raw::*; use crate::services::http::backend::HttpBackend; -use http::StatusCode; pub struct HttpReader { core: HttpBackend, diff --git a/core/src/services/huggingface/error.rs b/core/src/services/huggingface/error.rs index 2a2e49e55bdd..c17cd2ee0d3f 100644 --- a/core/src/services/huggingface/error.rs +++ b/core/src/services/huggingface/error.rs @@ -15,9 +15,9 @@ // specific language governing permissions and limitations // under the License. -use bytes::Buf; use std::fmt::Debug; +use bytes::Buf; use http::Response; use http::StatusCode; use serde::Deserialize; diff --git a/core/src/services/huggingface/lister.rs b/core/src/services/huggingface/lister.rs index 9c7555e7967b..8c2189c8b0a6 100644 --- a/core/src/services/huggingface/lister.rs +++ b/core/src/services/huggingface/lister.rs @@ -15,9 +15,10 @@ // specific language governing permissions and limitations // under the License. -use bytes::Buf; use std::sync::Arc; +use bytes::Buf; + use super::core::HuggingfaceCore; use super::core::HuggingfaceStatus; use super::error::parse_error; diff --git a/core/src/services/huggingface/reader.rs b/core/src/services/huggingface/reader.rs index dcf98c0afcd9..d7caca047748 100644 --- a/core/src/services/huggingface/reader.rs +++ b/core/src/services/huggingface/reader.rs @@ -15,12 +15,13 @@ // specific language governing permissions and limitations // under the License. +use std::sync::Arc; + +use http::StatusCode; + use super::core::HuggingfaceCore; use super::error::parse_error; use crate::raw::*; -use http::StatusCode; - -use std::sync::Arc; pub struct HuggingfaceReader { core: Arc, diff --git a/core/src/services/icloud/backend.rs b/core/src/services/icloud/backend.rs index 7a2075335cc3..188147b76934 100644 --- a/core/src/services/icloud/backend.rs +++ b/core/src/services/icloud/backend.rs @@ -21,7 +21,6 @@ use std::fmt::Formatter; use std::sync::Arc; use async_trait::async_trait; - use serde::Deserialize; use tokio::sync::Mutex; diff --git a/core/src/services/icloud/core.rs b/core/src/services/icloud/core.rs index fe1008c0b55d..26c2aed890d9 100644 --- a/core/src/services/icloud/core.rs +++ b/core/src/services/icloud/core.rs @@ -719,9 +719,10 @@ pub struct IcloudCreateFolder { #[cfg(test)] mod tests { + use pretty_assertions::assert_eq; + use super::IcloudRoot; use super::IcloudWebservicesResponse; - use pretty_assertions::assert_eq; #[test] fn test_parse_icloud_drive_root_json() { diff --git a/core/src/services/icloud/reader.rs b/core/src/services/icloud/reader.rs index 8cbbdc832ad4..f001cb001a78 100644 --- a/core/src/services/icloud/reader.rs +++ b/core/src/services/icloud/reader.rs @@ -15,11 +15,13 @@ // specific language governing permissions and limitations // under the License. -use super::core::{parse_error, IcloudCore}; -use crate::raw::*; +use std::sync::Arc; + use http::StatusCode; -use std::sync::Arc; +use super::core::parse_error; +use super::core::IcloudCore; +use crate::raw::*; pub struct IcloudReader { core: Arc, diff --git a/core/src/services/ipfs/reader.rs b/core/src/services/ipfs/reader.rs index 1af91f145718..75b1cd1c197b 100644 --- a/core/src/services/ipfs/reader.rs +++ b/core/src/services/ipfs/reader.rs @@ -15,10 +15,11 @@ // specific language governing permissions and limitations // under the License. +use http::StatusCode; + use super::error::parse_error; use crate::raw::*; use crate::services::ipfs::backend::IpfsBackend; -use http::StatusCode; pub struct IpfsReader { core: IpfsBackend, diff --git a/core/src/services/ipmfs/backend.rs b/core/src/services/ipmfs/backend.rs index 9e8b7519efa6..1659806d1211 100644 --- a/core/src/services/ipmfs/backend.rs +++ b/core/src/services/ipmfs/backend.rs @@ -21,7 +21,8 @@ use std::str; use std::sync::Arc; use async_trait::async_trait; -use bytes::{Buf, Bytes}; +use bytes::Buf; +use bytes::Bytes; use http::Request; use http::Response; use http::StatusCode; diff --git a/core/src/services/ipmfs/lister.rs b/core/src/services/ipmfs/lister.rs index 5a66c7ba9cae..0ba252da6255 100644 --- a/core/src/services/ipmfs/lister.rs +++ b/core/src/services/ipmfs/lister.rs @@ -15,9 +15,9 @@ // specific language governing permissions and limitations // under the License. -use bytes::Buf; use std::sync::Arc; +use bytes::Buf; use http::StatusCode; use serde::Deserialize; diff --git a/core/src/services/ipmfs/reader.rs b/core/src/services/ipmfs/reader.rs index 5e09d4be67cb..3aaac81a3e7d 100644 --- a/core/src/services/ipmfs/reader.rs +++ b/core/src/services/ipmfs/reader.rs @@ -15,10 +15,11 @@ // specific language governing permissions and limitations // under the License. +use http::StatusCode; + use super::error::parse_error; use crate::raw::*; use crate::services::ipmfs::backend::IpmfsBackend; -use http::StatusCode; pub struct IpmfsReader { core: IpmfsBackend, diff --git a/core/src/services/ipmfs/writer.rs b/core/src/services/ipmfs/writer.rs index c8608d533569..f27fe5c77b3a 100644 --- a/core/src/services/ipmfs/writer.rs +++ b/core/src/services/ipmfs/writer.rs @@ -20,7 +20,6 @@ use http::StatusCode; use super::backend::IpmfsBackend; use super::error::parse_error; - use crate::raw::*; use crate::*; diff --git a/core/src/services/koofr/core.rs b/core/src/services/koofr/core.rs index bc836b3affd6..dc6372d2cae9 100644 --- a/core/src/services/koofr/core.rs +++ b/core/src/services/koofr/core.rs @@ -20,7 +20,8 @@ use std::fmt::Debug; use std::fmt::Formatter; use std::sync::Arc; -use bytes::{Buf, Bytes}; +use bytes::Buf; +use bytes::Bytes; use http::header; use http::request; use http::Request; diff --git a/core/src/services/koofr/lister.rs b/core/src/services/koofr/lister.rs index e2dcd9a997ac..fdb7487fe483 100644 --- a/core/src/services/koofr/lister.rs +++ b/core/src/services/koofr/lister.rs @@ -15,9 +15,10 @@ // specific language governing permissions and limitations // under the License. -use bytes::Buf; use std::sync::Arc; +use bytes::Buf; + use super::core::KoofrCore; use super::core::ListResponse; use super::error::parse_error; diff --git a/core/src/services/koofr/reader.rs b/core/src/services/koofr/reader.rs index 532fe2d9e03b..94c48ea93b20 100644 --- a/core/src/services/koofr/reader.rs +++ b/core/src/services/koofr/reader.rs @@ -15,12 +15,13 @@ // specific language governing permissions and limitations // under the License. +use std::sync::Arc; + +use http::StatusCode; + use super::core::KoofrCore; use super::error::parse_error; use crate::raw::*; -use http::StatusCode; - -use std::sync::Arc; pub struct KoofrReader { core: Arc, diff --git a/core/src/services/libsql/backend.rs b/core/src/services/libsql/backend.rs index e9f406ecb02a..2fbbf47606aa 100644 --- a/core/src/services/libsql/backend.rs +++ b/core/src/services/libsql/backend.rs @@ -20,7 +20,8 @@ use std::fmt::Debug; use std::str; use async_trait::async_trait; -use bytes::{Buf, Bytes}; +use bytes::Buf; +use bytes::Bytes; use hrana_client_proto::pipeline::ClientMsg; use hrana_client_proto::pipeline::Response; use hrana_client_proto::pipeline::ServerMsg; diff --git a/core/src/services/memory/backend.rs b/core/src/services/memory/backend.rs index 03c59ee297dc..47220e0db047 100644 --- a/core/src/services/memory/backend.rs +++ b/core/src/services/memory/backend.rs @@ -22,13 +22,11 @@ use std::sync::Arc; use std::sync::Mutex; use async_trait::async_trait; - -use crate::raw::adapters::typed_kv; -use crate::*; - use serde::Deserialize; use self::raw::ConfigDeserializer; +use crate::raw::adapters::typed_kv; +use crate::*; ///Config for memory. #[derive(Default, Deserialize)] diff --git a/core/src/services/obs/reader.rs b/core/src/services/obs/reader.rs index fa15062a061c..fb220e2eeb9d 100644 --- a/core/src/services/obs/reader.rs +++ b/core/src/services/obs/reader.rs @@ -15,12 +15,13 @@ // specific language governing permissions and limitations // under the License. +use std::sync::Arc; + +use http::StatusCode; + use super::core::ObsCore; use super::error::parse_error; use crate::raw::*; -use http::StatusCode; - -use std::sync::Arc; pub struct ObsReader { core: Arc, diff --git a/core/src/services/onedrive/backend.rs b/core/src/services/onedrive/backend.rs index 91bd4c111947..40f26967d044 100644 --- a/core/src/services/onedrive/backend.rs +++ b/core/src/services/onedrive/backend.rs @@ -18,7 +18,8 @@ use std::fmt::Debug; use async_trait::async_trait; -use bytes::{Buf, Bytes}; +use bytes::Buf; +use bytes::Bytes; use http::header; use http::Request; use http::Response; diff --git a/core/src/services/onedrive/lister.rs b/core/src/services/onedrive/lister.rs index f2bc24ddb0b8..591d44e9f1f3 100644 --- a/core/src/services/onedrive/lister.rs +++ b/core/src/services/onedrive/lister.rs @@ -15,6 +15,8 @@ // specific language governing permissions and limitations // under the License. +use bytes::Buf; + use super::backend::OnedriveBackend; use super::error::parse_error; use super::graph_model::GraphApiOnedriveListResponse; @@ -22,7 +24,6 @@ use super::graph_model::ItemType; use crate::raw::oio; use crate::raw::*; use crate::*; -use bytes::Buf; pub struct OnedriveLister { root: String, diff --git a/core/src/services/onedrive/reader.rs b/core/src/services/onedrive/reader.rs index 857919dc732d..355789936a7d 100644 --- a/core/src/services/onedrive/reader.rs +++ b/core/src/services/onedrive/reader.rs @@ -15,10 +15,11 @@ // specific language governing permissions and limitations // under the License. +use http::StatusCode; + use super::error::parse_error; use crate::raw::*; use crate::services::onedrive::backend::OnedriveBackend; -use http::StatusCode; pub struct OnedriveReader { core: OnedriveBackend, diff --git a/core/src/services/onedrive/writer.rs b/core/src/services/onedrive/writer.rs index 05d0250f042a..8efb65b5c849 100644 --- a/core/src/services/onedrive/writer.rs +++ b/core/src/services/onedrive/writer.rs @@ -15,7 +15,8 @@ // specific language governing permissions and limitations // under the License. -use bytes::{Buf, Bytes}; +use bytes::Buf; +use bytes::Bytes; use http::StatusCode; use super::backend::OnedriveBackend; diff --git a/core/src/services/oss/reader.rs b/core/src/services/oss/reader.rs index 366b44d474cc..4efb4f6dfe96 100644 --- a/core/src/services/oss/reader.rs +++ b/core/src/services/oss/reader.rs @@ -15,12 +15,13 @@ // specific language governing permissions and limitations // under the License. +use std::sync::Arc; + +use http::StatusCode; + use super::core::OssCore; use super::error::parse_error; use crate::raw::*; -use http::StatusCode; - -use std::sync::Arc; pub struct OssReader { core: Arc, diff --git a/core/src/services/pcloud/core.rs b/core/src/services/pcloud/core.rs index 4e84b3584372..9bdaaa5c6f45 100644 --- a/core/src/services/pcloud/core.rs +++ b/core/src/services/pcloud/core.rs @@ -18,10 +18,12 @@ use std::fmt::Debug; use std::fmt::Formatter; -use bytes::{Buf, Bytes}; +use bytes::Buf; +use bytes::Bytes; +use http::header; +use http::Request; use http::Response; use http::StatusCode; -use http::{header, Request}; use serde::Deserialize; use super::error::parse_error; diff --git a/core/src/services/pcloud/error.rs b/core/src/services/pcloud/error.rs index 59f0bd47ad80..2f840969101e 100644 --- a/core/src/services/pcloud/error.rs +++ b/core/src/services/pcloud/error.rs @@ -15,10 +15,10 @@ // specific language governing permissions and limitations // under the License. -use bytes::Buf; use std::fmt::Debug; use std::fmt::Formatter; +use bytes::Buf; use http::Response; use serde::Deserialize; diff --git a/core/src/services/pcloud/lister.rs b/core/src/services/pcloud/lister.rs index 8b3abd54620f..18d706529d07 100644 --- a/core/src/services/pcloud/lister.rs +++ b/core/src/services/pcloud/lister.rs @@ -15,9 +15,9 @@ // specific language governing permissions and limitations // under the License. -use bytes::Buf; use std::sync::Arc; +use bytes::Buf; use http::StatusCode; use super::core::*; diff --git a/core/src/services/pcloud/reader.rs b/core/src/services/pcloud/reader.rs index 93d2c864b847..317073a670c7 100644 --- a/core/src/services/pcloud/reader.rs +++ b/core/src/services/pcloud/reader.rs @@ -15,12 +15,13 @@ // specific language governing permissions and limitations // under the License. +use std::sync::Arc; + +use http::StatusCode; + use super::core::PcloudCore; use super::error::parse_error; use crate::raw::*; -use http::StatusCode; - -use std::sync::Arc; pub struct PcloudReader { core: Arc, diff --git a/core/src/services/pcloud/writer.rs b/core/src/services/pcloud/writer.rs index 3ba4074bd045..d4d3bd53cade 100644 --- a/core/src/services/pcloud/writer.rs +++ b/core/src/services/pcloud/writer.rs @@ -17,7 +17,8 @@ use std::sync::Arc; -use bytes::{Buf, Bytes}; +use bytes::Buf; +use bytes::Bytes; use http::StatusCode; use super::core::PcloudCore; diff --git a/core/src/services/s3/reader.rs b/core/src/services/s3/reader.rs index 61a267c3b795..91edf6f43d03 100644 --- a/core/src/services/s3/reader.rs +++ b/core/src/services/s3/reader.rs @@ -15,12 +15,14 @@ // specific language governing permissions and limitations // under the License. +use std::sync::Arc; + +use http::StatusCode; + use super::core::*; use super::error::*; use crate::raw::*; use crate::*; -use http::StatusCode; -use std::sync::Arc; pub struct S3Reader { core: Arc, diff --git a/core/src/services/s3/writer.rs b/core/src/services/s3/writer.rs index 619101447fe4..43ef3b37ba20 100644 --- a/core/src/services/s3/writer.rs +++ b/core/src/services/s3/writer.rs @@ -15,9 +15,9 @@ // specific language governing permissions and limitations // under the License. -use bytes::Buf; use std::sync::Arc; +use bytes::Buf; use http::StatusCode; use super::core::*; diff --git a/core/src/services/seafile/backend.rs b/core/src/services/seafile/backend.rs index 08a772d2b93d..18d2c9f84718 100644 --- a/core/src/services/seafile/backend.rs +++ b/core/src/services/seafile/backend.rs @@ -21,7 +21,6 @@ use std::fmt::Formatter; use std::sync::Arc; use async_trait::async_trait; - use log::debug; use serde::Deserialize; use tokio::sync::RwLock; @@ -29,7 +28,6 @@ use tokio::sync::RwLock; use super::core::parse_dir_detail; use super::core::parse_file_detail; use super::core::SeafileCore; - use super::lister::SeafileLister; use super::writer::SeafileWriter; use super::writer::SeafileWriters; diff --git a/core/src/services/seafile/core.rs b/core/src/services/seafile/core.rs index 3b9396b2807e..9640b5a3531f 100644 --- a/core/src/services/seafile/core.rs +++ b/core/src/services/seafile/core.rs @@ -19,7 +19,8 @@ use std::fmt::Debug; use std::fmt::Formatter; use std::sync::Arc; -use bytes::{Buf, Bytes}; +use bytes::Buf; +use bytes::Bytes; use http::header; use http::Request; use http::Response; diff --git a/core/src/services/seafile/lister.rs b/core/src/services/seafile/lister.rs index 25a0d2e837e9..c07e75d7c245 100644 --- a/core/src/services/seafile/lister.rs +++ b/core/src/services/seafile/lister.rs @@ -15,9 +15,9 @@ // specific language governing permissions and limitations // under the License. -use bytes::Buf; use std::sync::Arc; +use bytes::Buf; use http::header; use http::Request; use http::StatusCode; diff --git a/core/src/services/seafile/reader.rs b/core/src/services/seafile/reader.rs index ba9855969cd7..8db38dba88d5 100644 --- a/core/src/services/seafile/reader.rs +++ b/core/src/services/seafile/reader.rs @@ -15,12 +15,13 @@ // specific language governing permissions and limitations // under the License. +use std::sync::Arc; + +use http::StatusCode; + use super::core::SeafileCore; use super::error::parse_error; use crate::raw::*; -use http::StatusCode; - -use std::sync::Arc; pub struct SeafileReader { core: Arc, diff --git a/core/src/services/sftp/backend.rs b/core/src/services/sftp/backend.rs index f7c1b7500ef4..e967b93777fc 100644 --- a/core/src/services/sftp/backend.rs +++ b/core/src/services/sftp/backend.rs @@ -21,14 +21,12 @@ use std::fmt::Formatter; use std::path::Path; use std::path::PathBuf; - use async_trait::async_trait; use bytes::Bytes; use futures::StreamExt; use log::debug; use openssh::KnownHosts; use openssh::SessionBuilder; - use openssh_sftp_client::Sftp; use openssh_sftp_client::SftpOptions; use serde::Deserialize; diff --git a/core/src/services/supabase/reader.rs b/core/src/services/supabase/reader.rs index 2ea7896acaed..7554b57bd90c 100644 --- a/core/src/services/supabase/reader.rs +++ b/core/src/services/supabase/reader.rs @@ -15,12 +15,13 @@ // specific language governing permissions and limitations // under the License. +use std::sync::Arc; + +use http::StatusCode; + use super::core::SupabaseCore; use super::error::parse_error; use crate::raw::*; -use http::StatusCode; - -use std::sync::Arc; pub struct SupabaseReader { core: Arc, diff --git a/core/src/services/supabase/writer.rs b/core/src/services/supabase/writer.rs index 7268c150af78..38be282c3a66 100644 --- a/core/src/services/supabase/writer.rs +++ b/core/src/services/supabase/writer.rs @@ -22,7 +22,6 @@ use http::StatusCode; use super::core::*; use super::error::parse_error; - use crate::raw::*; use crate::*; diff --git a/core/src/services/swift/lister.rs b/core/src/services/swift/lister.rs index 3c68c12817b2..703780d71f79 100644 --- a/core/src/services/swift/lister.rs +++ b/core/src/services/swift/lister.rs @@ -15,9 +15,10 @@ // specific language governing permissions and limitations // under the License. -use bytes::Buf; use std::sync::Arc; +use bytes::Buf; + use super::core::*; use super::error::parse_error; use crate::raw::*; diff --git a/core/src/services/swift/reader.rs b/core/src/services/swift/reader.rs index b0dfbe046a4a..885655761e73 100644 --- a/core/src/services/swift/reader.rs +++ b/core/src/services/swift/reader.rs @@ -15,12 +15,13 @@ // specific language governing permissions and limitations // under the License. +use std::sync::Arc; + +use http::StatusCode; + use super::core::SwiftCore; use super::error::parse_error; use crate::raw::*; -use http::StatusCode; - -use std::sync::Arc; pub struct SwiftReader { core: Arc, diff --git a/core/src/services/swift/writer.rs b/core/src/services/swift/writer.rs index c63a4fbaa787..d4ab3ca297ef 100644 --- a/core/src/services/swift/writer.rs +++ b/core/src/services/swift/writer.rs @@ -22,7 +22,6 @@ use http::StatusCode; use super::core::SwiftCore; use super::error::parse_error; - use crate::raw::*; use crate::*; diff --git a/core/src/services/upyun/lister.rs b/core/src/services/upyun/lister.rs index b1dd13a4a1b2..9d6a756d2d49 100644 --- a/core/src/services/upyun/lister.rs +++ b/core/src/services/upyun/lister.rs @@ -15,9 +15,10 @@ // specific language governing permissions and limitations // under the License. -use bytes::Buf; use std::sync::Arc; +use bytes::Buf; + use super::core::ListObjectsResponse; use super::core::UpyunCore; use super::error::parse_error; diff --git a/core/src/services/upyun/reader.rs b/core/src/services/upyun/reader.rs index 3a04e2c5b151..e2c129cf4f75 100644 --- a/core/src/services/upyun/reader.rs +++ b/core/src/services/upyun/reader.rs @@ -15,12 +15,13 @@ // specific language governing permissions and limitations // under the License. +use std::sync::Arc; + +use http::StatusCode; + use super::core::UpyunCore; use super::error::parse_error; use crate::raw::*; -use http::StatusCode; - -use std::sync::Arc; pub struct UpyunReader { core: Arc, diff --git a/core/src/services/vercel_artifacts/reader.rs b/core/src/services/vercel_artifacts/reader.rs index 55d7e1562967..1e80edb23244 100644 --- a/core/src/services/vercel_artifacts/reader.rs +++ b/core/src/services/vercel_artifacts/reader.rs @@ -15,10 +15,11 @@ // specific language governing permissions and limitations // under the License. +use http::StatusCode; + use super::error::parse_error; use crate::raw::*; use crate::services::vercel_artifacts::backend::VercelArtifactsBackend; -use http::StatusCode; pub struct VercelArtifactsReader { core: VercelArtifactsBackend, diff --git a/core/src/services/vercel_blob/core.rs b/core/src/services/vercel_blob/core.rs index 45d97c5e9fbd..7ab84b11b6c0 100644 --- a/core/src/services/vercel_blob/core.rs +++ b/core/src/services/vercel_blob/core.rs @@ -18,7 +18,8 @@ use std::fmt::Debug; use std::fmt::Formatter; -use bytes::{Buf, Bytes}; +use bytes::Buf; +use bytes::Bytes; use http::header; use http::request; use http::Request; diff --git a/core/src/services/vercel_blob/reader.rs b/core/src/services/vercel_blob/reader.rs index f7e16e7b4dbe..b43f4ec66f59 100644 --- a/core/src/services/vercel_blob/reader.rs +++ b/core/src/services/vercel_blob/reader.rs @@ -15,12 +15,13 @@ // specific language governing permissions and limitations // under the License. +use std::sync::Arc; + +use http::StatusCode; + use super::core::VercelBlobCore; use super::error::parse_error; use crate::raw::*; -use http::StatusCode; - -use std::sync::Arc; pub struct VercelBlobReader { core: Arc, diff --git a/core/src/services/vercel_blob/writer.rs b/core/src/services/vercel_blob/writer.rs index dbe9f5e57d7b..23938ed8a5f8 100644 --- a/core/src/services/vercel_blob/writer.rs +++ b/core/src/services/vercel_blob/writer.rs @@ -15,9 +15,9 @@ // specific language governing permissions and limitations // under the License. -use bytes::Buf; use std::sync::Arc; +use bytes::Buf; use http::StatusCode; use super::core::InitiateMultipartUploadResponse; diff --git a/core/src/services/webdav/reader.rs b/core/src/services/webdav/reader.rs index 8176ae239b47..b7d6426b7d7d 100644 --- a/core/src/services/webdav/reader.rs +++ b/core/src/services/webdav/reader.rs @@ -15,12 +15,13 @@ // specific language governing permissions and limitations // under the License. +use std::sync::Arc; + +use http::StatusCode; + use super::core::WebdavCore; use super::error::parse_error; use crate::raw::*; -use http::StatusCode; - -use std::sync::Arc; pub struct WebdavReader { core: Arc, diff --git a/core/src/services/webdav/writer.rs b/core/src/services/webdav/writer.rs index d51fd053a6f2..f4755f54fa64 100644 --- a/core/src/services/webdav/writer.rs +++ b/core/src/services/webdav/writer.rs @@ -22,7 +22,6 @@ use http::StatusCode; use super::core::*; use super::error::parse_error; - use crate::raw::*; use crate::*; diff --git a/core/src/services/webhdfs/backend.rs b/core/src/services/webhdfs/backend.rs index cbd02a522eaf..6803b1bca1c2 100644 --- a/core/src/services/webhdfs/backend.rs +++ b/core/src/services/webhdfs/backend.rs @@ -30,7 +30,6 @@ use serde::Deserialize; use tokio::sync::OnceCell; use super::error::parse_error; - use super::lister::WebhdfsLister; use super::message::BooleanResp; use super::message::FileStatusType; diff --git a/core/src/services/webhdfs/reader.rs b/core/src/services/webhdfs/reader.rs index 20f120142945..76786398f878 100644 --- a/core/src/services/webhdfs/reader.rs +++ b/core/src/services/webhdfs/reader.rs @@ -15,12 +15,14 @@ // specific language governing permissions and limitations // under the License. -use super::error::{parse_error, parse_error_msg}; -use crate::raw::*; -use crate::services::webhdfs::backend::WebhdfsBackend; use bytes::Buf; use http::StatusCode; +use super::error::parse_error; +use super::error::parse_error_msg; +use crate::raw::*; +use crate::services::webhdfs::backend::WebhdfsBackend; + pub struct WebhdfsReader { core: WebhdfsBackend, diff --git a/core/src/services/yandex_disk/backend.rs b/core/src/services/yandex_disk/backend.rs index 58ceb9485aef..90630284396c 100644 --- a/core/src/services/yandex_disk/backend.rs +++ b/core/src/services/yandex_disk/backend.rs @@ -22,7 +22,6 @@ use std::sync::Arc; use async_trait::async_trait; use bytes::Buf; - use http::StatusCode; use log::debug; use serde::Deserialize; diff --git a/core/src/services/yandex_disk/core.rs b/core/src/services/yandex_disk/core.rs index 28fba494a3ec..7c76f4d47f2a 100644 --- a/core/src/services/yandex_disk/core.rs +++ b/core/src/services/yandex_disk/core.rs @@ -15,10 +15,10 @@ // specific language governing permissions and limitations // under the License. -use bytes::Buf; use std::fmt::Debug; use std::fmt::Formatter; +use bytes::Buf; use http::header; use http::request; use http::Request; diff --git a/core/src/services/yandex_disk/lister.rs b/core/src/services/yandex_disk/lister.rs index d7c947f1a352..4bdd8034cf54 100644 --- a/core/src/services/yandex_disk/lister.rs +++ b/core/src/services/yandex_disk/lister.rs @@ -15,9 +15,10 @@ // specific language governing permissions and limitations // under the License. -use bytes::Buf; use std::sync::Arc; +use bytes::Buf; + use super::core::parse_info; use super::core::MetainformationResponse; use super::core::YandexDiskCore; diff --git a/core/src/services/yandex_disk/reader.rs b/core/src/services/yandex_disk/reader.rs index 4dafffcd0ecf..ec9f3ffd437e 100644 --- a/core/src/services/yandex_disk/reader.rs +++ b/core/src/services/yandex_disk/reader.rs @@ -15,12 +15,15 @@ // specific language governing permissions and limitations // under the License. +use std::sync::Arc; + +use http::header; +use http::Request; +use http::StatusCode; + use super::core::YandexDiskCore; use super::error::parse_error; use crate::raw::*; -use http::{header, Request, StatusCode}; - -use std::sync::Arc; pub struct YandexDiskReader { core: Arc, diff --git a/core/src/types/blocking_reader.rs b/core/src/types/blocking_reader.rs index 62468f50ce9d..65cd4957fcc4 100644 --- a/core/src/types/blocking_reader.rs +++ b/core/src/types/blocking_reader.rs @@ -16,10 +16,10 @@ // under the License. use std::collections::Bound; - use std::ops::RangeBounds; -use bytes::{Buf, BufMut}; +use bytes::Buf; +use bytes::BufMut; use crate::raw::oio::BlockingRead; use crate::raw::*; diff --git a/core/src/types/operator/blocking_operator.rs b/core/src/types/operator/blocking_operator.rs index 6fe3ba0862c3..c410bd817391 100644 --- a/core/src/types/operator/blocking_operator.rs +++ b/core/src/types/operator/blocking_operator.rs @@ -15,7 +15,8 @@ // specific language governing permissions and limitations // under the License. -use bytes::{Buf, Bytes}; +use bytes::Buf; +use bytes::Bytes; use super::operator_functions::*; use crate::raw::*; diff --git a/core/src/types/reader.rs b/core/src/types/reader.rs index 9c3f6e7404f5..6696adb33cb8 100644 --- a/core/src/types/reader.rs +++ b/core/src/types/reader.rs @@ -15,9 +15,12 @@ // specific language governing permissions and limitations // under the License. -use std::ops::{Bound, Range, RangeBounds}; +use std::ops::Bound; +use std::ops::Range; +use std::ops::RangeBounds; -use bytes::{Buf, BufMut}; +use bytes::Buf; +use bytes::BufMut; use crate::raw::*; use crate::*; @@ -156,15 +159,21 @@ impl Reader { } mod impl_futures_async_read { - use crate::raw::*; - use crate::*; - use bytes::Buf; - use futures::{AsyncBufRead, AsyncRead, AsyncSeek}; use std::io; use std::io::SeekFrom; use std::ops::Range; use std::pin::Pin; - use std::task::{ready, Context, Poll}; + use std::task::ready; + use std::task::Context; + use std::task::Poll; + + use bytes::Buf; + use futures::AsyncBufRead; + use futures::AsyncRead; + use futures::AsyncSeek; + + use crate::raw::*; + use crate::*; pub struct FuturesReader { state: State, @@ -289,14 +298,19 @@ mod impl_futures_async_read { } mod impl_futures_stream { - use crate::raw::*; - use crate::*; - use bytes::{Buf, Bytes}; - use futures::Stream; use std::io; use std::ops::Range; use std::pin::Pin; - use std::task::{ready, Context, Poll}; + use std::task::ready; + use std::task::Context; + use std::task::Poll; + + use bytes::Buf; + use bytes::Bytes; + use futures::Stream; + + use crate::raw::*; + use crate::*; pub struct FuturesStream { state: State, diff --git a/core/src/types/writer.rs b/core/src/types/writer.rs index 9a235c0a0de6..64c32667c025 100644 --- a/core/src/types/writer.rs +++ b/core/src/types/writer.rs @@ -126,12 +126,11 @@ impl Writer { /// # Examples /// /// ```no_run - /// use opendal::Result; - /// /// use bytes::Bytes; /// use futures::stream; /// use futures::StreamExt; /// use opendal::Operator; + /// use opendal::Result; /// /// async fn sink_example(op: Operator) -> Result<()> { /// let mut w = op.writer_with("path/to/file").await?; @@ -176,13 +175,12 @@ impl Writer { /// # Examples /// /// ```no_run - /// use opendal::Result; - /// /// use bytes::Bytes; /// use futures::io::Cursor; /// use futures::stream; /// use futures::StreamExt; /// use opendal::Operator; + /// use opendal::Result; /// /// async fn copy_example(op: Operator) -> Result<()> { /// let mut w = op.writer_with("path/to/file").await?; From 298007730122216c957bb90e660736b9b719a032 Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Thu, 21 Mar 2024 20:22:30 +0800 Subject: [PATCH 063/111] Fix license Signed-off-by: Xuanwo --- core/src/services/koofr/backend.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/services/koofr/backend.rs b/core/src/services/koofr/backend.rs index 91e08a1e7507..859be7dadef9 100644 --- a/core/src/services/koofr/backend.rs +++ b/core/src/services/koofr/backend.rs @@ -12,7 +12,7 @@ // 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 limi tations +// specific language governing permissions and limitations // under the License. use std::collections::HashMap; From 73076d8059ff552fe4b785e526314c0f84645680 Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Thu, 21 Mar 2024 20:28:55 +0800 Subject: [PATCH 064/111] Remove complete Signed-off-by: Xuanwo --- core/src/layers/complete.rs | 96 ------------------------------------- 1 file changed, 96 deletions(-) diff --git a/core/src/layers/complete.rs b/core/src/layers/complete.rs index 17f59e811614..d614612ca9cd 100644 --- a/core/src/layers/complete.rs +++ b/core/src/layers/complete.rs @@ -274,92 +274,6 @@ impl CompleteAccessor { }) } - // async fn complete_read( - // &self, - // path: &str, - // args: OpRead, - // ) -> Result<(RpRead, CompleteReader)> { - // let capability = self.meta.native_capability(); - // if !capability.read { - // return Err(self.new_unsupported_error(Operation::Read)); - // } - // - // let seekable = capability.read_can_seek; - // let streamable = capability.read_can_next; - // let buffer_cap = args.buffer(); - // - // let r = match (seekable, streamable) { - // (true, true) => { - // let r = LazyReader::new(self.inner.clone(), path, args); - // InnerCompleteReader::One(r) - // } - // (true, false) => { - // let r = FileReader::new(self.inner.clone(), path, args); - // InnerCompleteReader::Two(r) - // } - // _ => { - // let r = RangeReader::new(self.inner.clone(), path, args); - // - // if streamable { - // InnerCompleteReader::Three(r) - // } else { - // let r = oio::into_streamable_read(r, 256 * 1024); - // InnerCompleteReader::Four(r) - // } - // } - // }; - // - // let r = match buffer_cap { - // None => CompleteReader::One(r), - // Some(cap) => CompleteReader::Two(BufferReader::new(r, cap)), - // }; - // - // Ok((RpRead::new(), r)) - // } - - // fn complete_blocking_read( - // &self, - // path: &str, - // args: OpRead, - // ) -> Result<(RpRead, CompleteReader)> { - // let capability = self.meta.full_capability(); - // if !capability.read || !capability.blocking { - // return Err(self.new_unsupported_error(Operation::BlockingRead)); - // } - // - // let seekable = capability.read_can_seek; - // let streamable = capability.read_can_next; - // let buffer_cap = args.buffer(); - // - // let r = match (seekable, streamable) { - // (true, true) => { - // let r = LazyReader::new(self.inner.clone(), path, args); - // InnerCompleteReader::One(r) - // } - // (true, false) => { - // let r = FileReader::new(self.inner.clone(), path, args); - // InnerCompleteReader::Two(r) - // } - // _ => { - // let r = RangeReader::new(self.inner.clone(), path, args); - // - // if streamable { - // InnerCompleteReader::Three(r) - // } else { - // let r = oio::into_streamable_read(r, 256 * 1024); - // InnerCompleteReader::Four(r) - // } - // } - // }; - // - // let r = match buffer_cap { - // None => CompleteReader::One(r), - // Some(cap) => CompleteReader::Two(BufferReader::new(r, cap)), - // }; - // - // Ok((RpRead::new(), r)) - // } - async fn complete_list( &self, path: &str, @@ -667,16 +581,6 @@ impl LayeredAccessor for CompleteAccessor { } } -// pub type CompleteReader = -// TwoWays, BufferReader>>; -// -// type InnerCompleteReader = FourWays< -// LazyReader, -// FileReader, -// RangeReader, -// StreamableReader>, -// >; - pub type CompleteLister = FourWays, P>, PrefixLister

, PrefixLister, P>>>; From 0d42758bc3bd565101fb074100d284012117e9d3 Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Thu, 21 Mar 2024 20:43:55 +0800 Subject: [PATCH 065/111] Remove range in OpRead Signed-off-by: Xuanwo --- core/src/layers/error_context.rs | 3 -- core/src/layers/logging.rs | 20 ++++------- core/src/raw/adapters/kv/backend.rs | 6 ++-- core/src/raw/adapters/typed_kv/backend.rs | 6 ++-- core/src/raw/ops.rs | 13 ------- core/src/services/b2/backend.rs | 6 ++-- core/src/services/b2/core.rs | 5 --- core/src/services/dbfs/backend.rs | 4 +-- core/src/services/dbfs/reader.rs | 10 +----- core/src/services/gcs/core.rs | 3 -- core/src/types/operator/blocking_operator.rs | 9 +++-- core/src/types/operator/operator.rs | 12 +++---- core/src/types/operator/operator_functions.rs | 10 ++---- core/src/types/operator/operator_futures.rs | 34 ++++--------------- 14 files changed, 35 insertions(+), 106 deletions(-) diff --git a/core/src/layers/error_context.rs b/core/src/layers/error_context.rs index 2db5b5b5ea39..f57383c91296 100644 --- a/core/src/layers/error_context.rs +++ b/core/src/layers/error_context.rs @@ -91,8 +91,6 @@ impl LayeredAccessor for ErrorContextAccessor { } async fn read(&self, path: &str, args: OpRead) -> Result<(RpRead, Self::Reader)> { - let br = args.range(); - self.inner .read(path, args) .map_ok(|(rp, r)| { @@ -109,7 +107,6 @@ impl LayeredAccessor for ErrorContextAccessor { err.with_operation(Operation::Read) .with_context("service", self.meta.scheme()) .with_context("path", path) - .with_context("range", br.to_string()) }) .await } diff --git a/core/src/layers/logging.rs b/core/src/layers/logging.rs index b74dc5f80c52..d846315dcda2 100644 --- a/core/src/layers/logging.rs +++ b/core/src/layers/logging.rs @@ -285,26 +285,22 @@ impl LayeredAccessor for LoggingAccessor { async fn read(&self, path: &str, args: OpRead) -> Result<(RpRead, Self::Reader)> { debug!( target: LOGGING_TARGET, - "service={} operation={} path={} range={} -> started", + "service={} operation={} path={} -> started", self.ctx.scheme, Operation::Read, path, - args.range() ); - let range = args.range(); - self.inner .read(path, args) .await .map(|(rp, r)| { debug!( target: LOGGING_TARGET, - "service={} operation={} path={} range={} -> got reader", + "service={} operation={} path={} -> got reader", self.ctx.scheme, Operation::Read, path, - range ); ( rp, @@ -316,11 +312,10 @@ impl LayeredAccessor for LoggingAccessor { log!( target: LOGGING_TARGET, lvl, - "service={} operation={} path={} range={} -> {}", + "service={} operation={} path={} -> {}", self.ctx.scheme, Operation::Read, path, - range, self.ctx.error_print(&err) ) } @@ -684,11 +679,10 @@ impl LayeredAccessor for LoggingAccessor { fn blocking_read(&self, path: &str, args: OpRead) -> Result<(RpRead, Self::BlockingReader)> { debug!( target: LOGGING_TARGET, - "service={} operation={} path={} range={} -> started", + "service={} operation={} path={} -> started", self.ctx.scheme, Operation::BlockingRead, path, - args.range(), ); self.inner @@ -696,11 +690,10 @@ impl LayeredAccessor for LoggingAccessor { .map(|(rp, r)| { debug!( target: LOGGING_TARGET, - "service={} operation={} path={} range={} -> got reader", + "service={} operation={} path={} -> got reader", self.ctx.scheme, Operation::BlockingRead, path, - args.range(), ); let r = LoggingReader::new(self.ctx.clone(), Operation::BlockingRead, path, r); (rp, r) @@ -710,11 +703,10 @@ impl LayeredAccessor for LoggingAccessor { log!( target: LOGGING_TARGET, lvl, - "service={} operation={} path={} range={} -> {}", + "service={} operation={} path={} -> {}", self.ctx.scheme, Operation::BlockingRead, path, - args.range(), self.ctx.error_print(&err) ); } diff --git a/core/src/raw/adapters/kv/backend.rs b/core/src/raw/adapters/kv/backend.rs index cea7c982c7fa..fbcf5f6bd1ff 100644 --- a/core/src/raw/adapters/kv/backend.rs +++ b/core/src/raw/adapters/kv/backend.rs @@ -95,7 +95,7 @@ impl Accessor for Backend { am } - async fn read(&self, path: &str, args: OpRead) -> Result<(RpRead, Self::Reader)> { + async fn read(&self, path: &str, _: OpRead) -> Result<(RpRead, Self::Reader)> { let p = build_abs_path(&self.root, path); let bs = match self.kv.get(&p).await? { @@ -104,12 +104,11 @@ impl Accessor for Backend { }; let bs = Bytes::from(bs); - let bs = args.range().apply_on_bytes(bs); Ok((RpRead::new(), bs)) } - fn blocking_read(&self, path: &str, args: OpRead) -> Result<(RpRead, Self::BlockingReader)> { + fn blocking_read(&self, path: &str, _: OpRead) -> Result<(RpRead, Self::BlockingReader)> { let p = build_abs_path(&self.root, path); let bs = match self.kv.blocking_get(&p)? { @@ -118,7 +117,6 @@ impl Accessor for Backend { }; let bs = Bytes::from(bs); - let bs = args.range().apply_on_bytes(bs); Ok((RpRead::new(), bs)) } diff --git a/core/src/raw/adapters/typed_kv/backend.rs b/core/src/raw/adapters/typed_kv/backend.rs index edc48285df67..5f9f44f14f8f 100644 --- a/core/src/raw/adapters/typed_kv/backend.rs +++ b/core/src/raw/adapters/typed_kv/backend.rs @@ -101,7 +101,7 @@ impl Accessor for Backend { am } - async fn read(&self, path: &str, args: OpRead) -> Result<(RpRead, Self::Reader)> { + async fn read(&self, path: &str, _: OpRead) -> Result<(RpRead, Self::Reader)> { let p = build_abs_path(&self.root, path); let bs = match self.kv.get(&p).await? { @@ -110,11 +110,10 @@ impl Accessor for Backend { None => return Err(Error::new(ErrorKind::NotFound, "kv doesn't have this path")), }; - let bs = args.range().apply_on_bytes(bs); Ok((RpRead::new(), bs)) } - fn blocking_read(&self, path: &str, args: OpRead) -> Result<(RpRead, Self::BlockingReader)> { + fn blocking_read(&self, path: &str, _: OpRead) -> Result<(RpRead, Self::BlockingReader)> { let p = build_abs_path(&self.root, path); let bs = match self.kv.blocking_get(&p)? { @@ -123,7 +122,6 @@ impl Accessor for Backend { None => return Err(Error::new(ErrorKind::NotFound, "kv doesn't have this path")), }; - let bs = args.range().apply_on_bytes(bs); Ok((RpRead::new(), bs)) } diff --git a/core/src/raw/ops.rs b/core/src/raw/ops.rs index 32788aa4db55..3bbef0eb26a7 100644 --- a/core/src/raw/ops.rs +++ b/core/src/raw/ops.rs @@ -300,8 +300,6 @@ impl BatchOperation { /// Args for `read` operation. #[derive(Debug, Clone, Default)] pub struct OpRead { - /// NOTE: range is not apply for reader. - range: BytesRange, if_match: Option, if_none_match: Option, override_content_type: Option, @@ -316,17 +314,6 @@ impl OpRead { Self::default() } - /// Create a new OpRead with range. - pub fn with_range(mut self, range: BytesRange) -> Self { - self.range = range; - self - } - - /// Get range from OpRead. - pub fn range(&self) -> BytesRange { - self.range - } - /// Sets the content-disposition header that should be send back by the remote read operation. pub fn with_override_content_disposition(mut self, content_disposition: &str) -> Self { self.override_content_disposition = Some(content_disposition.into()); diff --git a/core/src/services/b2/backend.rs b/core/src/services/b2/backend.rs index f76849795f20..93739027856b 100644 --- a/core/src/services/b2/backend.rs +++ b/core/src/services/b2/backend.rs @@ -446,7 +446,7 @@ impl Accessor for B2Backend { PresignOperation::Stat(_) => { let resp = self .core - .get_download_authorization(path, &OpRead::default(), args.expire()) + .get_download_authorization(path, args.expire()) .await?; let path = build_abs_path(&self.core.root, path); @@ -472,10 +472,10 @@ impl Accessor for B2Backend { parts.headers, ))) } - PresignOperation::Read(op) => { + PresignOperation::Read(_) => { let resp = self .core - .get_download_authorization(path, op, args.expire()) + .get_download_authorization(path, args.expire()) .await?; let path = build_abs_path(&self.core.root, path); diff --git a/core/src/services/b2/core.rs b/core/src/services/b2/core.rs index 461d1a7b2fc6..8d01338f3151 100644 --- a/core/src/services/b2/core.rs +++ b/core/src/services/b2/core.rs @@ -194,7 +194,6 @@ impl B2Core { pub async fn get_download_authorization( &self, path: &str, - args: &OpRead, expire: Duration, ) -> Result { let path = build_abs_path(&self.root, path); @@ -210,10 +209,6 @@ impl B2Core { req = req.header(header::AUTHORIZATION, auth_info.authorization_token); - let range = args.range(); - if !range.is_full() { - req = req.header(header::RANGE, range.to_header()); - } let body = GetDownloadAuthorizationRequest { bucket_id: self.bucket_id.clone(), file_name_prefix: path, diff --git a/core/src/services/dbfs/backend.rs b/core/src/services/dbfs/backend.rs index 2ee42040bab2..14ebf398a910 100644 --- a/core/src/services/dbfs/backend.rs +++ b/core/src/services/dbfs/backend.rs @@ -244,8 +244,8 @@ impl Accessor for DbfsBackend { } } - async fn read(&self, path: &str, args: OpRead) -> Result<(RpRead, Self::Reader)> { - let op = DbfsReader::new(self.core.clone(), args, path.to_string()); + async fn read(&self, path: &str, _: OpRead) -> Result<(RpRead, Self::Reader)> { + let op = DbfsReader::new(self.core.clone(), path.to_string()); Ok((RpRead::new(), op)) } diff --git a/core/src/services/dbfs/reader.rs b/core/src/services/dbfs/reader.rs index 1c5858855855..ce267d297b17 100644 --- a/core/src/services/dbfs/reader.rs +++ b/core/src/services/dbfs/reader.rs @@ -35,26 +35,18 @@ use crate::*; pub struct DbfsReader { core: Arc, path: String, - offset: u64, has_filled: u64, } impl DbfsReader { - pub fn new(core: Arc, op: OpRead, path: String) -> Self { + pub fn new(core: Arc, path: String) -> Self { DbfsReader { core, path, - offset: op.range().offset(), has_filled: 0, } } - #[inline] - #[allow(dead_code)] - fn set_offset(&mut self, offset: u64) { - self.offset = offset; - } - #[allow(dead_code)] fn serde_json_decode(&self, bs: &Bytes) -> Result { let response_body = match serde_json::from_slice::(bs) { diff --git a/core/src/services/gcs/core.rs b/core/src/services/gcs/core.rs index 8935c29cf0e8..bbed7951b6e9 100644 --- a/core/src/services/gcs/core.rs +++ b/core/src/services/gcs/core.rs @@ -199,9 +199,6 @@ impl GcsCore { if let Some(if_none_match) = args.if_none_match() { req = req.header(IF_NONE_MATCH, if_none_match); } - if !args.range().is_full() { - req = req.header(http::header::RANGE, args.range().to_header()); - } let req = req .body(AsyncBody::Empty) diff --git a/core/src/types/operator/blocking_operator.rs b/core/src/types/operator/blocking_operator.rs index c410bd817391..36100de0322c 100644 --- a/core/src/types/operator/blocking_operator.rs +++ b/core/src/types/operator/blocking_operator.rs @@ -385,8 +385,8 @@ impl BlockingOperator { FunctionRead(OperatorFunction::new( self.inner().clone(), path, - OpRead::default(), - |inner, path, args| { + (OpRead::default(), BytesRange::default()), + |inner, path, (args, range)| { if !validate_path(&path, EntryMode::FILE) { return Err( Error::new(ErrorKind::IsADirectory, "read path is a directory") @@ -396,12 +396,11 @@ impl BlockingOperator { ); } - let size_hint = args.range().size(); + let size_hint = range.size(); let r = BlockingReader::create(inner, &path, args)?; let mut buf = Vec::with_capacity(size_hint.unwrap_or_default() as _); - r.read_to_end(&mut buf)?; - + r.read_range(&mut buf, range.to_range())?; Ok(buf) }, )) diff --git a/core/src/types/operator/operator.rs b/core/src/types/operator/operator.rs index 70ab664b6092..4f25e4a102bf 100644 --- a/core/src/types/operator/operator.rs +++ b/core/src/types/operator/operator.rs @@ -500,8 +500,8 @@ impl Operator { OperatorFuture::new( self.inner().clone(), path, - OpRead::default(), - |inner, path, args| async move { + (OpRead::default(), BytesRange::default()), + |inner, path, (args, range)| async move { if !validate_path(&path, EntryMode::FILE) { return Err( Error::new(ErrorKind::IsADirectory, "read path is a directory") @@ -511,11 +511,11 @@ impl Operator { ); } - let size_hint = args.range().size(); + let size_hint = range.size(); let r = Reader::create(inner, &path, args).await?; let mut buf = Vec::with_capacity(size_hint.unwrap_or_default() as _); - r.read_to_end(&mut buf).await?; + r.read_range(&mut buf, range.to_range()).await?; Ok(buf) }, ) @@ -574,7 +574,7 @@ impl Operator { OperatorFuture::new( self.inner().clone(), path, - (OpRead::default(), ()), + OpRead::default(), |inner, path, args| async move { if !validate_path(&path, EntryMode::FILE) { return Err( @@ -585,7 +585,7 @@ impl Operator { ); } - Reader::create(inner.clone(), &path, args.0).await + Reader::create(inner.clone(), &path, args).await }, ) } diff --git a/core/src/types/operator/operator_functions.rs b/core/src/types/operator/operator_functions.rs index 4afa157f02e8..1e40f78d41b0 100644 --- a/core/src/types/operator/operator_functions.rs +++ b/core/src/types/operator/operator_functions.rs @@ -320,12 +320,12 @@ impl FunctionLister { /// Function that generated by [`BlockingOperator::read_with`]. /// /// Users can add more options by public functions provided by this struct. -pub struct FunctionRead(pub(crate) OperatorFunction>); +pub struct FunctionRead(pub(crate) OperatorFunction<(OpRead, BytesRange), Vec>); impl FunctionRead { /// Set the range for this operation. pub fn range(mut self, range: impl RangeBounds) -> Self { - self.0 = self.0.map_args(|args| args.with_range(range.into())); + self.0 = self.0.map_args(|(args, _)| (args, range.into())); self } @@ -342,12 +342,6 @@ impl FunctionRead { pub struct FunctionReader(pub(crate) OperatorFunction); impl FunctionReader { - /// Set the range for this operation. - pub fn range(mut self, range: impl RangeBounds) -> Self { - self.0 = self.0.map_args(|args| args.with_range(range.into())); - self - } - /// Sets the content-disposition header that should be send back by the remote read operation. pub fn override_content_disposition(mut self, content_disposition: &str) -> Self { self.0 = self diff --git a/core/src/types/operator/operator_futures.rs b/core/src/types/operator/operator_futures.rs index d2c3940330df..c4ddd86e74b9 100644 --- a/core/src/types/operator/operator_futures.rs +++ b/core/src/types/operator/operator_futures.rs @@ -149,11 +149,6 @@ impl FuturePresignStat { pub type FuturePresignRead = OperatorFuture<(OpRead, Duration), F>; impl FuturePresignRead { - /// Create a new OpRead with range. - pub fn range(self, v: BytesRange) -> Self { - self.map(|(args, dur)| (args.with_range(v), dur)) - } - /// Sets the content-disposition header that should be send back by the remote read operation. pub fn override_content_disposition(self, v: &str) -> Self { self.map(|(args, dur)| (args.with_override_content_disposition(v), dur)) @@ -205,42 +200,27 @@ impl FuturePresignWrite { /// Future that generated by [`Operator::read_with`] or [`Operator::reader_with`]. /// /// Users can add more options by public functions provided by this struct. -pub type FutureRead = OperatorFuture; +pub type FutureRead = OperatorFuture<(OpRead, BytesRange), F>; impl FutureRead { /// Set the range header for this operation. pub fn range(self, range: impl RangeBounds) -> Self { - self.map(|args| args.with_range(range.into())) + self.map(|(args, _)| (args, range.into())) } /// Set the If-Match for this operation. pub fn if_match(self, v: &str) -> Self { - self.map(|args| args.with_if_match(v)) + self.map(|(args, range)| (args.with_if_match(v), range)) } /// Set the If-None-Match for this operation. pub fn if_none_match(self, v: &str) -> Self { - self.map(|args| args.with_if_none_match(v)) - } - - /// Sets the content-disposition header that should be send back by the remote read operation. - pub fn override_content_disposition(self, v: &str) -> Self { - self.map(|args| args.with_override_content_disposition(v)) - } - - /// Sets the cache-control header that should be send back by the remote read operation. - pub fn override_cache_control(self, v: &str) -> Self { - self.map(|args| args.with_override_cache_control(v)) - } - - /// Sets the content-type header that should be send back by the remote read operation. - pub fn override_content_type(self, v: &str) -> Self { - self.map(|args| args.with_override_content_type(v)) + self.map(|(args, range)| (args.with_if_none_match(v), range)) } /// Set the version for this operation. pub fn version(self, v: &str) -> Self { - self.map(|args| args.with_version(v)) + self.map(|(args, range)| (args.with_version(v), range)) } } @@ -251,12 +231,12 @@ impl FutureRead { /// # Notes /// /// `(OpRead, ())` is a trick to make sure `FutureReader` is different from `FutureRead` -pub type FutureReader = OperatorFuture<(OpRead, ()), F>; +pub type FutureReader = OperatorFuture; impl FutureReader { /// Set the version for this operation. pub fn version(self, v: &str) -> Self { - self.map(|(args, ())| (args.with_version(v), ())) + self.map(|args| args.with_version(v)) } } From 76a9aae4f33249dd98ed869aba49ceba7ef0f4f3 Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Fri, 22 Mar 2024 11:50:32 +0800 Subject: [PATCH 066/111] Remove not needed read can seek Signed-off-by: Xuanwo --- core/fuzz/fuzz_reader.rs | 6 ------ core/src/layers/complete.rs | 4 ---- core/src/layers/immutable_index.rs | 1 + core/src/raw/adapters/kv/backend.rs | 3 --- core/src/raw/adapters/typed_kv/backend.rs | 3 --- core/src/services/azblob/backend.rs | 3 +-- core/src/services/azdls/backend.rs | 2 -- core/src/services/azfile/backend.rs | 2 -- core/src/services/b2/backend.rs | 2 -- core/src/services/cos/backend.rs | 3 +-- core/src/services/dbfs/backend.rs | 2 -- core/src/services/dropbox/backend.rs | 1 - core/src/services/fs/backend.rs | 1 - core/src/services/ftp/backend.rs | 1 - core/src/services/gcs/backend.rs | 3 +-- core/src/services/ghac/backend.rs | 2 -- core/src/services/hdfs/backend.rs | 1 - core/src/services/http/backend.rs | 3 +-- core/src/services/huggingface/backend.rs | 2 -- core/src/services/ipfs/backend.rs | 2 -- core/src/services/ipmfs/backend.rs | 2 -- core/src/services/obs/backend.rs | 3 +-- core/src/services/oss/backend.rs | 3 +-- core/src/services/pcloud/backend.rs | 1 - core/src/services/s3/backend.rs | 3 +-- core/src/services/seafile/backend.rs | 1 - core/src/services/sftp/backend.rs | 1 - core/src/services/swift/backend.rs | 2 -- core/src/services/upyun/backend.rs | 1 - core/src/services/vercel_artifacts/backend.rs | 1 - core/src/services/vercel_blob/backend.rs | 2 -- core/src/services/webdav/backend.rs | 2 -- core/src/services/webhdfs/backend.rs | 2 -- core/src/types/capability.rs | 6 ------ core/src/types/list.rs | 1 + core/tests/behavior/async_read.rs | 8 -------- core/tests/behavior/blocking_read.rs | 8 -------- 37 files changed, 9 insertions(+), 85 deletions(-) diff --git a/core/fuzz/fuzz_reader.rs b/core/fuzz/fuzz_reader.rs index fb026862c081..3c91d01e8aaf 100644 --- a/core/fuzz/fuzz_reader.rs +++ b/core/fuzz/fuzz_reader.rs @@ -29,7 +29,6 @@ use opendal::raw::tests::ReadChecker; use opendal::raw::tests::TEST_RUNTIME; use opendal::Operator; use opendal::Result; -use tracing::warn; const MAX_DATA_SIZE: usize = 16 * 1024 * 1024; @@ -98,11 +97,6 @@ fuzz_target!(|input: FuzzInput| { let op = init_test_service().expect("operator init must succeed"); if let Some(op) = op { - if !op.info().full_capability().read_with_range { - warn!("service doesn't support read with range, skip fuzzing"); - return; - } - TEST_RUNTIME.block_on(async { fuzz_reader(op, input.clone()) .await diff --git a/core/src/layers/complete.rs b/core/src/layers/complete.rs index d614612ca9cd..71032cd2d8f0 100644 --- a/core/src/layers/complete.rs +++ b/core/src/layers/complete.rs @@ -390,10 +390,6 @@ impl LayeredAccessor for CompleteAccessor { fn metadata(&self) -> AccessorInfo { let mut meta = self.meta.clone(); let cap = meta.full_capability_mut(); - if cap.read { - cap.read_can_next = true; - cap.read_can_seek = true; - } if cap.list && cap.write_can_empty { cap.create_dir = true; } diff --git a/core/src/layers/immutable_index.rs b/core/src/layers/immutable_index.rs index cf7458cd6b4d..1b64e820ca3b 100644 --- a/core/src/layers/immutable_index.rs +++ b/core/src/layers/immutable_index.rs @@ -243,6 +243,7 @@ impl oio::BlockingList for ImmutableDir { } #[cfg(test)] +#[cfg(feature = "services-http")] mod tests { use std::collections::HashMap; use std::collections::HashSet; diff --git a/core/src/raw/adapters/kv/backend.rs b/core/src/raw/adapters/kv/backend.rs index fbcf5f6bd1ff..72ebf9ff68f2 100644 --- a/core/src/raw/adapters/kv/backend.rs +++ b/core/src/raw/adapters/kv/backend.rs @@ -75,9 +75,6 @@ impl Accessor for Backend { let mut cap = am.native_capability(); if cap.read { - cap.read_can_seek = true; - cap.read_can_next = true; - cap.read_with_range = true; cap.stat = true; } diff --git a/core/src/raw/adapters/typed_kv/backend.rs b/core/src/raw/adapters/typed_kv/backend.rs index 5f9f44f14f8f..a6910ead2c01 100644 --- a/core/src/raw/adapters/typed_kv/backend.rs +++ b/core/src/raw/adapters/typed_kv/backend.rs @@ -74,9 +74,6 @@ impl Accessor for Backend { let mut cap = Capability::default(); if kv_cap.get { cap.read = true; - cap.read_can_seek = true; - cap.read_can_next = true; - cap.read_with_range = true; cap.stat = true; } diff --git a/core/src/services/azblob/backend.rs b/core/src/services/azblob/backend.rs index b8fe029f2541..6367ba5135c2 100644 --- a/core/src/services/azblob/backend.rs +++ b/core/src/services/azblob/backend.rs @@ -563,8 +563,7 @@ impl Accessor for AzblobBackend { stat_with_if_none_match: true, read: true, - read_can_next: true, - read_with_range: true, + read_with_if_match: true, read_with_if_none_match: true, read_with_override_content_disposition: true, diff --git a/core/src/services/azdls/backend.rs b/core/src/services/azdls/backend.rs index 70390dafa555..e88a64dbcb11 100644 --- a/core/src/services/azdls/backend.rs +++ b/core/src/services/azdls/backend.rs @@ -262,8 +262,6 @@ impl Accessor for AzdlsBackend { stat: true, read: true, - read_can_next: true, - read_with_range: true, write: true, write_can_append: true, diff --git a/core/src/services/azfile/backend.rs b/core/src/services/azfile/backend.rs index 4ed4dfe35fce..128d22e40d6f 100644 --- a/core/src/services/azfile/backend.rs +++ b/core/src/services/azfile/backend.rs @@ -282,8 +282,6 @@ impl Accessor for AzfileBackend { stat: true, read: true, - read_can_next: true, - read_with_range: true, write: true, create_dir: true, diff --git a/core/src/services/b2/backend.rs b/core/src/services/b2/backend.rs index 93739027856b..4eecace5ac43 100644 --- a/core/src/services/b2/backend.rs +++ b/core/src/services/b2/backend.rs @@ -284,8 +284,6 @@ impl Accessor for B2Backend { stat: true, read: true, - read_can_next: true, - read_with_range: true, write: true, write_can_empty: true, diff --git a/core/src/services/cos/backend.rs b/core/src/services/cos/backend.rs index a86709fb9ac4..59ff0771c7d9 100644 --- a/core/src/services/cos/backend.rs +++ b/core/src/services/cos/backend.rs @@ -273,8 +273,7 @@ impl Accessor for CosBackend { stat_with_if_none_match: true, read: true, - read_can_next: true, - read_with_range: true, + read_with_if_match: true, read_with_if_none_match: true, diff --git a/core/src/services/dbfs/backend.rs b/core/src/services/dbfs/backend.rs index 14ebf398a910..7df87d371213 100644 --- a/core/src/services/dbfs/backend.rs +++ b/core/src/services/dbfs/backend.rs @@ -183,8 +183,6 @@ impl Accessor for DbfsBackend { stat: true, read: true, - read_can_next: true, - read_with_range: true, write: true, create_dir: true, diff --git a/core/src/services/dropbox/backend.rs b/core/src/services/dropbox/backend.rs index fc27c5b210cc..73c04bb861e5 100644 --- a/core/src/services/dropbox/backend.rs +++ b/core/src/services/dropbox/backend.rs @@ -52,7 +52,6 @@ impl Accessor for DropboxBackend { stat: true, read: true, - read_with_range: true, write: true, diff --git a/core/src/services/fs/backend.rs b/core/src/services/fs/backend.rs index 9128f5824f5e..7870892e69e8 100644 --- a/core/src/services/fs/backend.rs +++ b/core/src/services/fs/backend.rs @@ -256,7 +256,6 @@ impl Accessor for FsBackend { stat: true, read: true, - read_can_seek: true, write: true, write_can_empty: true, diff --git a/core/src/services/ftp/backend.rs b/core/src/services/ftp/backend.rs index 3f32e9f4f924..962fd8cd8e80 100644 --- a/core/src/services/ftp/backend.rs +++ b/core/src/services/ftp/backend.rs @@ -299,7 +299,6 @@ impl Accessor for FtpBackend { stat: true, read: true, - read_with_range: true, write: true, diff --git a/core/src/services/gcs/backend.rs b/core/src/services/gcs/backend.rs index 2d584f222316..6ec2f28b1c57 100644 --- a/core/src/services/gcs/backend.rs +++ b/core/src/services/gcs/backend.rs @@ -352,8 +352,7 @@ impl Accessor for GcsBackend { stat_with_if_none_match: true, read: true, - read_can_next: true, - read_with_range: true, + read_with_if_match: true, read_with_if_none_match: true, diff --git a/core/src/services/ghac/backend.rs b/core/src/services/ghac/backend.rs index 815133e475e1..76b1349319fd 100644 --- a/core/src/services/ghac/backend.rs +++ b/core/src/services/ghac/backend.rs @@ -245,8 +245,6 @@ impl Accessor for GhacBackend { stat: true, read: true, - read_can_next: true, - read_with_range: true, write: true, write_can_multi: true, diff --git a/core/src/services/hdfs/backend.rs b/core/src/services/hdfs/backend.rs index 129b0c7e9200..b58f3aba3a56 100644 --- a/core/src/services/hdfs/backend.rs +++ b/core/src/services/hdfs/backend.rs @@ -261,7 +261,6 @@ impl Accessor for HdfsBackend { stat: true, read: true, - read_can_seek: true, write: true, write_can_append: self.enable_append, diff --git a/core/src/services/http/backend.rs b/core/src/services/http/backend.rs index 60e80cef448c..78c6ceddab3e 100644 --- a/core/src/services/http/backend.rs +++ b/core/src/services/http/backend.rs @@ -241,8 +241,7 @@ impl Accessor for HttpBackend { stat_with_if_none_match: true, read: true, - read_can_next: true, - read_with_range: true, + read_with_if_match: true, read_with_if_none_match: true, diff --git a/core/src/services/huggingface/backend.rs b/core/src/services/huggingface/backend.rs index 671f6e5755af..380438ceeca0 100644 --- a/core/src/services/huggingface/backend.rs +++ b/core/src/services/huggingface/backend.rs @@ -259,8 +259,6 @@ impl Accessor for HuggingfaceBackend { stat: true, read: true, - read_can_next: true, - read_with_range: true, list: true, list_with_recursive: true, diff --git a/core/src/services/ipfs/backend.rs b/core/src/services/ipfs/backend.rs index 88e89fda1c88..d489832174b7 100644 --- a/core/src/services/ipfs/backend.rs +++ b/core/src/services/ipfs/backend.rs @@ -177,8 +177,6 @@ impl Accessor for IpfsBackend { stat: true, read: true, - read_can_next: true, - read_with_range: true, list: true, diff --git a/core/src/services/ipmfs/backend.rs b/core/src/services/ipmfs/backend.rs index 1659806d1211..7a5674d8d31c 100644 --- a/core/src/services/ipmfs/backend.rs +++ b/core/src/services/ipmfs/backend.rs @@ -80,8 +80,6 @@ impl Accessor for IpmfsBackend { stat: true, read: true, - read_can_next: true, - read_with_range: true, write: true, delete: true, diff --git a/core/src/services/obs/backend.rs b/core/src/services/obs/backend.rs index 7f4ff2633628..e3700eccbc81 100644 --- a/core/src/services/obs/backend.rs +++ b/core/src/services/obs/backend.rs @@ -267,8 +267,7 @@ impl Accessor for ObsBackend { stat_with_if_none_match: true, read: true, - read_can_next: true, - read_with_range: true, + read_with_if_match: true, read_with_if_none_match: true, diff --git a/core/src/services/oss/backend.rs b/core/src/services/oss/backend.rs index 58cf0d600d59..843b8ebc1051 100644 --- a/core/src/services/oss/backend.rs +++ b/core/src/services/oss/backend.rs @@ -395,8 +395,7 @@ impl Accessor for OssBackend { stat_with_if_none_match: true, read: true, - read_can_next: true, - read_with_range: true, + read_with_if_match: true, read_with_if_none_match: true, diff --git a/core/src/services/pcloud/backend.rs b/core/src/services/pcloud/backend.rs index 5d5dc6ccc8e3..aa6fade91fdc 100644 --- a/core/src/services/pcloud/backend.rs +++ b/core/src/services/pcloud/backend.rs @@ -247,7 +247,6 @@ impl Accessor for PcloudBackend { create_dir: true, read: true, - read_can_next: true, write: true, diff --git a/core/src/services/s3/backend.rs b/core/src/services/s3/backend.rs index 05cf4edeec12..cb58459abb8f 100644 --- a/core/src/services/s3/backend.rs +++ b/core/src/services/s3/backend.rs @@ -1006,8 +1006,7 @@ impl Accessor for S3Backend { stat_with_override_content_type: !self.core.disable_stat_with_override, read: true, - read_can_next: true, - read_with_range: true, + read_with_if_match: true, read_with_if_none_match: true, read_with_override_cache_control: true, diff --git a/core/src/services/seafile/backend.rs b/core/src/services/seafile/backend.rs index 18d2c9f84718..3529e2e50fd7 100644 --- a/core/src/services/seafile/backend.rs +++ b/core/src/services/seafile/backend.rs @@ -270,7 +270,6 @@ impl Accessor for SeafileBackend { stat: true, read: true, - read_can_next: true, write: true, write_can_empty: true, diff --git a/core/src/services/sftp/backend.rs b/core/src/services/sftp/backend.rs index e967b93777fc..936382dd9be7 100644 --- a/core/src/services/sftp/backend.rs +++ b/core/src/services/sftp/backend.rs @@ -256,7 +256,6 @@ impl Accessor for SftpBackend { stat: true, read: true, - read_can_seek: true, write: true, write_can_multi: true, diff --git a/core/src/services/swift/backend.rs b/core/src/services/swift/backend.rs index 0e1e5f27d20d..c3325bee39f2 100644 --- a/core/src/services/swift/backend.rs +++ b/core/src/services/swift/backend.rs @@ -206,8 +206,6 @@ impl Accessor for SwiftBackend { stat: true, read: true, - read_can_next: true, - read_with_range: true, write: true, write_can_empty: true, diff --git a/core/src/services/upyun/backend.rs b/core/src/services/upyun/backend.rs index c5f974f721f2..718f04d1dd68 100644 --- a/core/src/services/upyun/backend.rs +++ b/core/src/services/upyun/backend.rs @@ -251,7 +251,6 @@ impl Accessor for UpyunBackend { create_dir: true, read: true, - read_can_next: true, write: true, write_can_empty: true, diff --git a/core/src/services/vercel_artifacts/backend.rs b/core/src/services/vercel_artifacts/backend.rs index 7bc9c22beddb..b2e633df283d 100644 --- a/core/src/services/vercel_artifacts/backend.rs +++ b/core/src/services/vercel_artifacts/backend.rs @@ -60,7 +60,6 @@ impl Accessor for VercelArtifactsBackend { stat: true, read: true, - read_can_next: true, write: true, diff --git a/core/src/services/vercel_blob/backend.rs b/core/src/services/vercel_blob/backend.rs index 7e0c56dd9e49..d1bc66618894 100644 --- a/core/src/services/vercel_blob/backend.rs +++ b/core/src/services/vercel_blob/backend.rs @@ -195,8 +195,6 @@ impl Accessor for VercelBlobBackend { stat: true, read: true, - read_can_next: true, - read_with_range: true, write: true, write_can_empty: true, diff --git a/core/src/services/webdav/backend.rs b/core/src/services/webdav/backend.rs index 329660e2bcab..998a478d540f 100644 --- a/core/src/services/webdav/backend.rs +++ b/core/src/services/webdav/backend.rs @@ -253,8 +253,6 @@ impl Accessor for WebdavBackend { stat: true, read: true, - read_can_next: true, - read_with_range: true, write: true, write_can_empty: true, diff --git a/core/src/services/webhdfs/backend.rs b/core/src/services/webhdfs/backend.rs index 6803b1bca1c2..a9d3dd7c0d50 100644 --- a/core/src/services/webhdfs/backend.rs +++ b/core/src/services/webhdfs/backend.rs @@ -556,8 +556,6 @@ impl Accessor for WebhdfsBackend { stat: true, read: true, - read_can_next: true, - read_with_range: true, write: true, write_can_append: true, diff --git a/core/src/types/capability.rs b/core/src/types/capability.rs index ca0936b09521..45d53682a350 100644 --- a/core/src/types/capability.rs +++ b/core/src/types/capability.rs @@ -70,12 +70,6 @@ pub struct Capability { /// If operator supports read. pub read: bool, - /// If operator supports seek on returning reader. - pub read_can_seek: bool, - /// If operator supports next on returning reader. - pub read_can_next: bool, - /// If operator supports read with range. - pub read_with_range: bool, /// If operator supports read with if match. pub read_with_if_match: bool, /// If operator supports read with if none match. diff --git a/core/src/types/list.rs b/core/src/types/list.rs index 141a13bd70fd..d02f576a81ca 100644 --- a/core/src/types/list.rs +++ b/core/src/types/list.rs @@ -283,6 +283,7 @@ impl Iterator for BlockingLister { } #[cfg(test)] +#[cfg(feature = "services-azblob")] mod tests { use futures::future; use futures::StreamExt; diff --git a/core/tests/behavior/async_read.rs b/core/tests/behavior/async_read.rs index f8b75885d2ab..0ba7db507c42 100644 --- a/core/tests/behavior/async_read.rs +++ b/core/tests/behavior/async_read.rs @@ -81,10 +81,6 @@ pub async fn test_read_full(op: Operator) -> anyhow::Result<()> { /// Read range content should match. pub async fn test_read_range(op: Operator) -> anyhow::Result<()> { - if !op.info().full_capability().read_with_range { - return Ok(()); - } - let (path, content, size) = TEST_FIXTURE.new_file(op.clone()); let (offset, length) = gen_offset_length(size); @@ -108,10 +104,6 @@ pub async fn test_read_range(op: Operator) -> anyhow::Result<()> { /// Read large range content should match. pub async fn test_read_large_range(op: Operator) -> anyhow::Result<()> { - if !op.info().full_capability().read_with_range { - return Ok(()); - } - let (path, content, size) = TEST_FIXTURE.new_file(op.clone()); let (offset, _) = gen_offset_length(size); diff --git a/core/tests/behavior/blocking_read.rs b/core/tests/behavior/blocking_read.rs index 4b5de2d7e361..006d3349d9d0 100644 --- a/core/tests/behavior/blocking_read.rs +++ b/core/tests/behavior/blocking_read.rs @@ -71,10 +71,6 @@ pub fn test_blocking_read_full(op: BlockingOperator) -> Result<()> { /// Read range content should match. pub fn test_blocking_read_range(op: BlockingOperator) -> Result<()> { - if !op.info().full_capability().read_with_range { - return Ok(()); - } - let path = uuid::Uuid::new_v4().to_string(); debug!("Generate a random file: {}", &path); let (content, size) = gen_bytes(op.info().full_capability()); @@ -100,10 +96,6 @@ pub fn test_blocking_read_range(op: BlockingOperator) -> Result<()> { /// Read large range content should match. pub fn test_blocking_read_large_range(op: BlockingOperator) -> Result<()> { - if !op.info().full_capability().read_with_range { - return Ok(()); - } - let path = uuid::Uuid::new_v4().to_string(); debug!("Generate a random file: {}", &path); let (content, size) = gen_bytes(op.info().full_capability()); From d60f467edd451320660221894693801243a6f592 Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Fri, 22 Mar 2024 11:59:48 +0800 Subject: [PATCH 067/111] Fix tests Signed-off-by: Xuanwo --- core/Cargo.lock | 97 ------------- core/Cargo.toml | 1 - core/src/layers/complete.rs | 10 +- core/src/raw/http_util/bytes_range.rs | 41 ------ core/src/services/http/backend.rs | 196 -------------------------- 5 files changed, 8 insertions(+), 337 deletions(-) diff --git a/core/Cargo.lock b/core/Cargo.lock index 38f3ac047f06..87ec30fe1942 100644 --- a/core/Cargo.lock +++ b/core/Cargo.lock @@ -163,16 +163,6 @@ version = "0.7.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "96d30a06541fbafbc7f82ed10c06164cfbd2c401138f6addd8404629c4b16711" -[[package]] -name = "assert-json-diff" -version = "2.0.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "47e4f2b81832e72834d7518d8487a0396a28cc408186a2e8854c0f98011faf12" -dependencies = [ - "serde", - "serde_json", -] - [[package]] name = "async-backtrace" version = "0.2.6" @@ -1848,25 +1838,6 @@ version = "2.5.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "7e962a19be5cfc3f3bf6dd8f61eb50107f356ad6270fbb3ed41476571db78be5" -[[package]] -name = "deadpool" -version = "0.9.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "421fe0f90f2ab22016f32a9881be5134fdd71c65298917084b0c7477cbc3856e" -dependencies = [ - "async-trait", - "deadpool-runtime", - "num_cpus", - "retain_mut", - "tokio", -] - -[[package]] -name = "deadpool-runtime" -version = "0.1.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "63dfa964fe2a66f3fde91fc70b267fe193d822c7e603e2a675a49a7f46ad3f49" - [[package]] name = "der" version = "0.6.1" @@ -2887,27 +2858,6 @@ dependencies = [ "pin-project-lite", ] -[[package]] -name = "http-types" -version = "2.12.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6e9b187a72d63adbfba487f48095306ac823049cb504ee195541e91c7775f5ad" -dependencies = [ - "anyhow", - "async-channel 1.9.0", - "base64 0.13.1", - "futures-lite 1.13.0", - "http 0.2.11", - "infer", - "pin-project-lite", - "rand 0.7.3", - "serde", - "serde_json", - "serde_qs", - "serde_urlencoded", - "url", -] - [[package]] name = "httparse" version = "1.8.0" @@ -3048,12 +2998,6 @@ version = "4.6.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "c40411d0e5c63ef1323c3d09ce5ec6d84d71531e18daed0743fccea279d7deb6" -[[package]] -name = "infer" -version = "0.2.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "64e9829a50b42bb782c1df523f78d332fe371b10c661e78b7a3c34b0198e9fac" - [[package]] name = "inout" version = "0.1.3" @@ -4096,7 +4040,6 @@ dependencies = [ "tracing", "tracing-subscriber", "uuid", - "wiremock", ] [[package]] @@ -5371,12 +5314,6 @@ dependencies = [ "quick-error", ] -[[package]] -name = "retain_mut" -version = "0.1.9" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4389f1d5789befaf6029ebd9f7dac4af7f7e3d61b69d4f30e2ac02b57e7712b0" - [[package]] name = "rfc6979" version = "0.3.1" @@ -5864,17 +5801,6 @@ dependencies = [ "serde", ] -[[package]] -name = "serde_qs" -version = "0.8.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c7715380eec75f029a4ef7de39a9200e0a63823176b759d055b613f5a87df6a6" -dependencies = [ - "percent-encoding", - "serde", - "thiserror", -] - [[package]] name = "serde_spanned" version = "0.6.5" @@ -6974,7 +6900,6 @@ dependencies = [ "form_urlencoded", "idna 0.5.0", "percent-encoding", - "serde", ] [[package]] @@ -7553,28 +7478,6 @@ dependencies = [ "windows-sys 0.48.0", ] -[[package]] -name = "wiremock" -version = "0.5.22" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "13a3a53eaf34f390dd30d7b1b078287dd05df2aa2e21a589ccb80f5c7253c2e9" -dependencies = [ - "assert-json-diff", - "async-trait", - "base64 0.21.7", - "deadpool", - "futures", - "futures-timer", - "http-types", - "hyper", - "log", - "once_cell", - "regex", - "serde", - "serde_json", - "tokio", -] - [[package]] name = "wyz" version = "0.5.1" diff --git a/core/Cargo.toml b/core/Cargo.toml index 5b84d27ad45c..ee1219eab763 100644 --- a/core/Cargo.toml +++ b/core/Cargo.toml @@ -366,4 +366,3 @@ tracing-subscriber = { version = "0.3", features = [ "env-filter", "tracing-log", ] } -wiremock = "0.5" diff --git a/core/src/layers/complete.rs b/core/src/layers/complete.rs index 71032cd2d8f0..a48327d0450d 100644 --- a/core/src/layers/complete.rs +++ b/core/src/layers/complete.rs @@ -401,7 +401,10 @@ impl LayeredAccessor for CompleteAccessor { } async fn read(&self, path: &str, args: OpRead) -> Result<(RpRead, Self::Reader)> { - // self.complete_read(path, args).await + let capability = self.meta.full_capability(); + if !capability.read { + return Err(self.new_unsupported_error(Operation::Read)); + } self.inner.read(path, args).await } @@ -511,7 +514,10 @@ impl LayeredAccessor for CompleteAccessor { } fn blocking_read(&self, path: &str, args: OpRead) -> Result<(RpRead, Self::BlockingReader)> { - // self.complete_blocking_read(path, args) + let capability = self.meta.full_capability(); + if !capability.read || !capability.blocking { + return Err(self.new_unsupported_error(Operation::Read)); + } self.inner.blocking_read(path, args) } diff --git a/core/src/raw/http_util/bytes_range.rs b/core/src/raw/http_util/bytes_range.rs index d112ebaf6990..a5c7284c2a9e 100644 --- a/core/src/raw/http_util/bytes_range.rs +++ b/core/src/raw/http_util/bytes_range.rs @@ -22,8 +22,6 @@ use std::ops::Bound; use std::ops::RangeBounds; use std::str::FromStr; -use bytes::Bytes; - use crate::Error; use crate::ErrorKind; use crate::Result; @@ -102,17 +100,6 @@ impl BytesRange { }, ) } - - /// apply_on_bytes will apply range on bytes. - pub fn apply_on_bytes(&self, mut bs: Bytes) -> Bytes { - let mut bs = bs.split_off(self.0 as usize); - if let Some(size) = self.1 { - if (size as usize) < bs.len() { - let _ = bs.split_off(size as usize); - } - } - bs - } } impl Display for BytesRange { @@ -256,32 +243,4 @@ mod tests { Ok(()) } - - #[test] - fn test_apply_on_bytes() -> Result<()> { - let bs = Bytes::from_static("Hello, World!".as_bytes()); - - let cases = vec![ - ("full", (0, None), "Hello, World!"), - ("with_offset", (1, None), "ello, World!"), - ("with_size", (0, Some(1)), "!"), - ("with_larger_size", (0, Some(100)), "Hello, World!"), - ("with_offset_and_size", (1, Some(1)), "e"), - ( - "with_offset_and_larger_size", - (1, Some(100)), - "ello, World!", - ), - ("with_empty_offset", (0, Some(100)), "Hello, World!"), - ]; - - for (name, input, expected) in cases { - let actual = BytesRange(input.0, input.1).apply_on_bytes(bs.clone()); - let actual = String::from_utf8_lossy(&actual); - - assert_eq!(expected, &actual, "{name}"); - } - - Ok(()) - } } diff --git a/core/src/services/http/backend.rs b/core/src/services/http/backend.rs index 78c6ceddab3e..cd8c0c6e4050 100644 --- a/core/src/services/http/backend.rs +++ b/core/src/services/http/backend.rs @@ -339,199 +339,3 @@ impl HttpBackend { self.client.send(req).await } } - -#[cfg(test)] -mod tests { - use anyhow::Result; - use wiremock::matchers::basic_auth; - use wiremock::matchers::bearer_token; - use wiremock::matchers::headers; - use wiremock::matchers::method; - use wiremock::matchers::path; - use wiremock::Mock; - use wiremock::MockServer; - use wiremock::ResponseTemplate; - - use super::*; - use crate::Operator; - - #[tokio::test] - async fn test_read() -> Result<()> { - let _ = tracing_subscriber::fmt().with_test_writer().try_init(); - - let mock_server = MockServer::start().await; - Mock::given(method("GET")) - .and(path("/hello")) - .respond_with( - ResponseTemplate::new(200) - .insert_header("content-length", "13") - .set_body_string("Hello, World!"), - ) - .mount(&mock_server) - .await; - Mock::given(method("HEAD")) - .and(path("/hello")) - .respond_with(ResponseTemplate::new(200).insert_header("content-length", "13")) - .mount(&mock_server) - .await; - - let mut builder = HttpBuilder::default(); - builder.endpoint(&mock_server.uri()); - builder.root("/"); - let op = Operator::new(builder)?.finish(); - - let bs = op.read("hello").await?; - - assert_eq!(bs, b"Hello, World!"); - Ok(()) - } - - #[tokio::test] - async fn test_read_via_basic_auth() -> Result<()> { - let _ = tracing_subscriber::fmt().with_test_writer().try_init(); - - let (username, password) = ("your_username", "your_password"); - - let mock_server = MockServer::start().await; - Mock::given(method("GET")) - .and(path("/hello")) - .and(basic_auth(username, password)) - .respond_with( - ResponseTemplate::new(200) - .insert_header("content-length", "13") - .set_body_string("Hello, World!"), - ) - .mount(&mock_server) - .await; - Mock::given(method("HEAD")) - .and(path("/hello")) - .and(basic_auth(username, password)) - .respond_with(ResponseTemplate::new(200).insert_header("content-length", "13")) - .mount(&mock_server) - .await; - - let mut builder = HttpBuilder::default(); - builder.endpoint(&mock_server.uri()); - builder.root("/"); - builder.username(username).password(password); - let op = Operator::new(builder)?.finish(); - - let bs = op.read("hello").await?; - - assert_eq!(bs, b"Hello, World!"); - Ok(()) - } - - #[tokio::test] - async fn test_read_via_bearer_auth() -> Result<()> { - let _ = tracing_subscriber::fmt().with_test_writer().try_init(); - - let token = "your_token"; - - let mock_server = MockServer::start().await; - Mock::given(method("GET")) - .and(path("/hello")) - .and(bearer_token(token)) - .respond_with( - ResponseTemplate::new(200) - .insert_header("content-length", "13") - .set_body_string("Hello, World!"), - ) - .mount(&mock_server) - .await; - Mock::given(method("HEAD")) - .and(path("/hello")) - .and(bearer_token(token)) - .respond_with(ResponseTemplate::new(200).insert_header("content-length", "13")) - .mount(&mock_server) - .await; - - let mut builder = HttpBuilder::default(); - builder.endpoint(&mock_server.uri()); - builder.root("/"); - builder.token(token); - let op = Operator::new(builder)?.finish(); - - let bs = op.read("hello").await?; - - assert_eq!(bs, b"Hello, World!"); - Ok(()) - } - - #[tokio::test] - async fn test_stat() -> Result<()> { - let _ = tracing_subscriber::fmt().with_test_writer().try_init(); - - let mock_server = MockServer::start().await; - Mock::given(method("HEAD")) - .and(path("/hello")) - .respond_with(ResponseTemplate::new(200).insert_header("content-length", "128")) - .mount(&mock_server) - .await; - - let mut builder = HttpBuilder::default(); - builder.endpoint(&mock_server.uri()); - builder.root("/"); - let op = Operator::new(builder)?.finish(); - let bs = op.stat("hello").await?; - - assert_eq!(bs.mode(), EntryMode::FILE); - assert_eq!(bs.content_length(), 128); - Ok(()) - } - - #[tokio::test] - async fn test_read_with() -> Result<()> { - let _ = tracing_subscriber::fmt().with_test_writer().try_init(); - - let mock_server = MockServer::start().await; - Mock::given(method("GET")) - .and(path("/hello")) - .and(headers("if-none-match", vec!["*"])) - .respond_with( - ResponseTemplate::new(200) - .insert_header("content-length", "13") - .set_body_string("Hello, World!"), - ) - .mount(&mock_server) - .await; - Mock::given(method("HEAD")) - .and(path("/hello")) - .respond_with(ResponseTemplate::new(200).insert_header("content-length", "13")) - .mount(&mock_server) - .await; - - let mut builder = HttpBuilder::default(); - builder.endpoint(&mock_server.uri()); - builder.root("/"); - let op = Operator::new(builder)?.finish(); - - let match_bs = op.read_with("hello").if_none_match("*").await?; - assert_eq!(match_bs, b"Hello, World!"); - - Ok(()) - } - - #[tokio::test] - async fn test_stat_with() -> Result<()> { - let _ = tracing_subscriber::fmt().with_test_writer().try_init(); - - let mock_server = MockServer::start().await; - Mock::given(method("HEAD")) - .and(path("/hello")) - .and(headers("if-none-match", vec!["*"])) - .respond_with(ResponseTemplate::new(200).insert_header("content-length", "128")) - .mount(&mock_server) - .await; - - let mut builder = HttpBuilder::default(); - builder.endpoint(&mock_server.uri()); - builder.root("/"); - let op = Operator::new(builder)?.finish(); - let bs = op.stat_with("hello").if_none_match("*").await?; - - assert_eq!(bs.mode(), EntryMode::FILE); - assert_eq!(bs.content_length(), 128); - Ok(()) - } -} From 9bb1a91d403a86d1c4e1b89bd027901f332dbd8f Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Fri, 22 Mar 2024 12:13:30 +0800 Subject: [PATCH 068/111] Fix C binding Signed-off-by: Xuanwo --- bindings/c/include/opendal.h | 19 ++++--------------- bindings/c/src/operator_info.rs | 9 --------- bindings/c/src/reader.rs | 18 ++++++++---------- bindings/c/tests/bdd.cpp | 2 +- bindings/c/tests/opinfo.cpp | 6 ------ 5 files changed, 13 insertions(+), 41 deletions(-) diff --git a/bindings/c/include/opendal.h b/bindings/c/include/opendal.h index a7298caed57a..3478f4afb6a2 100644 --- a/bindings/c/include/opendal.h +++ b/bindings/c/include/opendal.h @@ -511,18 +511,6 @@ typedef struct opendal_capability { * If operator supports read. */ bool read; - /** - * If operator supports seek on returning reader. - */ - bool read_can_seek; - /** - * If operator supports next on returning reader. - */ - bool read_can_next; - /** - * If operator supports read with range. - */ - bool read_with_range; /** * If operator supports read with if match. */ @@ -1417,9 +1405,10 @@ void opendal_entry_free(struct opendal_entry *ptr); /** * \brief Read data from the reader. */ -struct opendal_result_reader_read opendal_reader_read(const struct opendal_reader *reader, - uint8_t *buf, - uintptr_t len); +struct opendal_result_reader_read opendal_reader_pread(const struct opendal_reader *reader, + uint8_t *buf, + uintptr_t offset, + uintptr_t len); /** * \brief Frees the heap memory used by the opendal_reader. diff --git a/bindings/c/src/operator_info.rs b/bindings/c/src/operator_info.rs index 6c7fc69d1ba7..c67092ad761b 100644 --- a/bindings/c/src/operator_info.rs +++ b/bindings/c/src/operator_info.rs @@ -42,12 +42,6 @@ pub struct opendal_capability { /// If operator supports read. pub read: bool, - /// If operator supports seek on returning reader. - pub read_can_seek: bool, - /// If operator supports next on returning reader. - pub read_can_next: bool, - /// If operator supports read with range. - pub read_with_range: bool, /// If operator supports read with if match. pub read_with_if_match: bool, /// If operator supports read with if none match. @@ -237,9 +231,6 @@ impl From for opendal_capability { stat_with_if_match: value.stat_with_if_match, stat_with_if_none_match: value.stat_with_if_none_match, read: value.read, - read_can_seek: value.read_can_seek, - read_can_next: value.read_can_next, - read_with_range: value.read_with_range, read_with_if_match: value.read_with_if_match, read_with_if_none_match: value.read_with_if_none_match, read_with_override_content_type: value.read_with_override_content_type, diff --git a/bindings/c/src/reader.rs b/bindings/c/src/reader.rs index 8f233dd8e4d5..e92d982786d7 100644 --- a/bindings/c/src/reader.rs +++ b/bindings/c/src/reader.rs @@ -37,27 +37,25 @@ impl opendal_reader { /// \brief Read data from the reader. #[no_mangle] - pub unsafe extern "C" fn opendal_reader_read( + pub unsafe extern "C" fn opendal_reader_pread( reader: *const Self, buf: *mut u8, + offset: usize, len: usize, ) -> opendal_result_reader_read { if buf.is_null() { panic!("The buffer given is pointing at NULL"); } - let buf = unsafe { std::slice::from_raw_parts_mut(buf, len) }; + let mut buf = unsafe { std::slice::from_raw_parts_mut(buf, len) }; let inner = unsafe { &mut *(*reader).inner }; - let r = inner.read(buf.len()); + let r = inner.read(&mut buf, offset as u64, len); match r { - Ok(bs) => { - buf[..bs.len()].copy_from_slice(&bs); - opendal_result_reader_read { - size: bs.len(), - error: std::ptr::null_mut(), - } - } + Ok(n) => opendal_result_reader_read { + size: n, + error: std::ptr::null_mut(), + }, Err(e) => opendal_result_reader_read { size: 0, error: opendal_error::new( diff --git a/bindings/c/tests/bdd.cpp b/bindings/c/tests/bdd.cpp index 370c17fd5b3f..2b38538afd94 100644 --- a/bindings/c/tests/bdd.cpp +++ b/bindings/c/tests/bdd.cpp @@ -95,7 +95,7 @@ TEST_F(OpendalBddTest, FeatureTest) unsigned char buffer[this->content.length()]; opendal_result_operator_reader reader = opendal_operator_reader(this->p, this->path.c_str()); EXPECT_EQ(reader.error, nullptr); - auto rst = opendal_reader_read(reader.reader, buffer, length); + auto rst = opendal_reader_pread(reader.reader, buffer, 0, length); EXPECT_EQ(rst.size, length); for (int i = 0; i < this->content.length(); i++) { EXPECT_EQ(this->content[i], buffer[i]); diff --git a/bindings/c/tests/opinfo.cpp b/bindings/c/tests/opinfo.cpp index 42c70fde2457..9684b0360473 100644 --- a/bindings/c/tests/opinfo.cpp +++ b/bindings/c/tests/opinfo.cpp @@ -65,9 +65,6 @@ TEST_F(OpendalOperatorInfoTest, CapabilityTest) EXPECT_TRUE(full_cap.blocking); EXPECT_TRUE(full_cap.read); - EXPECT_TRUE(full_cap.read_can_seek); - EXPECT_TRUE(full_cap.read_can_next); - EXPECT_TRUE(full_cap.read_with_range); EXPECT_TRUE(full_cap.stat); EXPECT_TRUE(full_cap.write); EXPECT_TRUE(full_cap.write_can_empty); @@ -78,9 +75,6 @@ TEST_F(OpendalOperatorInfoTest, CapabilityTest) EXPECT_TRUE(native_cap.blocking); EXPECT_TRUE(native_cap.read); - EXPECT_TRUE(native_cap.read_can_seek); - EXPECT_TRUE(native_cap.read_can_next); - EXPECT_TRUE(native_cap.read_with_range); EXPECT_TRUE(native_cap.stat); EXPECT_TRUE(native_cap.write); EXPECT_TRUE(native_cap.write_can_empty); From a81c30c7448380ae384efcfc3acfdbaf10dac66e Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Fri, 22 Mar 2024 13:05:54 +0800 Subject: [PATCH 069/111] Export adapter Signed-off-by: Xuanwo --- core/benches/ops/read.rs | 2 +- core/src/types/mod.rs | 2 + core/src/types/reader.rs | 86 +++++++++++++++++++++++++++------------- 3 files changed, 62 insertions(+), 28 deletions(-) diff --git a/core/benches/ops/read.rs b/core/benches/ops/read.rs index 4b8f40a0cc10..aad05f946f88 100644 --- a/core/benches/ops/read.rs +++ b/core/benches/ops/read.rs @@ -52,7 +52,7 @@ fn bench_read_full(c: &mut Criterion, name: &str, op: Operator) { b.to_async(&*TEST_RUNTIME).iter(|| async { let r = op.reader_with(path).await.unwrap(); - let r = r.into_async_read(0..size.bytes() as u64); + let r = r.into_futures_async_read(0..size.bytes() as u64); io::copy(r, &mut io::sink()).await.unwrap(); }) }); diff --git a/core/src/types/mod.rs b/core/src/types/mod.rs index 804334f6e299..3dbe767e8431 100644 --- a/core/src/types/mod.rs +++ b/core/src/types/mod.rs @@ -26,6 +26,8 @@ pub use metadata::Metadata; pub use metadata::Metakey; mod reader; +pub use reader::into_futures_async_read::FuturesAsyncReader; +pub use reader::into_futures_stream::FuturesStream; pub use reader::Reader; mod blocking_reader; diff --git a/core/src/types/reader.rs b/core/src/types/reader.rs index 6696adb33cb8..ae4d4e1a0d4e 100644 --- a/core/src/types/reader.rs +++ b/core/src/types/reader.rs @@ -130,35 +130,29 @@ impl Reader { self.read_range(buf, ..).await } - /// Convert reader into async read. + /// Convert reader into [`FuturesAsyncReader`] which implements [`futures::AsyncRead`], + /// [`futures::AsyncSeek`] and [`futures::AsyncBufRead`]. #[inline] - pub fn into_async_read( + pub fn into_futures_async_read( self, range: Range, ) -> impl futures::AsyncRead + futures::AsyncSeek + Send + Sync + Unpin { // TODO: the capacity should be decided by services. - impl_futures_async_read::FuturesReader::new(self.inner, range, 4 * 1024 * 1024) + FuturesAsyncReader::new(self.inner, range) } - /// Convert reader into async buf read. + /// Convert reader into [`FuturesStream`] which implements [`futures::Stream`], + /// [`futures::AsyncSeek`] and [`futures::AsyncBufRead`]. #[inline] - pub fn into_async_buf_read( + pub fn into_futures_stream( self, range: Range, - capacity: usize, - ) -> impl futures::AsyncBufRead + futures::AsyncSeek + Send + Sync + Unpin { - impl_futures_async_read::FuturesReader::new(self.inner, range, capacity) - } - - /// Convert reader into stream. - #[inline] - pub fn into_stream(self, range: Range) -> impl futures::Stream + Send + Sync + Unpin { - // TODO: the capacity should be decided by services. - impl_futures_stream::FuturesStream::new(self.inner, range, 4 * 1024 * 1024) + ) -> impl futures::Stream + Send + Sync + Unpin { + FuturesStream::new(self.inner, range) } } -mod impl_futures_async_read { +pub mod into_futures_async_read { use std::io; use std::io::SeekFrom; use std::ops::Range; @@ -175,7 +169,19 @@ mod impl_futures_async_read { use crate::raw::*; use crate::*; - pub struct FuturesReader { + /// FuturesAsyncReader is the adapter of [`AsyncRead`] for [`Reader`]. + /// + /// Users can use this adapter in cases where they need to use [`AsyncRead`] trait. + /// + /// FuturesAsyncReader implements the following trait: + /// + /// - [`AsyncBufRead`] + /// - [`AsyncRead`] + /// - [`AsyncSeek`] + /// - [`Unpin`] + /// - [`Send`] + /// - [`Sync`] + pub struct FuturesAsyncReader { state: State, offset: u64, size: u64, @@ -195,22 +201,30 @@ mod impl_futures_async_read { /// FuturesReader only exposes `&mut self` to the outside world, so it's safe to be `Sync`. unsafe impl Sync for State {} - impl FuturesReader { + impl FuturesAsyncReader { + /// NOTE: don't allow users to create FuturesAsyncReader directly. #[inline] - pub fn new(r: oio::Reader, range: Range, cap: usize) -> Self { - FuturesReader { + pub(super) fn new(r: oio::Reader, range: Range) -> Self { + FuturesAsyncReader { state: State::Idle(Some(r)), offset: range.start, size: range.end - range.start, - cap, + // TODO: should use services preferred io size. + cap: 4 * 1024 * 1024, cur: 0, buf: oio::Buffer::new(), } } + + /// Set the capacity of this reader to control the IO size. + pub fn with_capacity(mut self, cap: usize) -> Self { + self.cap = cap; + self + } } - impl AsyncBufRead for FuturesReader { + impl AsyncBufRead for FuturesAsyncReader { fn poll_fill_buf(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { let this = self.get_mut(); loop { @@ -249,7 +263,7 @@ mod impl_futures_async_read { } } - impl AsyncRead for FuturesReader { + impl AsyncRead for FuturesAsyncReader { fn poll_read( mut self: Pin<&mut Self>, cx: &mut Context<'_>, @@ -263,7 +277,7 @@ mod impl_futures_async_read { } } - impl AsyncSeek for FuturesReader { + impl AsyncSeek for FuturesAsyncReader { fn poll_seek( mut self: Pin<&mut Self>, _: &mut Context<'_>, @@ -297,7 +311,7 @@ mod impl_futures_async_read { } } -mod impl_futures_stream { +pub mod into_futures_stream { use std::io; use std::ops::Range; use std::pin::Pin; @@ -312,6 +326,16 @@ mod impl_futures_stream { use crate::raw::*; use crate::*; + /// FuturesStream is the adapter of [`Stream`] for [`Reader`]. + /// + /// Users can use this adapter in cases where they need to use [`Stream`] trait. + /// + /// FuturesStream implements the following trait: + /// + /// - [`Stream`] + /// - [`Unpin`] + /// - [`Send`] + /// - [`Sync`] pub struct FuturesStream { state: State, offset: u64, @@ -332,17 +356,25 @@ mod impl_futures_stream { unsafe impl Sync for State {} impl FuturesStream { + /// NOTE: don't allow users to create FuturesStream directly. #[inline] - pub fn new(r: oio::Reader, range: Range, cap: usize) -> Self { + pub(crate) fn new(r: oio::Reader, range: Range) -> Self { FuturesStream { state: State::Idle(Some(r)), offset: range.start, size: range.end - range.start, - cap, + // TODO: should use services preferred io size. + cap: 4 * 1024 * 1024, cur: 0, } } + + /// Set the capacity of this reader to control the IO size. + pub fn with_capacity(mut self, cap: usize) -> Self { + self.cap = cap; + self + } } impl Stream for FuturesStream { From 5c851ca7875c105232bb47b92ee41541701d1932 Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Fri, 22 Mar 2024 13:50:02 +0800 Subject: [PATCH 070/111] Polish API Signed-off-by: Xuanwo --- core/benches/ops/read.rs | 2 +- core/src/types/blocking_reader.rs | 225 +++++++++++++++++++++++++----- core/src/types/mod.rs | 6 +- core/src/types/reader.rs | 64 +++------ 4 files changed, 218 insertions(+), 79 deletions(-) diff --git a/core/benches/ops/read.rs b/core/benches/ops/read.rs index aad05f946f88..84a1e9b48cf3 100644 --- a/core/benches/ops/read.rs +++ b/core/benches/ops/read.rs @@ -52,7 +52,7 @@ fn bench_read_full(c: &mut Criterion, name: &str, op: Operator) { b.to_async(&*TEST_RUNTIME).iter(|| async { let r = op.reader_with(path).await.unwrap(); - let r = r.into_futures_async_read(0..size.bytes() as u64); + let r = r.into_futures_io_async_read(0..size.bytes() as u64); io::copy(r, &mut io::sink()).await.unwrap(); }) }); diff --git a/core/src/types/blocking_reader.rs b/core/src/types/blocking_reader.rs index 65cd4957fcc4..a8dba97d545d 100644 --- a/core/src/types/blocking_reader.rs +++ b/core/src/types/blocking_reader.rs @@ -16,7 +16,7 @@ // under the License. use std::collections::Bound; -use std::ops::RangeBounds; +use std::ops::{Range, RangeBounds}; use bytes::Buf; use bytes::BufMut; @@ -111,37 +111,196 @@ impl BlockingReader { pub fn read_to_end(&self, buf: &mut impl BufMut) -> Result { self.read_range(buf, ..) } + + /// Convert reader into [`FuturesIoAsyncReader`] which implements [`futures::AsyncRead`], + /// [`futures::AsyncSeek`] and [`futures::AsyncBufRead`]. + #[inline] + pub fn into_std_io_read(self, range: Range) -> StdReader { + // TODO: the capacity should be decided by services. + StdReader::new(self.inner, range) + } + + /// Convert reader into [`FuturesBytesStream`] which implements [`futures::Stream`], + /// [`futures::AsyncSeek`] and [`futures::AsyncBufRead`]. + #[inline] + pub fn into_std_bytes_iterator(self, range: Range) -> StdBytesIterator { + StdBytesIterator::new(self.inner, range) + } } -// impl io::Read for BlockingReader { -// #[inline] -// fn read(&mut self, buf: &mut [u8]) -> io::Result { -// let bs = self.inner.read(buf.len()).map_err(format_std_io_error)?; -// buf[..bs.len()].copy_from_slice(&bs); -// Ok(bs.len()) -// } -// } -// -// impl io::Seek for BlockingReader { -// #[inline] -// fn seek(&mut self, pos: io::SeekFrom) -> io::Result { -// self.inner.seek(pos).map_err(format_std_io_error) -// } -// } -// -// impl Iterator for BlockingReader { -// type Item = io::Result; -// -// #[inline] -// fn next(&mut self) -> Option { -// match self -// .inner -// .read(4 * 1024 * 1024) -// .map_err(format_std_io_error) -// { -// Ok(bs) if bs.is_empty() => None, -// Ok(bs) => Some(Ok(bs)), -// Err(err) => Some(Err(err)), -// } -// } -// } +pub mod into_std_read { + use crate::raw::{format_std_io_error, oio}; + use bytes::Buf; + use std::io; + use std::io::Read; + use std::io::Seek; + use std::io::{BufRead, SeekFrom}; + use std::ops::Range; + + /// StdReader is the adapter of [`Read`], [`Seek`] and [`BufRead`] for [`BlockingReader`][crate::BlockingReader]. + /// + /// Users can use this adapter in cases where they need to use [`Read`] or [`BufRead`] trait. + /// + /// StdReader also implements [`Send`] and [`Sync`]. + pub struct StdReader { + inner: oio::BlockingReader, + offset: u64, + size: u64, + cap: usize, + + cur: u64, + buf: oio::Buffer, + } + + impl StdReader { + /// NOTE: don't allow users to create StdReader directly. + #[inline] + pub(super) fn new(r: oio::BlockingReader, range: Range) -> Self { + StdReader { + inner: r, + offset: range.start, + size: range.end - range.start, + // TODO: should use services preferred io size. + cap: 4 * 1024 * 1024, + + cur: 0, + buf: oio::Buffer::new(), + } + } + + /// Set the capacity of this reader to control the IO size. + pub fn with_capacity(mut self, cap: usize) -> Self { + self.cap = cap; + self + } + } + + impl BufRead for StdReader { + fn fill_buf(&mut self) -> io::Result<&[u8]> { + if self.buf.has_remaining() { + return Ok(self.buf.chunk()); + } + + // Make sure cur didn't exceed size. + if self.cur >= self.size { + return Ok(&[]); + } + + let next_offset = self.offset + self.cur; + let next_size = (self.size - self.cur).min(self.cap as u64) as usize; + self.buf = self + .inner + .read_at(next_offset, next_size) + .map_err(format_std_io_error)?; + Ok(self.buf.chunk()) + } + + fn consume(&mut self, amt: usize) { + self.buf.advance(amt); + self.cur += amt as u64; + } + } + + impl Read for StdReader { + #[inline] + fn read(&mut self, buf: &mut [u8]) -> io::Result { + let bs = self.fill_buf()?; + let n = bs.len().min(buf.len()); + buf[..n].copy_from_slice(&bs[..n]); + self.consume(n); + Ok(n) + } + } + + impl Seek for StdReader { + #[inline] + fn seek(&mut self, pos: SeekFrom) -> io::Result { + let new_pos = match pos { + SeekFrom::Start(pos) => pos as i64, + SeekFrom::End(pos) => self.size as i64 + pos, + SeekFrom::Current(pos) => self.cur as i64 + pos, + }; + + if new_pos < 0 { + return Err(io::Error::new( + io::ErrorKind::InvalidInput, + "invalid seek to a negative position", + )); + } + + let new_pos = new_pos as u64; + + if (self.cur..self.cur + self.buf.remaining() as u64).contains(&new_pos) { + let cnt = new_pos - self.cur; + self.buf.advance(cnt as _); + } else { + self.buf = oio::Buffer::new() + } + + self.cur = new_pos; + Ok(self.cur) + } + } +} + +pub mod into_std_iterator { + use crate::raw::*; + use bytes::{Buf, Bytes}; + use std::io; + + /// StdIterator is the adapter of [`Iterator`] for [`BlockingReader`][crate::BlockingReader]. + /// + /// Users can use this adapter in cases where they need to use [`Iterator`] trait. + /// + /// StdIterator also implements [`Send`] and [`Sync`]. + pub struct StdBytesIterator { + inner: oio::BlockingReader, + offset: u64, + size: u64, + cap: usize, + + cur: u64, + } + + impl StdBytesIterator { + /// NOTE: don't allow users to create StdIterator directly. + #[inline] + pub(crate) fn new(r: oio::BlockingReader, range: std::ops::Range) -> Self { + StdBytesIterator { + inner: r, + offset: range.start, + size: range.end - range.start, + // TODO: should use services preferred io size. + cap: 4 * 1024 * 1024, + cur: 0, + } + } + + /// Set the capacity of this reader to control the IO size. + pub fn with_capacity(mut self, cap: usize) -> Self { + self.cap = cap; + self + } + } + + impl Iterator for StdBytesIterator { + type Item = io::Result; + + fn next(&mut self) -> Option { + if self.cur >= self.size { + return None; + } + + let next_offset = self.offset + self.cur; + let next_size = (self.size - self.cur).min(self.cap as u64) as usize; + match self.inner.read_at(next_offset, next_size) { + Ok(buf) if !buf.has_remaining() => None, + Ok(mut buf) => { + self.cur += buf.remaining() as u64; + Some(Ok(buf.copy_to_bytes(buf.remaining()))) + } + Err(err) => Some(Err(format_std_io_error(err))), + } + } + } +} diff --git a/core/src/types/mod.rs b/core/src/types/mod.rs index 3dbe767e8431..b1dd33c44356 100644 --- a/core/src/types/mod.rs +++ b/core/src/types/mod.rs @@ -26,11 +26,13 @@ pub use metadata::Metadata; pub use metadata::Metakey; mod reader; -pub use reader::into_futures_async_read::FuturesAsyncReader; -pub use reader::into_futures_stream::FuturesStream; +pub use reader::into_futures_async_read::FuturesIoAsyncReader; +pub use reader::into_futures_stream::FuturesBytesStream; pub use reader::Reader; mod blocking_reader; +pub use blocking_reader::into_std_iterator::StdBytesIterator; +pub use blocking_reader::into_std_read::StdReader; pub use blocking_reader::BlockingReader; mod writer; diff --git a/core/src/types/reader.rs b/core/src/types/reader.rs index ae4d4e1a0d4e..d2cad6732f14 100644 --- a/core/src/types/reader.rs +++ b/core/src/types/reader.rs @@ -36,10 +36,6 @@ use crate::*; /// ## Direct /// /// [`Reader`] provides public API including [`Reader::read`], [`Reader:read_range`], and [`Reader::read_to_end`]. You can use those APIs directly without extra copy. -/// -/// # TODO -/// -/// Implement `into_async_read` and `into_stream`. pub struct Reader { inner: oio::Reader, } @@ -130,25 +126,18 @@ impl Reader { self.read_range(buf, ..).await } - /// Convert reader into [`FuturesAsyncReader`] which implements [`futures::AsyncRead`], + /// Convert reader into [`FuturesIoAsyncReader`] which implements [`futures::AsyncRead`], /// [`futures::AsyncSeek`] and [`futures::AsyncBufRead`]. #[inline] - pub fn into_futures_async_read( - self, - range: Range, - ) -> impl futures::AsyncRead + futures::AsyncSeek + Send + Sync + Unpin { - // TODO: the capacity should be decided by services. - FuturesAsyncReader::new(self.inner, range) + pub fn into_futures_io_async_read(self, range: Range) -> FuturesIoAsyncReader { + FuturesIoAsyncReader::new(self.inner, range) } - /// Convert reader into [`FuturesStream`] which implements [`futures::Stream`], + /// Convert reader into [`FuturesBytesStream`] which implements [`futures::Stream`], /// [`futures::AsyncSeek`] and [`futures::AsyncBufRead`]. #[inline] - pub fn into_futures_stream( - self, - range: Range, - ) -> impl futures::Stream + Send + Sync + Unpin { - FuturesStream::new(self.inner, range) + pub fn into_futures_bytes_stream(self, range: Range) -> FuturesBytesStream { + FuturesBytesStream::new(self.inner, range) } } @@ -169,19 +158,13 @@ pub mod into_futures_async_read { use crate::raw::*; use crate::*; - /// FuturesAsyncReader is the adapter of [`AsyncRead`] for [`Reader`]. + /// FuturesAsyncReader is the adapter of [`AsyncRead`], [`AsyncBufRead`] and [`AsyncSeek`] + /// for [`Reader`]. /// - /// Users can use this adapter in cases where they need to use [`AsyncRead`] trait. + /// Users can use this adapter in cases where they need to use [`AsyncRead`] related trait. /// - /// FuturesAsyncReader implements the following trait: - /// - /// - [`AsyncBufRead`] - /// - [`AsyncRead`] - /// - [`AsyncSeek`] - /// - [`Unpin`] - /// - [`Send`] - /// - [`Sync`] - pub struct FuturesAsyncReader { + /// FuturesAsyncReader also implements [`Unpin`], [`Send`] and [`Sync`] + pub struct FuturesIoAsyncReader { state: State, offset: u64, size: u64, @@ -201,11 +184,11 @@ pub mod into_futures_async_read { /// FuturesReader only exposes `&mut self` to the outside world, so it's safe to be `Sync`. unsafe impl Sync for State {} - impl FuturesAsyncReader { + impl FuturesIoAsyncReader { /// NOTE: don't allow users to create FuturesAsyncReader directly. #[inline] pub(super) fn new(r: oio::Reader, range: Range) -> Self { - FuturesAsyncReader { + FuturesIoAsyncReader { state: State::Idle(Some(r)), offset: range.start, size: range.end - range.start, @@ -224,7 +207,7 @@ pub mod into_futures_async_read { } } - impl AsyncBufRead for FuturesAsyncReader { + impl AsyncBufRead for FuturesIoAsyncReader { fn poll_fill_buf(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { let this = self.get_mut(); loop { @@ -263,7 +246,7 @@ pub mod into_futures_async_read { } } - impl AsyncRead for FuturesAsyncReader { + impl AsyncRead for FuturesIoAsyncReader { fn poll_read( mut self: Pin<&mut Self>, cx: &mut Context<'_>, @@ -277,7 +260,7 @@ pub mod into_futures_async_read { } } - impl AsyncSeek for FuturesAsyncReader { + impl AsyncSeek for FuturesIoAsyncReader { fn poll_seek( mut self: Pin<&mut Self>, _: &mut Context<'_>, @@ -330,13 +313,8 @@ pub mod into_futures_stream { /// /// Users can use this adapter in cases where they need to use [`Stream`] trait. /// - /// FuturesStream implements the following trait: - /// - /// - [`Stream`] - /// - [`Unpin`] - /// - [`Send`] - /// - [`Sync`] - pub struct FuturesStream { + /// FuturesStream also implements [`Unpin`], [`Send`] and [`Sync`]. + pub struct FuturesBytesStream { state: State, offset: u64, size: u64, @@ -355,11 +333,11 @@ pub mod into_futures_stream { /// FuturesReader only exposes `&mut self` to the outside world, so it's safe to be `Sync`. unsafe impl Sync for State {} - impl FuturesStream { + impl FuturesBytesStream { /// NOTE: don't allow users to create FuturesStream directly. #[inline] pub(crate) fn new(r: oio::Reader, range: Range) -> Self { - FuturesStream { + FuturesBytesStream { state: State::Idle(Some(r)), offset: range.start, size: range.end - range.start, @@ -377,7 +355,7 @@ pub mod into_futures_stream { } } - impl Stream for FuturesStream { + impl Stream for FuturesBytesStream { type Item = io::Result; fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { From ec814270c5dbd2ed69ed1714215498cc1ecb9135 Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Fri, 22 Mar 2024 14:48:24 +0800 Subject: [PATCH 071/111] Fix c Signed-off-by: Xuanwo --- bindings/c/include/opendal.h | 4 ++-- bindings/c/src/reader.rs | 2 +- bindings/c/tests/bdd.cpp | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/bindings/c/include/opendal.h b/bindings/c/include/opendal.h index 3478f4afb6a2..0ee9aec37c16 100644 --- a/bindings/c/include/opendal.h +++ b/bindings/c/include/opendal.h @@ -1407,8 +1407,8 @@ void opendal_entry_free(struct opendal_entry *ptr); */ struct opendal_result_reader_read opendal_reader_pread(const struct opendal_reader *reader, uint8_t *buf, - uintptr_t offset, - uintptr_t len); + uintptr_t len, + uintptr_t offset); /** * \brief Frees the heap memory used by the opendal_reader. diff --git a/bindings/c/src/reader.rs b/bindings/c/src/reader.rs index e92d982786d7..b6aee94dc80f 100644 --- a/bindings/c/src/reader.rs +++ b/bindings/c/src/reader.rs @@ -40,8 +40,8 @@ impl opendal_reader { pub unsafe extern "C" fn opendal_reader_pread( reader: *const Self, buf: *mut u8, - offset: usize, len: usize, + offset: usize, ) -> opendal_result_reader_read { if buf.is_null() { panic!("The buffer given is pointing at NULL"); diff --git a/bindings/c/tests/bdd.cpp b/bindings/c/tests/bdd.cpp index 2b38538afd94..3d2ad33b6174 100644 --- a/bindings/c/tests/bdd.cpp +++ b/bindings/c/tests/bdd.cpp @@ -95,7 +95,7 @@ TEST_F(OpendalBddTest, FeatureTest) unsigned char buffer[this->content.length()]; opendal_result_operator_reader reader = opendal_operator_reader(this->p, this->path.c_str()); EXPECT_EQ(reader.error, nullptr); - auto rst = opendal_reader_pread(reader.reader, buffer, 0, length); + auto rst = opendal_reader_pread(reader.reader, buffer, length, 0); EXPECT_EQ(rst.size, length); for (int i = 0; i < this->content.length(); i++) { EXPECT_EQ(this->content[i], buffer[i]); From ad645d1aadf48ba22662512dfbf84e9a3a340f99 Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Fri, 22 Mar 2024 14:49:51 +0800 Subject: [PATCH 072/111] FIx naming Signed-off-by: Xuanwo --- core/src/types/blocking_reader.rs | 16 ++++++++-------- core/src/types/mod.rs | 2 +- 2 files changed, 9 insertions(+), 9 deletions(-) diff --git a/core/src/types/blocking_reader.rs b/core/src/types/blocking_reader.rs index a8dba97d545d..1cac547c7d08 100644 --- a/core/src/types/blocking_reader.rs +++ b/core/src/types/blocking_reader.rs @@ -115,9 +115,9 @@ impl BlockingReader { /// Convert reader into [`FuturesIoAsyncReader`] which implements [`futures::AsyncRead`], /// [`futures::AsyncSeek`] and [`futures::AsyncBufRead`]. #[inline] - pub fn into_std_io_read(self, range: Range) -> StdReader { + pub fn into_std_io_read(self, range: Range) -> StdIoReader { // TODO: the capacity should be decided by services. - StdReader::new(self.inner, range) + StdIoReader::new(self.inner, range) } /// Convert reader into [`FuturesBytesStream`] which implements [`futures::Stream`], @@ -142,7 +142,7 @@ pub mod into_std_read { /// Users can use this adapter in cases where they need to use [`Read`] or [`BufRead`] trait. /// /// StdReader also implements [`Send`] and [`Sync`]. - pub struct StdReader { + pub struct StdIoReader { inner: oio::BlockingReader, offset: u64, size: u64, @@ -152,11 +152,11 @@ pub mod into_std_read { buf: oio::Buffer, } - impl StdReader { + impl StdIoReader { /// NOTE: don't allow users to create StdReader directly. #[inline] pub(super) fn new(r: oio::BlockingReader, range: Range) -> Self { - StdReader { + StdIoReader { inner: r, offset: range.start, size: range.end - range.start, @@ -175,7 +175,7 @@ pub mod into_std_read { } } - impl BufRead for StdReader { + impl BufRead for StdIoReader { fn fill_buf(&mut self) -> io::Result<&[u8]> { if self.buf.has_remaining() { return Ok(self.buf.chunk()); @@ -201,7 +201,7 @@ pub mod into_std_read { } } - impl Read for StdReader { + impl Read for StdIoReader { #[inline] fn read(&mut self, buf: &mut [u8]) -> io::Result { let bs = self.fill_buf()?; @@ -212,7 +212,7 @@ pub mod into_std_read { } } - impl Seek for StdReader { + impl Seek for StdIoReader { #[inline] fn seek(&mut self, pos: SeekFrom) -> io::Result { let new_pos = match pos { diff --git a/core/src/types/mod.rs b/core/src/types/mod.rs index b1dd33c44356..daf1a25b9d51 100644 --- a/core/src/types/mod.rs +++ b/core/src/types/mod.rs @@ -32,7 +32,7 @@ pub use reader::Reader; mod blocking_reader; pub use blocking_reader::into_std_iterator::StdBytesIterator; -pub use blocking_reader::into_std_read::StdReader; +pub use blocking_reader::into_std_read::StdIoReader; pub use blocking_reader::BlockingReader; mod writer; From 3f62edcf51ed474d54be0f6f7fec58899b0db64d Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Fri, 22 Mar 2024 14:51:52 +0800 Subject: [PATCH 073/111] Fix cpp Signed-off-by: Xuanwo --- bindings/cpp/src/lib.rs | 7 ++++++- bindings/cpp/src/reader.rs | 8 ++++---- 2 files changed, 10 insertions(+), 5 deletions(-) diff --git a/bindings/cpp/src/lib.rs b/bindings/cpp/src/lib.rs index e59c1482d5e8..88d6c0092a45 100644 --- a/bindings/cpp/src/lib.rs +++ b/bindings/cpp/src/lib.rs @@ -154,7 +154,12 @@ impl Operator { } fn reader(&self, path: &str) -> Result> { - Ok(Box::new(Reader(self.0.reader(path)?))) + let meta = self.0.stat(path)?; + Ok(Box::new(Reader( + self.0 + .reader(path)? + .into_std_io_read(0..meta.content_length()), + ))) } fn lister(&self, path: &str) -> Result> { diff --git a/bindings/cpp/src/reader.rs b/bindings/cpp/src/reader.rs index 6f9bfd8fe9ec..7c083501bca4 100644 --- a/bindings/cpp/src/reader.rs +++ b/bindings/cpp/src/reader.rs @@ -17,16 +17,16 @@ use anyhow::Result; use opendal as od; +use std::io::{Read, Seek}; use super::ffi; -pub struct Reader(pub od::BlockingReader); +pub struct Reader(pub od::StdIoReader); impl Reader { pub fn read(&mut self, buf: &mut [u8]) -> Result { - let bs = self.0.read(buf.len())?; - buf[..bs.len()].copy_from_slice(&bs); - Ok(bs.len()) + let n = self.0.read(buf)?; + Ok(n) } pub fn seek(&mut self, offset: u64, dir: ffi::SeekFrom) -> Result { From 1d6b51299eafdbed5d84a1fd9aa366e1f3132e7c Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Fri, 22 Mar 2024 15:53:18 +0800 Subject: [PATCH 074/111] revert Signed-off-by: Xuanwo --- bindings/c/include/opendal.h | 19 +++++++++++++++---- bindings/c/src/operator_info.rs | 9 +++++++++ bindings/c/src/reader.rs | 18 ++++++++++-------- bindings/c/tests/bdd.cpp | 2 +- bindings/c/tests/opinfo.cpp | 6 ++++++ 5 files changed, 41 insertions(+), 13 deletions(-) diff --git a/bindings/c/include/opendal.h b/bindings/c/include/opendal.h index 0ee9aec37c16..a7298caed57a 100644 --- a/bindings/c/include/opendal.h +++ b/bindings/c/include/opendal.h @@ -511,6 +511,18 @@ typedef struct opendal_capability { * If operator supports read. */ bool read; + /** + * If operator supports seek on returning reader. + */ + bool read_can_seek; + /** + * If operator supports next on returning reader. + */ + bool read_can_next; + /** + * If operator supports read with range. + */ + bool read_with_range; /** * If operator supports read with if match. */ @@ -1405,10 +1417,9 @@ void opendal_entry_free(struct opendal_entry *ptr); /** * \brief Read data from the reader. */ -struct opendal_result_reader_read opendal_reader_pread(const struct opendal_reader *reader, - uint8_t *buf, - uintptr_t len, - uintptr_t offset); +struct opendal_result_reader_read opendal_reader_read(const struct opendal_reader *reader, + uint8_t *buf, + uintptr_t len); /** * \brief Frees the heap memory used by the opendal_reader. diff --git a/bindings/c/src/operator_info.rs b/bindings/c/src/operator_info.rs index c67092ad761b..6c7fc69d1ba7 100644 --- a/bindings/c/src/operator_info.rs +++ b/bindings/c/src/operator_info.rs @@ -42,6 +42,12 @@ pub struct opendal_capability { /// If operator supports read. pub read: bool, + /// If operator supports seek on returning reader. + pub read_can_seek: bool, + /// If operator supports next on returning reader. + pub read_can_next: bool, + /// If operator supports read with range. + pub read_with_range: bool, /// If operator supports read with if match. pub read_with_if_match: bool, /// If operator supports read with if none match. @@ -231,6 +237,9 @@ impl From for opendal_capability { stat_with_if_match: value.stat_with_if_match, stat_with_if_none_match: value.stat_with_if_none_match, read: value.read, + read_can_seek: value.read_can_seek, + read_can_next: value.read_can_next, + read_with_range: value.read_with_range, read_with_if_match: value.read_with_if_match, read_with_if_none_match: value.read_with_if_none_match, read_with_override_content_type: value.read_with_override_content_type, diff --git a/bindings/c/src/reader.rs b/bindings/c/src/reader.rs index b6aee94dc80f..8f233dd8e4d5 100644 --- a/bindings/c/src/reader.rs +++ b/bindings/c/src/reader.rs @@ -37,25 +37,27 @@ impl opendal_reader { /// \brief Read data from the reader. #[no_mangle] - pub unsafe extern "C" fn opendal_reader_pread( + pub unsafe extern "C" fn opendal_reader_read( reader: *const Self, buf: *mut u8, len: usize, - offset: usize, ) -> opendal_result_reader_read { if buf.is_null() { panic!("The buffer given is pointing at NULL"); } - let mut buf = unsafe { std::slice::from_raw_parts_mut(buf, len) }; + let buf = unsafe { std::slice::from_raw_parts_mut(buf, len) }; let inner = unsafe { &mut *(*reader).inner }; - let r = inner.read(&mut buf, offset as u64, len); + let r = inner.read(buf.len()); match r { - Ok(n) => opendal_result_reader_read { - size: n, - error: std::ptr::null_mut(), - }, + Ok(bs) => { + buf[..bs.len()].copy_from_slice(&bs); + opendal_result_reader_read { + size: bs.len(), + error: std::ptr::null_mut(), + } + } Err(e) => opendal_result_reader_read { size: 0, error: opendal_error::new( diff --git a/bindings/c/tests/bdd.cpp b/bindings/c/tests/bdd.cpp index 3d2ad33b6174..370c17fd5b3f 100644 --- a/bindings/c/tests/bdd.cpp +++ b/bindings/c/tests/bdd.cpp @@ -95,7 +95,7 @@ TEST_F(OpendalBddTest, FeatureTest) unsigned char buffer[this->content.length()]; opendal_result_operator_reader reader = opendal_operator_reader(this->p, this->path.c_str()); EXPECT_EQ(reader.error, nullptr); - auto rst = opendal_reader_pread(reader.reader, buffer, length, 0); + auto rst = opendal_reader_read(reader.reader, buffer, length); EXPECT_EQ(rst.size, length); for (int i = 0; i < this->content.length(); i++) { EXPECT_EQ(this->content[i], buffer[i]); diff --git a/bindings/c/tests/opinfo.cpp b/bindings/c/tests/opinfo.cpp index 9684b0360473..42c70fde2457 100644 --- a/bindings/c/tests/opinfo.cpp +++ b/bindings/c/tests/opinfo.cpp @@ -65,6 +65,9 @@ TEST_F(OpendalOperatorInfoTest, CapabilityTest) EXPECT_TRUE(full_cap.blocking); EXPECT_TRUE(full_cap.read); + EXPECT_TRUE(full_cap.read_can_seek); + EXPECT_TRUE(full_cap.read_can_next); + EXPECT_TRUE(full_cap.read_with_range); EXPECT_TRUE(full_cap.stat); EXPECT_TRUE(full_cap.write); EXPECT_TRUE(full_cap.write_can_empty); @@ -75,6 +78,9 @@ TEST_F(OpendalOperatorInfoTest, CapabilityTest) EXPECT_TRUE(native_cap.blocking); EXPECT_TRUE(native_cap.read); + EXPECT_TRUE(native_cap.read_can_seek); + EXPECT_TRUE(native_cap.read_can_next); + EXPECT_TRUE(native_cap.read_with_range); EXPECT_TRUE(native_cap.stat); EXPECT_TRUE(native_cap.write); EXPECT_TRUE(native_cap.write_can_empty); From dccd89be0a60bab58415f0a37a5942007fde313f Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Fri, 22 Mar 2024 15:57:42 +0800 Subject: [PATCH 075/111] Fix C Signed-off-by: Xuanwo --- bindings/c/include/opendal.h | 29 ++++++++++------------------- bindings/c/src/operator.rs | 13 ++++++++++++- bindings/c/src/operator_info.rs | 9 --------- bindings/c/src/reader.rs | 22 ++++++++++------------ bindings/c/tests/opinfo.cpp | 6 ------ 5 files changed, 32 insertions(+), 47 deletions(-) diff --git a/bindings/c/include/opendal.h b/bindings/c/include/opendal.h index a7298caed57a..ca9de2042aed 100644 --- a/bindings/c/include/opendal.h +++ b/bindings/c/include/opendal.h @@ -144,12 +144,6 @@ typedef struct BlockingLister BlockingLister; */ typedef struct BlockingOperator BlockingOperator; -/** - * BlockingReader is designed to read data from given path in an blocking - * manner. - */ -typedef struct BlockingReader BlockingReader; - /** * Entry returned by [`Lister`] or [`BlockingLister`] to represent a path and it's relative metadata. * @@ -211,6 +205,15 @@ typedef struct Metadata Metadata; */ typedef struct OperatorInfo OperatorInfo; +/** + * StdReader is the adapter of [`Read`], [`Seek`] and [`BufRead`] for [`BlockingReader`][crate::BlockingReader]. + * + * Users can use this adapter in cases where they need to use [`Read`] or [`BufRead`] trait. + * + * StdReader also implements [`Send`] and [`Sync`]. + */ +typedef struct StdIoReader StdIoReader; + /** * \brief opendal_bytes carries raw-bytes with its length * @@ -405,7 +408,7 @@ typedef struct opendal_result_read { * a opendal::BlockingReader, which is inside the Rust core code. */ typedef struct opendal_reader { - struct BlockingReader *inner; + struct StdIoReader *inner; } opendal_reader; /** @@ -511,18 +514,6 @@ typedef struct opendal_capability { * If operator supports read. */ bool read; - /** - * If operator supports seek on returning reader. - */ - bool read_can_seek; - /** - * If operator supports next on returning reader. - */ - bool read_can_next; - /** - * If operator supports read with range. - */ - bool read_with_range; /** * If operator supports read with if match. */ diff --git a/bindings/c/src/operator.rs b/bindings/c/src/operator.rs index c8fd31b25047..021769bcc00f 100644 --- a/bindings/c/src/operator.rs +++ b/bindings/c/src/operator.rs @@ -367,10 +367,21 @@ pub unsafe extern "C" fn opendal_operator_reader( panic!("The path given is pointing at NULL"); } let op = (*op).as_ref(); + let path = unsafe { std::ffi::CStr::from_ptr(path).to_str().unwrap() }; + let meta = match op.stat(path) { + Ok(meta) => meta, + Err(err) => { + return opendal_result_operator_reader { + reader: std::ptr::null_mut(), + error: opendal_error::new(err), + } + } + }; + match op.reader(path) { Ok(reader) => opendal_result_operator_reader { - reader: Box::into_raw(Box::new(opendal_reader::new(reader))), + reader: Box::into_raw(Box::new(opendal_reader::new(reader, meta.content_length()))), error: std::ptr::null_mut(), }, Err(e) => opendal_result_operator_reader { diff --git a/bindings/c/src/operator_info.rs b/bindings/c/src/operator_info.rs index 6c7fc69d1ba7..c67092ad761b 100644 --- a/bindings/c/src/operator_info.rs +++ b/bindings/c/src/operator_info.rs @@ -42,12 +42,6 @@ pub struct opendal_capability { /// If operator supports read. pub read: bool, - /// If operator supports seek on returning reader. - pub read_can_seek: bool, - /// If operator supports next on returning reader. - pub read_can_next: bool, - /// If operator supports read with range. - pub read_with_range: bool, /// If operator supports read with if match. pub read_with_if_match: bool, /// If operator supports read with if none match. @@ -237,9 +231,6 @@ impl From for opendal_capability { stat_with_if_match: value.stat_with_if_match, stat_with_if_none_match: value.stat_with_if_none_match, read: value.read, - read_can_seek: value.read_can_seek, - read_can_next: value.read_can_next, - read_with_range: value.read_with_range, read_with_if_match: value.read_with_if_match, read_with_if_none_match: value.read_with_if_none_match, read_with_override_content_type: value.read_with_override_content_type, diff --git a/bindings/c/src/reader.rs b/bindings/c/src/reader.rs index 8f233dd8e4d5..5d93f7d28969 100644 --- a/bindings/c/src/reader.rs +++ b/bindings/c/src/reader.rs @@ -16,6 +16,7 @@ // under the License. use ::opendal as core; +use std::io::Read; use super::*; @@ -25,13 +26,13 @@ use super::*; /// a opendal::BlockingReader, which is inside the Rust core code. #[repr(C)] pub struct opendal_reader { - inner: *mut core::BlockingReader, + inner: *mut core::StdIoReader, } impl opendal_reader { - pub(crate) fn new(reader: core::BlockingReader) -> Self { + pub(crate) fn new(reader: core::BlockingReader, size: u64) -> Self { Self { - inner: Box::into_raw(Box::new(reader)), + inner: Box::into_raw(Box::new(reader.into_std_io_read(0..size))), } } @@ -49,15 +50,12 @@ impl opendal_reader { let buf = unsafe { std::slice::from_raw_parts_mut(buf, len) }; let inner = unsafe { &mut *(*reader).inner }; - let r = inner.read(buf.len()); - match r { - Ok(bs) => { - buf[..bs.len()].copy_from_slice(&bs); - opendal_result_reader_read { - size: bs.len(), - error: std::ptr::null_mut(), - } - } + let n = inner.read(buf); + match n { + Ok(n) => opendal_result_reader_read { + size: n, + error: std::ptr::null_mut(), + }, Err(e) => opendal_result_reader_read { size: 0, error: opendal_error::new( diff --git a/bindings/c/tests/opinfo.cpp b/bindings/c/tests/opinfo.cpp index 42c70fde2457..9684b0360473 100644 --- a/bindings/c/tests/opinfo.cpp +++ b/bindings/c/tests/opinfo.cpp @@ -65,9 +65,6 @@ TEST_F(OpendalOperatorInfoTest, CapabilityTest) EXPECT_TRUE(full_cap.blocking); EXPECT_TRUE(full_cap.read); - EXPECT_TRUE(full_cap.read_can_seek); - EXPECT_TRUE(full_cap.read_can_next); - EXPECT_TRUE(full_cap.read_with_range); EXPECT_TRUE(full_cap.stat); EXPECT_TRUE(full_cap.write); EXPECT_TRUE(full_cap.write_can_empty); @@ -78,9 +75,6 @@ TEST_F(OpendalOperatorInfoTest, CapabilityTest) EXPECT_TRUE(native_cap.blocking); EXPECT_TRUE(native_cap.read); - EXPECT_TRUE(native_cap.read_can_seek); - EXPECT_TRUE(native_cap.read_can_next); - EXPECT_TRUE(native_cap.read_with_range); EXPECT_TRUE(native_cap.stat); EXPECT_TRUE(native_cap.write); EXPECT_TRUE(native_cap.write_can_empty); From 14a9cb18dfc1422e87e34def0c42585c913cfa5d Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Fri, 22 Mar 2024 16:05:46 +0800 Subject: [PATCH 076/111] FIx java Signed-off-by: Xuanwo --- bindings/java/src/lib.rs | 5 +- .../java/org/apache/opendal/Capability.java | 91 +++++++------------ 2 files changed, 36 insertions(+), 60 deletions(-) diff --git a/bindings/java/src/lib.rs b/bindings/java/src/lib.rs index d95c0debd71e..1f6e1f9de797 100644 --- a/bindings/java/src/lib.rs +++ b/bindings/java/src/lib.rs @@ -153,15 +153,12 @@ fn make_operator_info<'a>(env: &mut JNIEnv<'a>, info: OperatorInfo) -> Result(env: &mut JNIEnv<'a>, cap: Capability) -> Result> { let capability = env.new_object( "org/apache/opendal/Capability", - "(ZZZZZZZZZZZZZZZZZZJJJZZZZZZZZZZZZZZJZ)V", + "(ZZZZZZZZZZZZZZZJJJZZZZZZZZZZZZZZJZ)V", &[ JValue::Bool(cap.stat as jboolean), JValue::Bool(cap.stat_with_if_match as jboolean), JValue::Bool(cap.stat_with_if_none_match as jboolean), JValue::Bool(cap.read as jboolean), - JValue::Bool(cap.read_can_seek as jboolean), - JValue::Bool(cap.read_can_next as jboolean), - JValue::Bool(cap.read_with_range as jboolean), JValue::Bool(cap.read_with_if_match as jboolean), JValue::Bool(cap.read_with_if_none_match as jboolean), JValue::Bool(cap.read_with_override_cache_control as jboolean), diff --git a/bindings/java/src/main/java/org/apache/opendal/Capability.java b/bindings/java/src/main/java/org/apache/opendal/Capability.java index 45a76e2b7fa4..b5ac5a988cc8 100644 --- a/bindings/java/src/main/java/org/apache/opendal/Capability.java +++ b/bindings/java/src/main/java/org/apache/opendal/Capability.java @@ -24,7 +24,7 @@ @Data public class Capability { /** - * If operator supports stat. + * If operator supports stat. */ public final boolean stat; @@ -43,21 +43,6 @@ public class Capability { */ public final boolean read; - /** - * If operator supports seek on returning reader. - */ - public final boolean readCanSeek; - - /** - * If operator supports next on returning reader. - */ - public final boolean readCanNext; - - /** - * If operator supports read with range. - */ - public final boolean readWithRange; - /** * If operator supports read with if matched. */ @@ -212,50 +197,44 @@ public class Capability { public final boolean blocking; public Capability( - boolean stat, - boolean statWithIfMatch, - boolean statWithIfNoneMatch, - boolean read, - boolean readCanSeek, - boolean readCanNext, - boolean readWithRange, - boolean readWithIfMatch, - boolean readWithIfNoneMatch, - boolean readWithOverrideCacheControl, - boolean readWithOverrideContentDisposition, - boolean readWithOverrideContentType, - boolean write, - boolean writeCanMulti, - boolean writeCanAppend, - boolean writeWithContentType, - boolean writeWithContentDisposition, - boolean writeWithCacheControl, - long writeMultiMaxSize, - long writeMultiMinSize, - long writeMultiAlignSize, - boolean createDir, - boolean delete, - boolean copy, - boolean rename, - boolean list, - boolean listWithLimit, - boolean listWithStartAfter, - boolean listWithRecursive, - boolean presign, - boolean presignRead, - boolean presignStat, - boolean presignWrite, - boolean batch, - boolean batchDelete, - long batchMaxOperations, - boolean blocking) { + boolean stat, + boolean statWithIfMatch, + boolean statWithIfNoneMatch, + boolean read, + boolean readWithIfMatch, + boolean readWithIfNoneMatch, + boolean readWithOverrideCacheControl, + boolean readWithOverrideContentDisposition, + boolean readWithOverrideContentType, + boolean write, + boolean writeCanMulti, + boolean writeCanAppend, + boolean writeWithContentType, + boolean writeWithContentDisposition, + boolean writeWithCacheControl, + long writeMultiMaxSize, + long writeMultiMinSize, + long writeMultiAlignSize, + boolean createDir, + boolean delete, + boolean copy, + boolean rename, + boolean list, + boolean listWithLimit, + boolean listWithStartAfter, + boolean listWithRecursive, + boolean presign, + boolean presignRead, + boolean presignStat, + boolean presignWrite, + boolean batch, + boolean batchDelete, + long batchMaxOperations, + boolean blocking) { this.stat = stat; this.statWithIfMatch = statWithIfMatch; this.statWithIfNoneMatch = statWithIfNoneMatch; this.read = read; - this.readCanSeek = readCanSeek; - this.readCanNext = readCanNext; - this.readWithRange = readWithRange; this.readWithIfMatch = readWithIfMatch; this.readWithIfNoneMatch = readWithIfNoneMatch; this.readWithOverrideCacheControl = readWithOverrideCacheControl; From 34344b1b079ea621263cfe038fb0fd9c0c7512bf Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Fri, 22 Mar 2024 16:14:58 +0800 Subject: [PATCH 077/111] Fix nodejs Signed-off-by: Xuanwo --- bindings/nodejs/src/capability.rs | 18 ------------ bindings/nodejs/src/lib.rs | 47 ++++++++++++++++++++++--------- 2 files changed, 33 insertions(+), 32 deletions(-) diff --git a/bindings/nodejs/src/capability.rs b/bindings/nodejs/src/capability.rs index 742f3c9a6652..b19861359ecd 100644 --- a/bindings/nodejs/src/capability.rs +++ b/bindings/nodejs/src/capability.rs @@ -60,24 +60,6 @@ impl Capability { self.0.read } - /// If operator supports seek on returning reader. - #[napi(getter)] - pub fn read_can_seek(&self) -> bool { - self.0.read_can_seek - } - - /// If operator supports next on returning reader. - #[napi(getter)] - pub fn read_can_next(&self) -> bool { - self.0.read_can_next - } - - /// If operator supports read with range. - #[napi(getter)] - pub fn read_with_range(&self) -> bool { - self.0.read_with_range - } - /// If operator supports read with if match. #[napi(getter)] pub fn read_with_if_match(&self) -> bool { diff --git a/bindings/nodejs/src/lib.rs b/bindings/nodejs/src/lib.rs index fec882b879f8..fe0b3e6c53a8 100644 --- a/bindings/nodejs/src/lib.rs +++ b/bindings/nodejs/src/lib.rs @@ -182,7 +182,10 @@ impl Operator { #[napi] pub async fn reader(&self, path: String) -> Result { let r = self.0.reader(&path).await.map_err(format_napi_error)?; - Ok(Reader(r)) + Ok(Reader { + inner: r, + offset: 0, + }) } /// Read the whole path into a buffer synchronously. @@ -203,7 +206,10 @@ impl Operator { #[napi] pub fn reader_sync(&self, path: String) -> Result { let r = self.0.blocking().reader(&path).map_err(format_napi_error)?; - Ok(BlockingReader(r)) + Ok(BlockingReader { + inner: r, + offset: 0, + }) } /// Write bytes into path. @@ -641,23 +647,33 @@ pub struct ListOptions { /// BlockingReader is designed to read data from given path in an blocking /// manner. #[napi] -pub struct BlockingReader(opendal::BlockingReader); +pub struct BlockingReader { + inner: opendal::BlockingReader, + offset: u64, +} #[napi] impl BlockingReader { #[napi] pub fn read(&mut self, mut buf: Buffer) -> Result { - let buf = buf.as_mut(); - let bs = self.0.read(buf.len()).map_err(format_napi_error)?; - buf[..bs.len()].copy_from_slice(&bs); - Ok(bs.len()) + let mut buf = buf.as_mut(); + let size = buf.len(); + let n = self + .inner + .read(&mut buf, self.offset, size) + .map_err(format_napi_error)?; + self.offset += n as u64; + Ok(n) } } /// Reader is designed to read data from given path in an asynchronous /// manner. #[napi] -pub struct Reader(opendal::Reader); +pub struct Reader { + inner: opendal::Reader, + offset: u64, +} #[napi] impl Reader { @@ -666,14 +682,17 @@ impl Reader { /// > &mut self in async napi methods should be marked as unsafe /// /// Read bytes from this reader into given buffer. - /// - /// TODO: change api into stream based. #[napi] pub async unsafe fn read(&mut self, mut buf: Buffer) -> Result { - let buf = buf.as_mut(); - let bs = self.0.read(buf.len()).await.map_err(format_napi_error)?; - buf[..bs.len()].copy_from_slice(&bs); - Ok(bs.len()) + let mut buf = buf.as_mut(); + let size = buf.len(); + let n = self + .inner + .read(&mut buf, self.offset, size) + .await + .map_err(format_napi_error)?; + self.offset += n as u64; + Ok(n) } } From 4524f291e05c000c2d3848c891dd210017e2e4ab Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Fri, 22 Mar 2024 16:36:01 +0800 Subject: [PATCH 078/111] Fix ocaml Signed-off-by: Xuanwo --- bindings/ocaml/lib/operator.ml | 11 +---------- bindings/ocaml/lib/operator.mli | 27 ++++++++++----------------- bindings/ocaml/src/operator.ml | 3 +-- bindings/ocaml/src/operator.mli | 3 +-- bindings/ocaml/src/operator/reader.rs | 18 +++++------------- bindings/ocaml/test/test.ml | 4 +--- 6 files changed, 19 insertions(+), 47 deletions(-) diff --git a/bindings/ocaml/lib/operator.ml b/bindings/ocaml/lib/operator.ml index 94841a8b39fc..5c6f6e502bf5 100644 --- a/bindings/ocaml/lib/operator.ml +++ b/bindings/ocaml/lib/operator.ml @@ -32,16 +32,7 @@ let remove = Opendal_core.Operator.blocking_remove let remove_all = Opendal_core.Operator.blocking_remove_all module Reader = struct - let read = Opendal_core.Operator.reader_read - - let seek reader pos mode = - let inner_pos = - match mode with - | Unix.SEEK_CUR -> Opendal_core.Seek_from.Current pos - | Unix.SEEK_END -> Opendal_core.Seek_from.End pos - | Unix.SEEK_SET -> Opendal_core.Seek_from.Start pos - in - Opendal_core.Operator.reader_seek reader inner_pos + let pread = Opendal_core.Operator.reader_pread end module Metadata = struct diff --git a/bindings/ocaml/lib/operator.mli b/bindings/ocaml/lib/operator.mli index 2403dadd5f8b..44bca526fdb8 100644 --- a/bindings/ocaml/lib/operator.mli +++ b/bindings/ocaml/lib/operator.mli @@ -22,7 +22,7 @@ val new_operator : (string * string) list -> (Opendal_core.Operator.operator, string) result (** [new_operator scheme config_map] Create a new block operator from given scheme and config_map. - + @param scheme Supported services, for details, refer to https://opendal.apache.org/docs/category/services/ @param config_map Configuration information required by the target service @return The block operator @@ -38,7 +38,7 @@ val stat : string -> (Opendal_core.Operator.metadata, string) result (** [is_exist operator path] Get current path's metadata **without cache** directly. - + @param operator The operator @param path want to stat @return metadata @@ -46,7 +46,7 @@ val stat : val is_exist : Opendal_core.Operator.operator -> string -> (bool, string) result (** [is_exist operator path] Check if this path exists or not. - + @param operator The operator @param path want to check @return is exists @@ -55,15 +55,15 @@ val is_exist : Opendal_core.Operator.operator -> string -> (bool, string) result val create_dir : Opendal_core.Operator.operator -> string -> (bool, string) result (** [create_dir operator path] Create a dir at given path. - + # Notes - + To indicate that a path is a directory, it is compulsory to include a trailing / in the path. Failure to do so may result in `NotADirectory` error being returned by OpenDAL. - + # Behavior - + - Create on existing dir will succeed. - Create dir is always recursive, works like `mkdir -p` @param operator The operator @@ -73,7 +73,7 @@ val create_dir : val read : Opendal_core.Operator.operator -> string -> (char array, string) result (** [read operator path] Read the whole path into a bytes. - + @param operator The operator @param path want to read @return data of path @@ -84,7 +84,7 @@ val reader : string -> (Opendal_core.Operator.reader, string) result (** [read operator path] Create a new reader which can read the whole path. - + @param operator The operator @param path want to read @return reader @@ -146,15 +146,8 @@ val remove_all : *) module Reader : sig - val read : Opendal_core.Operator.reader -> bytes -> (int, string) result + val pread : Opendal_core.Operator.reader -> bytes -> int64 -> (int, string) result (** [read reader buf] Read data to [buf] and return data size.*) - - val seek : - Opendal_core.Operator.reader -> - int64 -> - Unix.seek_command -> - (int64, string) result - (** [seek reader pos mode] is a function that seeks data to the given position [pos].*) end module Metadata : sig diff --git a/bindings/ocaml/src/operator.ml b/bindings/ocaml/src/operator.ml index 4c6e012dc5e3..91d8a16e36cf 100644 --- a/bindings/ocaml/src/operator.ml +++ b/bindings/ocaml/src/operator.ml @@ -44,5 +44,4 @@ external blocking_remove_all: operator -> string -> (unit, string) Result.t = " (* file: reader.rs *) -external reader_read: reader -> bytes -> (int, string) Result.t = "reader_read" -external reader_seek: reader -> Seek_from.seek_from -> (int64, string) Result.t = "reader_seek" +external reader_pread: reader -> bytes -> int64 -> (int, string) Result.t = "reader_pread" diff --git a/bindings/ocaml/src/operator.mli b/bindings/ocaml/src/operator.mli index 4c6e012dc5e3..91d8a16e36cf 100644 --- a/bindings/ocaml/src/operator.mli +++ b/bindings/ocaml/src/operator.mli @@ -44,5 +44,4 @@ external blocking_remove_all: operator -> string -> (unit, string) Result.t = " (* file: reader.rs *) -external reader_read: reader -> bytes -> (int, string) Result.t = "reader_read" -external reader_seek: reader -> Seek_from.seek_from -> (int64, string) Result.t = "reader_seek" +external reader_pread: reader -> bytes -> int64 -> (int, string) Result.t = "reader_pread" diff --git a/bindings/ocaml/src/operator/reader.rs b/bindings/ocaml/src/operator/reader.rs index 8ae7b4583b97..8cae9ca88ca4 100644 --- a/bindings/ocaml/src/operator/reader.rs +++ b/bindings/ocaml/src/operator/reader.rs @@ -15,20 +15,12 @@ // specific language governing permissions and limitations // under the License. -use std::io; - use super::*; #[ocaml::func] -#[ocaml::sig("reader -> bytes -> (int, string) Result.t ")] -pub fn reader_read(reader: &mut Reader, buf: &mut [u8]) -> Result { - let bs = map_res_error(reader.0.read(buf.len()))?; - buf[..bs.len()].copy_from_slice(&bs); - Ok(bs.len()) -} - -#[ocaml::func] -#[ocaml::sig("reader -> Seek_from.seek_from -> (int64, string) Result.t ")] -pub fn reader_seek(reader: &mut Reader, pos: seek_from::SeekFrom) -> Result { - map_res_error(reader.0.seek(io::SeekFrom::from(pos))) +#[ocaml::sig("reader -> bytes -> int64 -> (int, string) Result.t ")] +pub fn reader_pread(reader: &mut Reader, mut buf: &mut [u8], offset: u64) -> Result { + let size = buf.len(); + let n = map_res_error(reader.0.read(&mut buf, offset, size))?; + Ok(n) } diff --git a/bindings/ocaml/test/test.ml b/bindings/ocaml/test/test.ml index 35c23f988548..f9770d9aa0ab 100644 --- a/bindings/ocaml/test/test.ml +++ b/bindings/ocaml/test/test.ml @@ -56,10 +56,8 @@ let test_operator_reader test_ctxt = (test_check_result (Operator.write bo "tempfile" (Bytes.of_string "helloworld"))); let reader = Operator.reader bo "tempfile" |> test_check_result in - let s = Operator.Reader.seek reader 5L SEEK_CUR |> test_check_result in - assert_equal 5 (Int64.to_int s); let data = Bytes.create 5 in - let i = Operator.Reader.read reader data |> test_check_result in + let i = Operator.Reader.pread reader data 5L |> test_check_result in assert_equal 5 i; assert_equal "world" (Bytes.to_string data) From a304563d3aa5eaf46ecd7c01ea514904d7dd511a Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Fri, 22 Mar 2024 17:22:48 +0800 Subject: [PATCH 079/111] Fix python Signed-off-by: Xuanwo --- bindings/python/Cargo.toml | 1 + bindings/python/python/opendal/__init__.pyi | 63 ++++++++++++++++++++- bindings/python/src/capability.rs | 9 --- bindings/python/src/file.rs | 34 ++++++----- bindings/python/src/operator.rs | 6 +- 5 files changed, 86 insertions(+), 27 deletions(-) diff --git a/bindings/python/Cargo.toml b/bindings/python/Cargo.toml index 5d416c9cff79..b1fac6e2dd8a 100644 --- a/bindings/python/Cargo.toml +++ b/bindings/python/Cargo.toml @@ -161,6 +161,7 @@ opendal = { version = "0.45.1", path = "../../core", features = [ pyo3 = "0.20.1" pyo3-asyncio = { version = "0.20", features = ["tokio-runtime"] } tokio = "1" +bytes = "1.5.0" [target.'cfg(unix)'.dependencies.opendal] version = "0.45.1" diff --git a/bindings/python/python/opendal/__init__.pyi b/bindings/python/python/opendal/__init__.pyi index 79162f0d9ce6..abb136f3e2f5 100644 --- a/bindings/python/python/opendal/__init__.pyi +++ b/bindings/python/python/opendal/__init__.pyi @@ -19,11 +19,16 @@ from typing import AsyncIterable, Iterable, Optional from opendal.layers import Layer + class Operator: def __init__(self, scheme: str, **kwargs): ... + def layer(self, layer: Layer): ... + def open(self, path: str, mode: str) -> File: ... + def read(self, path: str) -> memoryview: ... + def write( self, path: str, @@ -34,22 +39,37 @@ class Operator: content_disposition: Optional[str] = None, cache_control: Optional[str] = None, ): ... + def stat(self, path: str) -> Metadata: ... + def create_dir(self, path: str): ... + def delete(self, path: str): ... + def list(self, path: str) -> Iterable[Entry]: ... + def scan(self, path: str) -> Iterable[Entry]: ... + def capability(self) -> Capability: ... + def copy(self, source: str, target: str): ... + def rename(self, source: str, target: str): ... + def remove_all(self, path: str): ... + def to_async_operator(self) -> AsyncOperator: ... + class AsyncOperator: def __init__(self, scheme: str, **kwargs): ... + def layer(self, layer: Layer): ... + async def open(self, path: str, mode: str) -> AsyncFile: ... + async def read(self, path: str) -> memoryview: ... + async def write( self, path: str, @@ -60,79 +80,116 @@ class AsyncOperator: content_disposition: Optional[str] = None, cache_control: Optional[str] = None, ): ... + async def stat(self, path: str) -> Metadata: ... + async def create_dir(self, path: str): ... + async def delete(self, path: str): ... + async def list(self, path: str) -> AsyncIterable[Entry]: ... + async def scan(self, path: str) -> AsyncIterable[Entry]: ... + async def presign_stat(self, path: str, expire_second: int) -> PresignedRequest: ... + async def presign_read(self, path: str, expire_second: int) -> PresignedRequest: ... + async def presign_write( self, path: str, expire_second: int ) -> PresignedRequest: ... + def capability(self) -> Capability: ... + async def copy(self, source: str, target: str): ... + async def rename(self, source: str, target: str): ... + async def remove_all(self, path: str): ... + def to_operator(self) -> Operator: ... + class File: def read(self, size: Optional[int] = None) -> memoryview: ... + def write(self, bs: bytes): ... + def seek(self, offset: int, whence: int = 0) -> int: ... + def tell(self) -> int: ... + def close(self): ... + def __enter__(self) -> File: ... + def __exit__(self, exc_type, exc_value, traceback) -> None: ... + class AsyncFile: async def read(self, size: Optional[int] = None) -> memoryview: ... + async def write(self, bs: bytes): ... + async def seek(self, offset: int, whence: int = 0) -> int: ... + async def tell(self) -> int: ... + async def close(self): ... + def __aenter__(self) -> AsyncFile: ... + def __aexit__(self, exc_type, exc_value, traceback) -> None: ... + class Entry: @property def path(self) -> str: ... + class Metadata: @property def content_disposition(self) -> Optional[str]: ... + @property def content_length(self) -> int: ... + @property def content_md5(self) -> Optional[str]: ... + @property def content_type(self) -> Optional[str]: ... + @property def etag(self) -> Optional[str]: ... + @property def mode(self) -> EntryMode: ... + class EntryMode: def is_file(self) -> bool: ... + def is_dir(self) -> bool: ... + class PresignedRequest: @property def url(self) -> str: ... + @property def method(self) -> str: ... + @property def headers(self) -> dict[str, str]: ... + class Capability: stat: bool stat_with_if_match: bool stat_with_if_none_match: bool read: bool - read_can_seek: bool - read_can_next: bool - read_with_range: bool read_with_if_match: bool read_with_if_none_match: bool read_with_override_cache_control: bool diff --git a/bindings/python/src/capability.rs b/bindings/python/src/capability.rs index 723949272dca..bcd98b32c110 100644 --- a/bindings/python/src/capability.rs +++ b/bindings/python/src/capability.rs @@ -30,12 +30,6 @@ pub struct Capability { /// If operator supports read. pub read: bool, - /// If operator supports seek on returning reader. - pub read_can_seek: bool, - /// If operator supports next on returning reader. - pub read_can_next: bool, - /// If operator supports read with range. - pub read_with_range: bool, /// If operator supports read with if match. pub read_with_if_match: bool, /// If operator supports read with if none match. @@ -126,9 +120,6 @@ impl Capability { stat_with_if_match: capability.stat_with_if_match, stat_with_if_none_match: capability.stat_with_if_none_match, read: capability.read, - read_can_seek: capability.read_can_seek, - read_can_next: capability.read_can_next, - read_with_range: capability.read_with_range, read_with_if_match: capability.read_with_if_match, read_with_if_none_match: capability.read_with_if_none_match, read_with_override_cache_control: capability.read_with_override_cache_control, diff --git a/bindings/python/src/file.rs b/bindings/python/src/file.rs index 3847eddb385c..a72719ec76eb 100644 --- a/bindings/python/src/file.rs +++ b/bindings/python/src/file.rs @@ -18,14 +18,15 @@ // Remove this `allow` after fixed. #![allow(clippy::unnecessary_fallible_conversions)] +use std::io::Read; use std::io::Seek; use std::io::SeekFrom; use std::io::Write; use std::ops::DerefMut; use std::sync::Arc; -use futures::AsyncSeekExt; use futures::AsyncWriteExt; +use futures::{AsyncReadExt, AsyncSeekExt}; use pyo3::exceptions::PyIOError; use pyo3::exceptions::PyValueError; use pyo3::prelude::*; @@ -40,14 +41,14 @@ use crate::*; pub struct File(FileState); enum FileState { - Reader(ocore::BlockingReader), + Reader(ocore::StdIoReader), Writer(ocore::BlockingWriter), Closed, } impl File { - pub fn new_reader(reader: ocore::BlockingReader) -> Self { - Self(FileState::Reader(reader)) + pub fn new_reader(reader: ocore::BlockingReader, size: u64) -> Self { + Self(FileState::Reader(reader.into_std_io_read(0..size))) } pub fn new_writer(writer: ocore::BlockingWriter) -> Self { @@ -76,10 +77,12 @@ impl File { let buffer = match size { Some(size) => { - let bs = reader - .read(size) + let mut bs = vec![0; size]; + let n = reader + .read(&mut bs) .map_err(|err| PyIOError::new_err(err.to_string()))?; - bs.to_vec() + bs.truncate(n); + bs } None => { let mut buffer = Vec::new(); @@ -202,14 +205,16 @@ impl File { pub struct AsyncFile(Arc>); enum AsyncFileState { - Reader(ocore::Reader), + Reader(ocore::FuturesIoAsyncReader), Writer(ocore::Writer), Closed, } impl AsyncFile { - pub fn new_reader(reader: ocore::Reader) -> Self { - Self(Arc::new(Mutex::new(AsyncFileState::Reader(reader)))) + pub fn new_reader(reader: ocore::Reader, size: u64) -> Self { + Self(Arc::new(Mutex::new(AsyncFileState::Reader( + reader.into_futures_io_async_read(0..size), + )))) } pub fn new_writer(writer: ocore::Writer) -> Self { @@ -241,11 +246,14 @@ impl AsyncFile { let buffer = match size { Some(size) => { - let buffer = reader - .read(size) + // TODO: optimize here by using uninit slice. + let mut bs = vec![0; size]; + let n = reader + .read(&mut bs) .await .map_err(|err| PyIOError::new_err(err.to_string()))?; - buffer.to_vec() + bs.truncate(n); + bs } None => { let mut buffer = Vec::new(); diff --git a/bindings/python/src/operator.rs b/bindings/python/src/operator.rs index e5194886e38a..32e82cdd6c54 100644 --- a/bindings/python/src/operator.rs +++ b/bindings/python/src/operator.rs @@ -80,8 +80,9 @@ impl Operator { let this = self.0.clone(); if mode == "rb" { + let meta = this.stat(&path).map_err(format_pyerr)?; let r = this.reader(&path).map_err(format_pyerr)?; - Ok(File::new_reader(r)) + Ok(File::new_reader(r, meta.content_length())) } else if mode == "wb" { let w = this.writer(&path).map_err(format_pyerr)?; Ok(File::new_writer(w)) @@ -243,8 +244,9 @@ impl AsyncOperator { future_into_py(py, async move { if mode == "rb" { + let meta = this.stat(&path).await.map_err(format_pyerr)?; let r = this.reader(&path).await.map_err(format_pyerr)?; - Ok(AsyncFile::new_reader(r)) + Ok(AsyncFile::new_reader(r, meta.content_length())) } else if mode == "wb" { let w = this.writer(&path).await.map_err(format_pyerr)?; Ok(AsyncFile::new_writer(w)) From 99c428dab9044f5fd8bc5d7076e9c745f9c6373a Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Fri, 22 Mar 2024 22:23:10 +0800 Subject: [PATCH 080/111] Fix oli Signed-off-by: Xuanwo --- bin/oay/Cargo.lock | 427 +----------------------------------- bin/oli/src/commands/cat.rs | 10 +- bin/oli/src/commands/cp.rs | 9 +- 3 files changed, 25 insertions(+), 421 deletions(-) diff --git a/bin/oay/Cargo.lock b/bin/oay/Cargo.lock index 7ed6313e9541..8c65de24ce21 100644 --- a/bin/oay/Cargo.lock +++ b/bin/oay/Cargo.lock @@ -181,9 +181,9 @@ dependencies = [ [[package]] name = "backon" -version = "0.4.1" +version = "0.4.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0c1a6197b2120bb2185a267f6515038558b019e92b832bb0320e96d66268dcf9" +checksum = "c491fa80d69c03084223a4e73c378dd9f9a1e612eb54051213f88b2d5249b458" dependencies = [ "fastrand", "futures-core", @@ -212,12 +212,6 @@ version = "0.21.7" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "9d297deb1925b89f2ccc13d7635fa0714f12c87adce1c75356b39ca9b7178567" -[[package]] -name = "base64ct" -version = "1.6.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8c3c1a368f70d6cf7302d78f8f7093da241fb8e8807c05cc9e51a125895a6d5b" - [[package]] name = "bitflags" version = "1.3.2" @@ -245,12 +239,6 @@ version = "3.14.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "7f30e7476521f6f8af1a1c4c0b8cc94f0bee37d91763d0ca2665f299b6cd8aec" -[[package]] -name = "byteorder" -version = "1.5.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1fd0f2584146f6f2ef48085050886acf353beff7305ebd1ae69500e27c67f64b" - [[package]] name = "bytes" version = "1.5.0" @@ -319,32 +307,6 @@ version = "1.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "acbf1af155f9b9ef647e42cdc158db4b64a1b61f743629225fde6f3e0be2a7c7" -[[package]] -name = "const-oid" -version = "0.9.6" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c2459377285ad874054d797f3ccebf984978aa39129f6eafde5cdc8315b612f8" - -[[package]] -name = "const-random" -version = "0.1.17" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5aaf16c9c2c612020bcfd042e170f6e32de9b9d75adb5277cdbbd2e2c8c8299a" -dependencies = [ - "const-random-macro", -] - -[[package]] -name = "const-random-macro" -version = "0.1.16" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f9d839f2a20b0aee515dc581a6172f2321f96cab76c1a38a4c584a194955390e" -dependencies = [ - "getrandom", - "once_cell", - "tiny-keccak", -] - [[package]] name = "core-foundation" version = "0.9.4" @@ -370,12 +332,6 @@ dependencies = [ "libc", ] -[[package]] -name = "crunchy" -version = "0.2.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7a81dae078cea95a014a339291cec439d2f232ebe854a9d672b796c6afafa9b7" - [[package]] name = "crypto-common" version = "0.1.6" @@ -434,17 +390,6 @@ dependencies = [ "tokio", ] -[[package]] -name = "der" -version = "0.7.8" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fffa369a668c8af7dbf8b5e56c9f744fbd399949ed171606040001947de40b1c" -dependencies = [ - "const-oid", - "pem-rfc7468", - "zeroize", -] - [[package]] name = "deranged" version = "0.3.11" @@ -461,9 +406,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "9ed9a281f7bc9b7576e61468ba615a66a5c8cfdff42420a70aa82701a3b1e292" dependencies = [ "block-buffer", - "const-oid", "crypto-common", - "subtle", ] [[package]] @@ -487,15 +430,6 @@ dependencies = [ "windows-sys 0.48.0", ] -[[package]] -name = "dlv-list" -version = "0.5.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "442039f5147480ba31067cb00ada1adae6892028e40e45fc5de7b7df6dcc1b5f" -dependencies = [ - "const-random", -] - [[package]] name = "encoding_rs" version = "0.8.33" @@ -513,12 +447,9 @@ checksum = "5443807d6dff69373d433ab9ef5378ad8df50ca6298caf15de6e52e24aaf54d5" [[package]] name = "fastrand" -version = "1.9.0" +version = "2.0.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e51093e27b0797c359783294ca4f0a911c270184cb10f85783b118614a1501be" -dependencies = [ - "instant", -] +checksum = "25cbce373ec4653f1a01a31e8a5e5ec0c622dc27ff9c4e6606eefef5cbbed4a5" [[package]] name = "flagset" @@ -718,30 +649,6 @@ version = "0.3.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "5d3d0e0f38255e7fa3cf31335b3a56f05febd18025f4db5ef7a0cfb4f8da651f" -[[package]] -name = "hex" -version = "0.4.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7f24254aa9a54b5c858eaee2f5bccdb46aaf0e486a595ed5fd8f86ba55232a70" - -[[package]] -name = "hmac" -version = "0.12.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6c49c37c09c17a53d937dfbb742eb3a961d65a994e6bcdcf37e7399d0cc8ab5e" -dependencies = [ - "digest", -] - -[[package]] -name = "home" -version = "0.5.9" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e3d1354bf6b7235cb4a0576c2619fd4ed18183f689b12b006a0ee7329eeff9a5" -dependencies = [ - "windows-sys 0.52.0", -] - [[package]] name = "htmlescape" version = "0.3.1" @@ -869,15 +776,6 @@ dependencies = [ "hashbrown", ] -[[package]] -name = "instant" -version = "0.1.12" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7a5bbe824c507c5da5956355e86a746d82e0e1464f65d862cc5e71da70e94b2c" -dependencies = [ - "cfg-if", -] - [[package]] name = "ipnet" version = "2.9.0" @@ -899,29 +797,11 @@ dependencies = [ "wasm-bindgen", ] -[[package]] -name = "jsonwebtoken" -version = "9.2.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5c7ea04a7c5c055c175f189b6dc6ba036fd62306b58c66c9f6389036c503a3f4" -dependencies = [ - "base64", - "js-sys", - "pem", - "ring", - "serde", - "serde_json", - "simple_asn1", -] - [[package]] name = "lazy_static" version = "1.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e2abad23fbc42b3700f2f279844dc832adb2b2eb069b2df918f455c4e18cc646" -dependencies = [ - "spin 0.5.2", -] [[package]] name = "libc" @@ -929,12 +809,6 @@ version = "0.2.153" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "9c198f91728a82281a64e1f4f9eeb25d82cb32a5de251c6bd1b5154d63a8e7bd" -[[package]] -name = "libm" -version = "0.2.8" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4ec2a862134d2a7d32d7983ddcdd1c4923530833c9f2ea1a44fc5fa473989058" - [[package]] name = "libredox" version = "0.0.1" @@ -1048,61 +922,12 @@ dependencies = [ "winapi", ] -[[package]] -name = "num-bigint" -version = "0.4.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "608e7659b5c3d7cba262d894801b9ec9d00de989e8a82bd4bef91d08da45cdc0" -dependencies = [ - "autocfg", - "num-integer", - "num-traits", -] - -[[package]] -name = "num-bigint-dig" -version = "0.8.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dc84195820f291c7697304f3cbdadd1cb7199c0efc917ff5eafd71225c136151" -dependencies = [ - "byteorder", - "lazy_static", - "libm", - "num-integer", - "num-iter", - "num-traits", - "rand", - "smallvec", - "zeroize", -] - [[package]] name = "num-conv" version = "0.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "51d515d32fb182ee37cda2ccdcb92950d6a3c2893aa280e540671c2cd0f3b1d9" -[[package]] -name = "num-integer" -version = "0.1.45" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "225d3389fb3509a24c93f5c29eb6bde2586b98d9f016636dff58d7c6f7569cd9" -dependencies = [ - "autocfg", - "num-traits", -] - -[[package]] -name = "num-iter" -version = "0.1.43" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7d03e6c028c5dc5cac6e2dec0efda81fc887605bb3d884578bb6d6bf7514e252" -dependencies = [ - "autocfg", - "num-integer", - "num-traits", -] - [[package]] name = "num-traits" version = "0.2.17" @@ -1110,7 +935,6 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "39e3200413f237f41ab11ad6d161bc7239c84dcb631773ccd7de3dfe4b5c267c" dependencies = [ "autocfg", - "libm", ] [[package]] @@ -1184,37 +1008,19 @@ dependencies = [ "once_cell", "percent-encoding", "quick-xml", - "reqsign", "reqwest", "serde", "serde_json", - "sha2", "tokio", "uuid", ] -[[package]] -name = "openssl-probe" -version = "0.1.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ff011a302c396a5197692431fc1948019154afc178baf7d8e37367442a4601cf" - [[package]] name = "option-ext" version = "0.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "04744f49eae99ab78e0d5c0b603ab218f515ea8cfe5a456d7629ad883a3b6e7d" -[[package]] -name = "ordered-multimap" -version = "0.7.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a4d6a8c22fc714f0c2373e6091bf6f5e9b37b1bc0b1184874b7e0a4e303d318f" -dependencies = [ - "dlv-list", - "hashbrown", -] - [[package]] name = "overload" version = "0.1.1" @@ -1244,25 +1050,6 @@ dependencies = [ "windows-targets 0.48.5", ] -[[package]] -name = "pem" -version = "3.0.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1b8fcc794035347fb64beda2d3b462595dd2753e3f268d89c5aae77e8cf2c310" -dependencies = [ - "base64", - "serde", -] - -[[package]] -name = "pem-rfc7468" -version = "0.7.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "88b39c9bfcfc231068454382784bb460aae594343fb030d46e9f50a645418412" -dependencies = [ - "base64ct", -] - [[package]] name = "percent-encoding" version = "2.3.1" @@ -1301,27 +1088,6 @@ version = "0.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "8b870d8c151b6f2fb93e84a13146138f05d02ed11c7e7c54f8826aaaf7c9f184" -[[package]] -name = "pkcs1" -version = "0.7.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c8ffb9f10fa047879315e6625af03c164b16962a5368d724ed16323b68ace47f" -dependencies = [ - "der", - "pkcs8", - "spki", -] - -[[package]] -name = "pkcs8" -version = "0.10.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f950b2377845cebe5cf8b5165cb3cc1a5e0fa5cfa3e1f7f55707d8fd82e0a7b7" -dependencies = [ - "der", - "spki", -] - [[package]] name = "powerfmt" version = "0.2.0" @@ -1456,37 +1222,6 @@ version = "0.8.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "c08c74e62047bb2de4ff487b251e4a92e24f48745648451635cec7d591162d9f" -[[package]] -name = "reqsign" -version = "0.14.7" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ed08ac3aa0676637644b1b892202f1ae789c28c15ebfa906128d111ae8086062" -dependencies = [ - "anyhow", - "async-trait", - "base64", - "chrono", - "form_urlencoded", - "getrandom", - "hex", - "hmac", - "home", - "http", - "jsonwebtoken", - "log", - "once_cell", - "percent-encoding", - "quick-xml", - "rand", - "reqwest", - "rsa", - "rust-ini", - "serde", - "serde_json", - "sha1", - "sha2", -] - [[package]] name = "reqwest" version = "0.11.24" @@ -1511,7 +1246,6 @@ dependencies = [ "percent-encoding", "pin-project-lite", "rustls", - "rustls-native-certs", "rustls-pemfile", "serde", "serde_json", @@ -1527,6 +1261,7 @@ dependencies = [ "wasm-bindgen-futures", "wasm-streams", "web-sys", + "webpki-roots", "winreg", ] @@ -1539,41 +1274,11 @@ dependencies = [ "cc", "getrandom", "libc", - "spin 0.9.8", + "spin", "untrusted", "windows-sys 0.48.0", ] -[[package]] -name = "rsa" -version = "0.9.6" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5d0e5124fcb30e76a7e79bfee683a2746db83784b86289f6251b54b7950a0dfc" -dependencies = [ - "const-oid", - "digest", - "num-bigint-dig", - "num-integer", - "num-traits", - "pkcs1", - "pkcs8", - "rand_core", - "signature", - "spki", - "subtle", - "zeroize", -] - -[[package]] -name = "rust-ini" -version = "0.20.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3e0698206bcb8882bf2a9ecb4c1e7785db57ff052297085a6efd4fe42302068a" -dependencies = [ - "cfg-if", - "ordered-multimap", -] - [[package]] name = "rustc-demangle" version = "0.1.23" @@ -1592,18 +1297,6 @@ dependencies = [ "sct", ] -[[package]] -name = "rustls-native-certs" -version = "0.6.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a9aace74cb666635c918e9c12bc0d348266037aa8eb599b5cba565709a8dff00" -dependencies = [ - "openssl-probe", - "rustls-pemfile", - "schannel", - "security-framework", -] - [[package]] name = "rustls-pemfile" version = "1.0.4" @@ -1635,15 +1328,6 @@ version = "1.0.16" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "f98d2aa92eebf49b69786be48e4477826b256916e84a57ff2a4f21923b48eb4c" -[[package]] -name = "schannel" -version = "0.1.23" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fbc91545643bcf3a0bbb6569265615222618bdf33ce4ffbbd13c4bbd4c093534" -dependencies = [ - "windows-sys 0.52.0", -] - [[package]] name = "scopeguard" version = "1.2.0" @@ -1660,29 +1344,6 @@ dependencies = [ "untrusted", ] -[[package]] -name = "security-framework" -version = "2.9.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "05b64fb303737d99b81884b2c63433e9ae28abebe5eb5045dcdd175dc2ecf4de" -dependencies = [ - "bitflags 1.3.2", - "core-foundation", - "core-foundation-sys", - "libc", - "security-framework-sys", -] - -[[package]] -name = "security-framework-sys" -version = "2.9.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e932934257d3b408ed8f30db49d85ea163bfe74961f017f405b025af298f0c7a" -dependencies = [ - "core-foundation-sys", - "libc", -] - [[package]] name = "serde" version = "1.0.197" @@ -1756,17 +1417,6 @@ dependencies = [ "digest", ] -[[package]] -name = "sha2" -version = "0.10.8" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "793db75ad2bcafc3ffa7c68b215fee268f537982cd901d132f89c6343f3a3dc8" -dependencies = [ - "cfg-if", - "cpufeatures", - "digest", -] - [[package]] name = "sharded-slab" version = "0.1.7" @@ -1776,28 +1426,6 @@ dependencies = [ "lazy_static", ] -[[package]] -name = "signature" -version = "2.2.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "77549399552de45a898a580c1b41d445bf730df867cc44e6c0233bbc4b8329de" -dependencies = [ - "digest", - "rand_core", -] - -[[package]] -name = "simple_asn1" -version = "0.6.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "adc4e5204eb1910f40f9cfa375f6f05b68c3abac4b6fd879c8ff5e7ae8a0a085" -dependencies = [ - "num-bigint", - "num-traits", - "thiserror", - "time", -] - [[package]] name = "slab" version = "0.4.9" @@ -1823,40 +1451,18 @@ dependencies = [ "windows-sys 0.48.0", ] -[[package]] -name = "spin" -version = "0.5.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6e63cff320ae2c57904679ba7cb63280a3dc4613885beafb148ee7bf9aa9042d" - [[package]] name = "spin" version = "0.9.8" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "6980e8d7511241f8acf4aebddbb1ff938df5eebe98691418c4468d0b72a96a67" -[[package]] -name = "spki" -version = "0.7.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d91ed6c858b01f942cd56b37a94b3e0a1798290327d1236e4d9cf4eaca44d29d" -dependencies = [ - "base64ct", - "der", -] - [[package]] name = "strsim" version = "0.11.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "5ee073c9e4cd00e28217186dbe12796d692868f432bf2e97ee73bed0c56dfa01" -[[package]] -name = "subtle" -version = "2.5.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "81cdd64d312baedb58e21336b31bc043b77e01cc99033ce76ef539f78e965ebc" - [[package]] name = "syn" version = "2.0.48" @@ -1956,15 +1562,6 @@ dependencies = [ "time-core", ] -[[package]] -name = "tiny-keccak" -version = "2.0.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2c9d3793400a45f954c52e73d068316d76b6f4e36977e3fcebb13a2721e80237" -dependencies = [ - "crunchy", -] - [[package]] name = "tinyvec" version = "1.6.0" @@ -2367,6 +1964,12 @@ dependencies = [ "wasm-bindgen", ] +[[package]] +name = "webpki-roots" +version = "0.25.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5f20c57d8d7db6d3b86154206ae5d8fba62dd39573114de97c2cb0578251f8e1" + [[package]] name = "winapi" version = "0.3.9" @@ -2583,9 +2186,3 @@ dependencies = [ "quote", "syn", ] - -[[package]] -name = "zeroize" -version = "1.7.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "525b4ec142c6b68a2d10f01f7bbf6755599ca3f81ea53b8431b7dd348f5fdb2d" diff --git a/bin/oli/src/commands/cat.rs b/bin/oli/src/commands/cat.rs index ff12038415f3..06f139fb3390 100644 --- a/bin/oli/src/commands/cat.rs +++ b/bin/oli/src/commands/cat.rs @@ -22,7 +22,7 @@ use anyhow::Result; use clap::Arg; use clap::ArgMatches; use clap::Command; -use tokio::io; +use futures::io; use crate::config::Config; @@ -37,9 +37,11 @@ pub async fn main(args: &ArgMatches) -> Result<()> { .ok_or_else(|| anyhow!("missing target"))?; let (op, path) = cfg.parse_location(target)?; - let mut reader = op.reader(&path).await?; - let mut stdout = io::stdout(); - io::copy(&mut reader, &mut stdout).await?; + let reader = op.reader(&path).await?; + let meta = op.stat(&path).await?; + let mut buf_reader = reader.into_futures_io_async_read(0..meta.content_length()); + let mut stdout = io::AllowStdIo::new(std::io::stdout()); + io::copy_buf(&mut buf_reader, &mut stdout).await?; Ok(()) } diff --git a/bin/oli/src/commands/cp.rs b/bin/oli/src/commands/cp.rs index 216fddbe5d5f..b433b90bb295 100644 --- a/bin/oli/src/commands/cp.rs +++ b/bin/oli/src/commands/cp.rs @@ -47,8 +47,11 @@ pub async fn main(args: &ArgMatches) -> Result<()> { if !recursive { let mut dst_w = dst_op.writer(&dst_path).await?; + let src_meta = src_op.stat(&src_path).await?; let reader = src_op.reader(&src_path).await?; - let buf_reader = futures::io::BufReader::with_capacity(8 * 1024 * 1024, reader); + let buf_reader = reader + .into_futures_io_async_read(0..src_meta.content_length()) + .with_capacity(8 * 1024 * 1024); futures::io::copy_buf(buf_reader, &mut dst_w).await?; // flush data dst_w.close().await?; @@ -70,7 +73,9 @@ pub async fn main(args: &ArgMatches) -> Result<()> { .strip_prefix(prefix) .expect("invalid path"); let reader = src_op.reader(de.path()).await?; - let buf_reader = futures::io::BufReader::with_capacity(8 * 1024 * 1024, reader); + let buf_reader = reader + .into_futures_io_async_read(0..meta.content_length()) + .with_capacity(8 * 1024 * 1024); let mut writer = dst_op.writer(&dst_root.join(fp).to_string_lossy()).await?; From b47ea51f566e8a5c3f4c328c506209d5cc8eaf0a Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Fri, 22 Mar 2024 22:26:44 +0800 Subject: [PATCH 081/111] Fix object_store Signed-off-by: Xuanwo --- integrations/object_store/src/lib.rs | 35 +++++++--------------------- 1 file changed, 9 insertions(+), 26 deletions(-) diff --git a/integrations/object_store/src/lib.rs b/integrations/object_store/src/lib.rs index 11f4e4f4f33e..dd523cc9192a 100644 --- a/integrations/object_store/src/lib.rs +++ b/integrations/object_store/src/lib.rs @@ -15,16 +15,10 @@ // specific language governing permissions and limitations // under the License. -use std::ops::Range; -use std::pin::Pin; -use std::task::Context; -use std::task::Poll; - use async_trait::async_trait; use bytes::Bytes; use futures::stream::BoxStream; use futures::FutureExt; -use futures::Stream; use futures::StreamExt; use futures::TryStreamExt; use object_store::path::Path; @@ -42,7 +36,7 @@ use opendal::Entry; use opendal::Metadata; use opendal::Metakey; use opendal::Operator; -use opendal::Reader; +use std::ops::Range; use tokio::io::AsyncWrite; #[derive(Debug)] @@ -140,8 +134,15 @@ impl ObjectStore for OpendalStore { .await .map_err(|err| format_object_store_error(err, location.as_ref()))?; + let stream = r + .into_futures_bytes_stream(0..meta.size as u64) + .map_err(|err| object_store::Error::Generic { + store: "IoError", + source: Box::new(err), + }); + Ok(GetResult { - payload: GetResultPayload::Stream(Box::pin(OpendalReader { inner: r })), + payload: GetResultPayload::Stream(Box::pin(stream)), range: (0..meta.size), meta, }) @@ -340,24 +341,6 @@ async fn try_format_object_meta(res: Result) -> Result; - - fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - let inner = Pin::new(&mut self.get_mut().inner); - inner - .poll_next(cx) - .map_err(|err| object_store::Error::Generic { - store: "IoError", - source: Box::new(err), - }) - } -} - #[cfg(test)] mod tests { use std::sync::Arc; From 0e764eae726ed3f0eb0ab09422e010a246eabf6a Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Fri, 22 Mar 2024 22:34:47 +0800 Subject: [PATCH 082/111] Fix lock Signed-off-by: Xuanwo --- core/Cargo.lock | 59 +++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 59 insertions(+) diff --git a/core/Cargo.lock b/core/Cargo.lock index 7c2153483056..b602bb8e4003 100644 --- a/core/Cargo.lock +++ b/core/Cargo.lock @@ -199,6 +199,15 @@ version = "0.7.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "96d30a06541fbafbc7f82ed10c06164cfbd2c401138f6addd8404629c4b16711" +[[package]] +name = "ascii-canvas" +version = "3.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8824ecca2e851cec16968d54a01dd372ef8f95b244fb84b84e70128be347c3c6" +dependencies = [ + "term", +] + [[package]] name = "async-backtrace" version = "0.2.6" @@ -5936,6 +5945,37 @@ dependencies = [ "quick-error", ] +[[package]] +name = "revision" +version = "0.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "87eb86913082f8976b06d07a59f17df9120e6f38b882cf3fc5a45b4499e224b6" +dependencies = [ + "bincode", + "chrono", + "geo 0.26.0", + "regex", + "revision-derive", + "roaring", + "rust_decimal", + "serde", + "thiserror", + "uuid", +] + +[[package]] +name = "revision-derive" +version = "0.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bf996fc5f61f1dbec35799b5c00c6dda12e8862e8cb782ed24e10d0292e60ed3" +dependencies = [ + "darling 0.20.5", + "proc-macro-error", + "proc-macro2", + "quote", + "syn 2.0.48", +] + [[package]] name = "rfc6979" version = "0.3.1" @@ -8520,6 +8560,25 @@ dependencies = [ "windows-sys 0.48.0", ] +[[package]] +name = "ws_stream_wasm" +version = "0.7.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7999f5f4217fe3818726b66257a4475f71e74ffd190776ad053fa159e50737f5" +dependencies = [ + "async_io_stream", + "futures", + "js-sys", + "log", + "pharos", + "rustc_version 0.4.0", + "send_wrapper", + "thiserror", + "wasm-bindgen", + "wasm-bindgen-futures", + "web-sys", +] + [[package]] name = "wyz" version = "0.5.1" From c6bf30dc17665505daf236ceb8dfee0a04605292 Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Fri, 22 Mar 2024 22:38:48 +0800 Subject: [PATCH 083/111] Format java Signed-off-by: Xuanwo --- .../java/org/apache/opendal/Capability.java | 68 +++++++++---------- 1 file changed, 34 insertions(+), 34 deletions(-) diff --git a/bindings/java/src/main/java/org/apache/opendal/Capability.java b/bindings/java/src/main/java/org/apache/opendal/Capability.java index b5ac5a988cc8..40c9eee83e2d 100644 --- a/bindings/java/src/main/java/org/apache/opendal/Capability.java +++ b/bindings/java/src/main/java/org/apache/opendal/Capability.java @@ -197,40 +197,40 @@ public class Capability { public final boolean blocking; public Capability( - boolean stat, - boolean statWithIfMatch, - boolean statWithIfNoneMatch, - boolean read, - boolean readWithIfMatch, - boolean readWithIfNoneMatch, - boolean readWithOverrideCacheControl, - boolean readWithOverrideContentDisposition, - boolean readWithOverrideContentType, - boolean write, - boolean writeCanMulti, - boolean writeCanAppend, - boolean writeWithContentType, - boolean writeWithContentDisposition, - boolean writeWithCacheControl, - long writeMultiMaxSize, - long writeMultiMinSize, - long writeMultiAlignSize, - boolean createDir, - boolean delete, - boolean copy, - boolean rename, - boolean list, - boolean listWithLimit, - boolean listWithStartAfter, - boolean listWithRecursive, - boolean presign, - boolean presignRead, - boolean presignStat, - boolean presignWrite, - boolean batch, - boolean batchDelete, - long batchMaxOperations, - boolean blocking) { + boolean stat, + boolean statWithIfMatch, + boolean statWithIfNoneMatch, + boolean read, + boolean readWithIfMatch, + boolean readWithIfNoneMatch, + boolean readWithOverrideCacheControl, + boolean readWithOverrideContentDisposition, + boolean readWithOverrideContentType, + boolean write, + boolean writeCanMulti, + boolean writeCanAppend, + boolean writeWithContentType, + boolean writeWithContentDisposition, + boolean writeWithCacheControl, + long writeMultiMaxSize, + long writeMultiMinSize, + long writeMultiAlignSize, + boolean createDir, + boolean delete, + boolean copy, + boolean rename, + boolean list, + boolean listWithLimit, + boolean listWithStartAfter, + boolean listWithRecursive, + boolean presign, + boolean presignRead, + boolean presignStat, + boolean presignWrite, + boolean batch, + boolean batchDelete, + long batchMaxOperations, + boolean blocking) { this.stat = stat; this.statWithIfMatch = statWithIfMatch; this.statWithIfNoneMatch = statWithIfNoneMatch; From 7385d73073e8f329cb8116b305559ab79c1997ae Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Fri, 22 Mar 2024 22:39:52 +0800 Subject: [PATCH 084/111] FIx nodejs Signed-off-by: Xuanwo --- bindings/nodejs/generated.d.ts | 8 -------- 1 file changed, 8 deletions(-) diff --git a/bindings/nodejs/generated.d.ts b/bindings/nodejs/generated.d.ts index a0c5329afeea..9048102520c7 100644 --- a/bindings/nodejs/generated.d.ts +++ b/bindings/nodejs/generated.d.ts @@ -62,12 +62,6 @@ export class Capability { get statWithIfNoneMatch(): boolean /** If operator supports read. */ get read(): boolean - /** If operator supports seek on returning reader. */ - get readCanSeek(): boolean - /** If operator supports next on returning reader. */ - get readCanNext(): boolean - /** If operator supports read with range. */ - get readWithRange(): boolean /** If operator supports read with if match. */ get readWithIfMatch(): boolean /** If operator supports read with if none match. */ @@ -563,8 +557,6 @@ export class Reader { * > &mut self in async napi methods should be marked as unsafe * * Read bytes from this reader into given buffer. - * - * TODO: change api into stream based. */ read(buf: Buffer): Promise } From 71dde4bae796413f2fe2259d4562905768b33a2d Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Fri, 22 Mar 2024 22:43:09 +0800 Subject: [PATCH 085/111] Fix ocaml Signed-off-by: Xuanwo --- bindings/ocaml/lib/operator.mli | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/bindings/ocaml/lib/operator.mli b/bindings/ocaml/lib/operator.mli index 44bca526fdb8..a7a31a6c4f4a 100644 --- a/bindings/ocaml/lib/operator.mli +++ b/bindings/ocaml/lib/operator.mli @@ -146,7 +146,8 @@ val remove_all : *) module Reader : sig - val pread : Opendal_core.Operator.reader -> bytes -> int64 -> (int, string) result + val pread : + Opendal_core.Operator.reader -> bytes -> int64 -> (int, string) result (** [read reader buf] Read data to [buf] and return data size.*) end From b9cc40f30c7c527957d1342d1ae841536abb9979 Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Fri, 22 Mar 2024 22:58:31 +0800 Subject: [PATCH 086/111] Fix build Signed-off-by: Xuanwo --- core/src/raw/http_util/multipart.rs | 117 +++++++++------------------- core/src/services/gcs/core.rs | 7 +- 2 files changed, 41 insertions(+), 83 deletions(-) diff --git a/core/src/raw/http_util/multipart.rs b/core/src/raw/http_util/multipart.rs index 1913add572e1..093242fb7b37 100644 --- a/core/src/raw/http_util/multipart.rs +++ b/core/src/raw/http_util/multipart.rs @@ -22,8 +22,8 @@ use std::task::ready; use std::task::Context; use std::task::Poll; -use bytes::Bytes; use bytes::BytesMut; +use bytes::{Buf, Bytes}; use http::header::CONTENT_DISPOSITION; use http::header::CONTENT_LENGTH; use http::header::CONTENT_TYPE; @@ -42,7 +42,6 @@ use super::new_request_build_error; use super::AsyncBody; use crate::raw::oio; use crate::raw::oio::Stream; -use crate::raw::oio::Streamer; use crate::*; /// Multipart is a builder for multipart/form-data. @@ -223,8 +222,7 @@ pub trait Part: Sized + 'static { pub struct FormDataPart { headers: HeaderMap, - content_length: u64, - content: Streamer, + content: Bytes, } impl FormDataPart { @@ -243,8 +241,7 @@ impl FormDataPart { Self { headers, - content_length: 0, - content: Box::new(Bytes::new()), + content: Bytes::new(), } } @@ -256,17 +253,7 @@ impl FormDataPart { /// Set the content for this part. pub fn content(mut self, content: impl Into) -> Self { - let content = content.into(); - - self.content_length = content.len() as u64; - self.content = Box::new(content); - self - } - - /// Set the stream content for this part. - pub fn stream(mut self, size: u64, content: Streamer) -> Self { - self.content_length = size; - self.content = content; + self.content = content.into(); self } } @@ -301,7 +288,7 @@ impl Part for FormDataPart { let bs = bs.freeze(); // pre-content + content + post-content (b`\r\n`) - let total_size = bs.len() as u64 + self.content_length + 2; + let total_size = bs.len() as u64 + self.content.len() as u64 + 2; ( total_size, @@ -323,23 +310,17 @@ impl Part for FormDataPart { pub struct FormDataPartStream { /// Including headers and the first `b\r\n` pre_content: Option, - content: Option, + content: Option, } impl Stream for FormDataPartStream { - fn poll_next(&mut self, cx: &mut Context<'_>) -> Poll>> { + fn poll_next(&mut self, _: &mut Context<'_>) -> Poll>> { if let Some(pre_content) = self.pre_content.take() { return Poll::Ready(Some(Ok(pre_content))); } - if let Some(stream) = self.content.as_mut() { - return match ready!(stream.poll_next(cx)) { - None => { - self.content = None; - Poll::Ready(Some(Ok(Bytes::from_static(b"\r\n")))) - } - Some(v) => Poll::Ready(Some(v)), - }; + if let Some(bs) = self.content.take() { + return Poll::Ready(Some(Ok(bs))); } Poll::Ready(None) @@ -353,8 +334,7 @@ pub struct MixedPart { /// Common version: Version, headers: HeaderMap, - content_length: u64, - content: Option, + content: Bytes, /// Request only method: Option, @@ -378,8 +358,7 @@ impl MixedPart { version: Version::HTTP_11, headers: HeaderMap::new(), - content_length: 0, - content: None, + content: Bytes::new(), uri: Some(uri), method: None, @@ -396,17 +375,11 @@ impl MixedPart { let (parts, body) = req.into_parts(); - let (content_length, content) = match body { - AsyncBody::Empty => (0, None), - AsyncBody::Bytes(bs) => (bs.len() as u64, Some(Box::new(bs) as Streamer)), - AsyncBody::Stream(stream) => { - let len = parts - .headers - .get(CONTENT_LENGTH) - .and_then(|v| v.to_str().ok()) - .and_then(|v| v.parse::().ok()) - .expect("the content length of a mixed part must be valid"); - (len, Some(stream)) + let content = match body { + AsyncBody::Empty => Bytes::new(), + AsyncBody::Bytes(bs) => bs, + AsyncBody::Stream(_) => { + unimplemented!("multipart upload does not support streaming body") } }; @@ -424,7 +397,6 @@ impl MixedPart { ), version: parts.version, headers: parts.headers, - content_length, content, method: Some(parts.method), @@ -441,16 +413,11 @@ impl MixedPart { // Swap headers directly instead of copy the entire map. mem::swap(builder.headers_mut().unwrap(), &mut self.headers); - // let body = if let Some(stream) = self.content { - // oio::Buffer::new(stream, Some(self.content_length)) - // } else { - // oio::Buffer::new(Box::new(oio::into_stream(stream::empty())), Some(0)) - // }; + let body = oio::Buffer::from(self.content); - // builder - // .body(body) - // .expect("mixed part must be valid response") - todo!() + builder + .body(body) + .expect("mixed part must be valid response") } /// Insert a part header into part. @@ -479,17 +446,7 @@ impl MixedPart { /// Set the content for this part. pub fn content(mut self, content: impl Into) -> Self { - let content = content.into(); - - self.content_length = content.len() as u64; - self.content = Some(Box::new(content)); - self - } - - /// Set the stream content for this part. - pub fn stream(mut self, size: u64, content: Streamer) -> Self { - self.content_length = size; - self.content = Some(content); + self.content = content.into(); self } } @@ -562,15 +519,15 @@ impl Part for MixedPart { // pre-content + content + post-content; let mut total_size = bs.len() as u64; - if self.content.is_some() { - total_size += self.content_length + 2; + if self.content.has_remaining() { + total_size += self.content.len() as u64 + 2; } ( total_size, MixedPartStream { pre_content: Some(bs), - content: self.content, + content: Some(self.content), }, ) } @@ -646,8 +603,7 @@ impl Part for MixedPart { part_headers, version: Version::HTTP_11, headers, - content_length: body_bytes.len() as u64, - content: Some(Box::new(body_bytes)), + content: body_bytes, method: None, uri: None, @@ -666,22 +622,21 @@ impl Part for MixedPart { pub struct MixedPartStream { /// Including headers and the first `b\r\n` pre_content: Option, - content: Option, + content: Option, } impl Stream for MixedPartStream { - fn poll_next(&mut self, cx: &mut Context<'_>) -> Poll>> { + fn poll_next(&mut self, _: &mut Context<'_>) -> Poll>> { if let Some(pre_content) = self.pre_content.take() { return Poll::Ready(Some(Ok(pre_content))); } - if let Some(stream) = self.content.as_mut() { - return match ready!(stream.poll_next(cx)) { - None => { - self.content = None; - Poll::Ready(Some(Ok(Bytes::from_static(b"\r\n")))) - } - Some(v) => Poll::Ready(Some(v)), + if let Some(bs) = self.content.as_mut() { + if bs.has_remaining() { + return Poll::Ready(Some(Ok(bs.copy_to_bytes(bs.remaining())))); + } else { + self.content = None; + return Poll::Ready(Some(Ok(Bytes::from_static(b"\r\n")))); }; } @@ -1116,7 +1071,7 @@ Content-Length: 846 h }); - assert_eq!(multipart.parts[0].content_length, part0_bs.len() as u64); + assert_eq!(multipart.parts[0].content.len(), part0_bs.len()); assert_eq!(multipart.parts[0].uri, None); assert_eq!(multipart.parts[0].method, None); assert_eq!( @@ -1156,7 +1111,7 @@ Content-Length: 846 h }); - assert_eq!(multipart.parts[1].content_length, part1_bs.len() as u64); + assert_eq!(multipart.parts[1].content.len(), part1_bs.len()); assert_eq!(multipart.parts[1].uri, None); assert_eq!(multipart.parts[1].method, None); assert_eq!( @@ -1196,7 +1151,7 @@ Content-Length: 846 h }); - assert_eq!(multipart.parts[2].content_length, part2_bs.len() as u64); + assert_eq!(multipart.parts[2].content.len(), part2_bs.len()); assert_eq!(multipart.parts[2].uri, None); assert_eq!(multipart.parts[2].method, None); assert_eq!( diff --git a/core/src/services/gcs/core.rs b/core/src/services/gcs/core.rs index bbed7951b6e9..491625d32276 100644 --- a/core/src/services/gcs/core.rs +++ b/core/src/services/gcs/core.rs @@ -291,8 +291,11 @@ impl GcsCore { AsyncBody::Bytes(bytes) => { media_part = media_part.content(bytes); } - AsyncBody::Stream(stream) => { - media_part = media_part.stream(size.unwrap(), stream); + _ => { + return Err(Error::new( + ErrorKind::Unexpected, + "multipart upload does not support streaming body", + )); } } From 583a6e795329d6a111a9e0974ee46d9617643c37 Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Fri, 22 Mar 2024 23:05:07 +0800 Subject: [PATCH 087/111] Fix test Signed-off-by: Xuanwo --- core/src/raw/http_util/multipart.rs | 47 ++++++++++++++++++++--------- 1 file changed, 32 insertions(+), 15 deletions(-) diff --git a/core/src/raw/http_util/multipart.rs b/core/src/raw/http_util/multipart.rs index 093242fb7b37..aefaedf9b3ca 100644 --- a/core/src/raw/http_util/multipart.rs +++ b/core/src/raw/http_util/multipart.rs @@ -319,8 +319,13 @@ impl Stream for FormDataPartStream { return Poll::Ready(Some(Ok(pre_content))); } - if let Some(bs) = self.content.take() { - return Poll::Ready(Some(Ok(bs))); + if let Some(bs) = self.content.as_mut() { + if bs.has_remaining() { + return Poll::Ready(Some(Ok(bs.copy_to_bytes(bs.remaining())))); + } else { + self.content = None; + return Poll::Ready(Some(Ok(Bytes::from_static(b"\r\n")))); + }; } Poll::Ready(None) @@ -334,7 +339,7 @@ pub struct MixedPart { /// Common version: Version, headers: HeaderMap, - content: Bytes, + content: Option, /// Request only method: Option, @@ -358,7 +363,7 @@ impl MixedPart { version: Version::HTTP_11, headers: HeaderMap::new(), - content: Bytes::new(), + content: None, uri: Some(uri), method: None, @@ -376,8 +381,8 @@ impl MixedPart { let (parts, body) = req.into_parts(); let content = match body { - AsyncBody::Empty => Bytes::new(), - AsyncBody::Bytes(bs) => bs, + AsyncBody::Empty => None, + AsyncBody::Bytes(bs) => Some(bs), AsyncBody::Stream(_) => { unimplemented!("multipart upload does not support streaming body") } @@ -413,7 +418,10 @@ impl MixedPart { // Swap headers directly instead of copy the entire map. mem::swap(builder.headers_mut().unwrap(), &mut self.headers); - let body = oio::Buffer::from(self.content); + let body = match self.content { + None => oio::Buffer::new(), + Some(bs) => oio::Buffer::from(bs), + }; builder .body(body) @@ -446,7 +454,7 @@ impl MixedPart { /// Set the content for this part. pub fn content(mut self, content: impl Into) -> Self { - self.content = content.into(); + self.content = Some(content.into()); self } } @@ -519,15 +527,15 @@ impl Part for MixedPart { // pre-content + content + post-content; let mut total_size = bs.len() as u64; - if self.content.has_remaining() { - total_size += self.content.len() as u64 + 2; + if let Some(bs) = &self.content { + total_size += bs.len() as u64 + 2; } ( total_size, MixedPartStream { pre_content: Some(bs), - content: Some(self.content), + content: self.content, }, ) } @@ -603,7 +611,7 @@ impl Part for MixedPart { part_headers, version: Version::HTTP_11, headers, - content: body_bytes, + content: Some(body_bytes), method: None, uri: None, @@ -1071,7 +1079,10 @@ Content-Length: 846 h }); - assert_eq!(multipart.parts[0].content.len(), part0_bs.len()); + assert_eq!( + multipart.parts[0].content.as_ref().unwrap().len(), + part0_bs.len() + ); assert_eq!(multipart.parts[0].uri, None); assert_eq!(multipart.parts[0].method, None); assert_eq!( @@ -1111,7 +1122,10 @@ Content-Length: 846 h }); - assert_eq!(multipart.parts[1].content.len(), part1_bs.len()); + assert_eq!( + multipart.parts[1].content.as_ref().unwrap().len(), + part1_bs.len() + ); assert_eq!(multipart.parts[1].uri, None); assert_eq!(multipart.parts[1].method, None); assert_eq!( @@ -1151,7 +1165,10 @@ Content-Length: 846 h }); - assert_eq!(multipart.parts[2].content.len(), part2_bs.len()); + assert_eq!( + multipart.parts[2].content.as_ref().unwrap().len(), + part2_bs.len() + ); assert_eq!(multipart.parts[2].uri, None); assert_eq!(multipart.parts[2].method, None); assert_eq!( From 8ba1c45eb03187999a47049b5ba0fe2cf4cdc111 Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Fri, 22 Mar 2024 23:07:30 +0800 Subject: [PATCH 088/111] Fix build Signed-off-by: Xuanwo --- core/src/raw/http_util/body.rs | 2 ++ 1 file changed, 2 insertions(+) diff --git a/core/src/raw/http_util/body.rs b/core/src/raw/http_util/body.rs index 068334044dae..917fc53b1f77 100644 --- a/core/src/raw/http_util/body.rs +++ b/core/src/raw/http_util/body.rs @@ -28,5 +28,7 @@ pub enum AsyncBody { /// Body with bytes. Bytes(Bytes), /// Body with stream. + /// + /// TODO: remove this variant once by adopting oio::Buffer in writing. Stream(oio::Streamer), } From fee4f8f8f21d9aa95d9bd43f8c7a0bad6104a92f Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Sat, 23 Mar 2024 00:34:19 +0800 Subject: [PATCH 089/111] Make oio::Buffer cheap to clone Signed-off-by: Xuanwo --- core/src/raw/oio/buf/buffer.rs | 103 +++++++++++++-------------------- 1 file changed, 39 insertions(+), 64 deletions(-) diff --git a/core/src/raw/oio/buf/buffer.rs b/core/src/raw/oio/buf/buffer.rs index e2e9a5260e81..9b30ba0fa15e 100644 --- a/core/src/raw/oio/buf/buffer.rs +++ b/core/src/raw/oio/buf/buffer.rs @@ -16,13 +16,12 @@ // under the License. use std::collections::VecDeque; +use std::sync::Arc; use bytes::Buf; -use bytes::BufMut; use bytes::Bytes; -use bytes::BytesMut; -/// Buffer is a wrapper of `Bytes` and `VecDeque`. +/// Buffer is a wrapper of contiguous `Bytes` and non contiguous `[Bytes]`. /// /// We designed buffer to allow underlying storage to return non-contiguous bytes. /// @@ -33,7 +32,11 @@ pub struct Buffer(Inner); #[derive(Clone)] enum Inner { Contiguous(Bytes), - NonContiguous(VecDeque), + NonContiguous { + parts: Arc<[Bytes]>, + idx: usize, + offset: usize, + }, } impl Buffer { @@ -42,7 +45,7 @@ impl Buffer { /// This operation is const and no allocation will be performed. #[inline] pub const fn new() -> Self { - Self(Inner::NonContiguous(VecDeque::new())) + Self(Inner::Contiguous(Bytes::new())) } /// Clone internal bytes to a new `Bytes`. @@ -51,29 +54,6 @@ impl Buffer { let mut bs = self.clone(); bs.copy_to_bytes(bs.remaining()) } - - /// Merge two buffer together without copying internal bytes. - pub fn merge(self, buf: Buffer) -> Self { - let mut vec = match self.0 { - Inner::Contiguous(b) => { - // NOTE: we will have at least two bytes in the vec. - let mut vec = VecDeque::with_capacity(2); - vec.push_back(b); - vec - } - Inner::NonContiguous(v) => v, - }; - - match buf.0 { - Inner::Contiguous(b) => vec.push_back(b), - Inner::NonContiguous(bs) => { - vec.reserve(bs.len()); - vec.extend(bs) - } - } - - Self(Inner::NonContiguous(vec)) - } } impl From> for Buffer { @@ -88,15 +68,25 @@ impl From for Buffer { } } +/// Transform `VecDeque` to `Arc<[Bytes]>`. impl From> for Buffer { fn from(bs: VecDeque) -> Self { - Self(Inner::NonContiguous(bs)) + Self(Inner::NonContiguous { + parts: Vec::from(bs).into(), + idx: 0, + offset: 0, + }) } } +/// Transform `Vec` to `Arc<[Bytes]>`. impl From> for Buffer { fn from(bs: Vec) -> Self { - Self(Inner::NonContiguous(bs.into())) + Self(Inner::NonContiguous { + parts: bs.into(), + idx: 0, + offset: 0, + }) } } @@ -105,7 +95,9 @@ impl Buf for Buffer { fn remaining(&self) -> usize { match &self.0 { Inner::Contiguous(b) => b.remaining(), - Inner::NonContiguous(v) => v.iter().map(|b| b.remaining()).sum(), + Inner::NonContiguous { parts, idx, offset } => { + parts[*idx..].iter().map(|p| p.len()).sum::() - offset + } } } @@ -113,53 +105,36 @@ impl Buf for Buffer { fn chunk(&self) -> &[u8] { match &self.0 { Inner::Contiguous(b) => b.chunk(), - Inner::NonContiguous(v) => { - if let Some(b) = v.front() { - b.chunk() - } else { + Inner::NonContiguous { parts, idx, offset } => { + if parts.is_empty() { &[] + } else { + &parts[*idx][*offset..] } } } } #[inline] - fn advance(&mut self, cnt: usize) { + fn advance(&mut self, mut cnt: usize) { match &mut self.0 { Inner::Contiguous(b) => b.advance(cnt), - Inner::NonContiguous(v) => { - let mut cnt = cnt; + Inner::NonContiguous { parts, idx, offset } => { while cnt > 0 { - let b = &mut v[0]; - if b.remaining() > cnt { - b.advance(cnt); - break; + let remaining = parts[*idx].len() - *offset; + if cnt < remaining { + *offset += cnt; + return; } else { - cnt -= b.remaining(); - v.remove(0); + cnt -= remaining; + *idx += 1; + *offset = 0; + if *idx >= parts.len() { + break; + } } } } } } - - #[inline] - fn copy_to_bytes(&mut self, len: usize) -> Bytes { - match &mut self.0 { - Inner::Contiguous(b) => b.copy_to_bytes(len), - Inner::NonContiguous(v) => { - if len > 0 && len <= v[0].remaining() { - let bs = v[0].copy_to_bytes(len); - if v[0].is_empty() { - v.remove(0); - } - return bs; - } - - let mut bs = BytesMut::with_capacity(len); - bs.put(self.take(len)); - bs.freeze() - } - } - } } From 3d66e0f8ff7c0885c662e598acdcf168931b1ad3 Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Mon, 25 Mar 2024 18:49:44 +0800 Subject: [PATCH 090/111] Fix empty buffer Signed-off-by: Xuanwo --- core/src/raw/http_util/client.rs | 17 +++++--- core/src/raw/oio/buf/buffer.rs | 69 +++++++++++++++++++++++++++----- 2 files changed, 71 insertions(+), 15 deletions(-) diff --git a/core/src/raw/http_util/client.rs b/core/src/raw/http_util/client.rs index 6b772da36523..a38b3f16c053 100644 --- a/core/src/raw/http_util/client.rs +++ b/core/src/raw/http_util/client.rs @@ -18,8 +18,8 @@ use std::cmp::Ordering; use std::fmt::Debug; use std::fmt::Formatter; -use std::mem; use std::str::FromStr; +use std::{future, mem}; use bytes::Buf; use bytes::Bytes; @@ -168,11 +168,16 @@ impl HttpClient { // Swap headers directly instead of copy the entire map. mem::swap(hr.headers_mut().unwrap(), resp.headers_mut()); - let bs: Vec = resp.bytes_stream().try_collect().await.map_err(|err| { - Error::new(ErrorKind::Unexpected, "read data from http response") - .with_context("url", uri.to_string()) - .set_source(err) - })?; + let bs: Vec = resp + .bytes_stream() + .try_filter(|v| future::ready(!v.is_empty())) + .try_collect() + .await + .map_err(|err| { + Error::new(ErrorKind::Unexpected, "read data from http response") + .with_context("url", uri.to_string()) + .set_source(err) + })?; let buffer = oio::Buffer::from(bs); if let Some(expect) = content_length { diff --git a/core/src/raw/oio/buf/buffer.rs b/core/src/raw/oio/buf/buffer.rs index 9b30ba0fa15e..16668da3c831 100644 --- a/core/src/raw/oio/buf/buffer.rs +++ b/core/src/raw/oio/buf/buffer.rs @@ -96,6 +96,13 @@ impl Buf for Buffer { match &self.0 { Inner::Contiguous(b) => b.remaining(), Inner::NonContiguous { parts, idx, offset } => { + assert!( + *idx <= parts.len(), + "idx larger than parts length: {:?} <= {:?}", + *idx, + parts.len(), + ); + parts[*idx..].iter().map(|p| p.len()).sum::() - offset } } @@ -106,35 +113,79 @@ impl Buf for Buffer { match &self.0 { Inner::Contiguous(b) => b.chunk(), Inner::NonContiguous { parts, idx, offset } => { - if parts.is_empty() { - &[] - } else { - &parts[*idx][*offset..] - } + assert!( + *idx <= parts.len(), + "idx larger than parts length: {:?} <= {:?}", + *idx, + parts.len(), + ); + + &parts[*idx][*offset..] } } } #[inline] fn advance(&mut self, mut cnt: usize) { + assert!( + cnt <= self.remaining(), + "cannot advance past `remaining`: {:?} <= {:?}", + cnt, + self.remaining(), + ); + match &mut self.0 { Inner::Contiguous(b) => b.advance(cnt), Inner::NonContiguous { parts, idx, offset } => { while cnt > 0 { let remaining = parts[*idx].len() - *offset; - if cnt < remaining { + if cnt <= remaining { *offset += cnt; return; } else { cnt -= remaining; *idx += 1; *offset = 0; - if *idx >= parts.len() { - break; - } } } } } } } + +#[cfg(test)] +mod tests { + use super::*; + use pretty_assertions::assert_eq; + + const EMPTY_SLICE: &[u8] = &[]; + + #[test] + fn test_contiguous_buffer() { + let buf = Buffer::new(); + + assert_eq!(buf.remaining(), 0); + assert_eq!(buf.chunk(), EMPTY_SLICE); + } + + #[test] + fn test_empty_non_contiguous_buffer() { + let buf = Buffer::from(vec![Bytes::new()]); + + assert_eq!(buf.remaining(), 0); + assert_eq!(buf.chunk(), EMPTY_SLICE); + } + + #[test] + fn test_non_contiguous_buffer_with_empty_chunks() { + let mut buf = Buffer::from(vec![Bytes::from("a")]); + + assert_eq!(buf.remaining(), 1); + assert_eq!(buf.chunk(), b"a"); + + buf.advance(1); + + assert_eq!(buf.remaining(), 0); + assert_eq!(buf.chunk(), EMPTY_SLICE); + } +} From 9bde47d5530183df1ff9b81f8e179cf10d58b31a Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Mon, 25 Mar 2024 19:22:58 +0800 Subject: [PATCH 091/111] Fix tests Signed-off-by: Xuanwo --- core/src/raw/http_util/bytes_range.rs | 5 --- core/src/raw/http_util/client.rs | 1 + core/src/raw/oio/buf/buffer.rs | 58 ++++++++++++++------------- core/src/types/reader.rs | 11 ++--- 4 files changed, 35 insertions(+), 40 deletions(-) diff --git a/core/src/raw/http_util/bytes_range.rs b/core/src/raw/http_util/bytes_range.rs index a5c7284c2a9e..b2bde391311a 100644 --- a/core/src/raw/http_util/bytes_range.rs +++ b/core/src/raw/http_util/bytes_range.rs @@ -81,11 +81,6 @@ impl BytesRange { } /// Convert bytes range into Range header. - /// - /// # NOTE - /// - /// - `bytes=-1023` means get the suffix of the file. - /// - `bytes=0-1023` means get the first 1024 bytes, we must set the end to 1023. pub fn to_header(&self) -> String { format!("bytes={self}") } diff --git a/core/src/raw/http_util/client.rs b/core/src/raw/http_util/client.rs index a38b3f16c053..6c78130c6cbc 100644 --- a/core/src/raw/http_util/client.rs +++ b/core/src/raw/http_util/client.rs @@ -178,6 +178,7 @@ impl HttpClient { .with_context("url", uri.to_string()) .set_source(err) })?; + let buffer = oio::Buffer::from(bs); if let Some(expect) = content_length { diff --git a/core/src/raw/oio/buf/buffer.rs b/core/src/raw/oio/buf/buffer.rs index 16668da3c831..ea68425a8648 100644 --- a/core/src/raw/oio/buf/buffer.rs +++ b/core/src/raw/oio/buf/buffer.rs @@ -96,12 +96,9 @@ impl Buf for Buffer { match &self.0 { Inner::Contiguous(b) => b.remaining(), Inner::NonContiguous { parts, idx, offset } => { - assert!( - *idx <= parts.len(), - "idx larger than parts length: {:?} <= {:?}", - *idx, - parts.len(), - ); + if *idx >= parts.len() { + return 0; + } parts[*idx..].iter().map(|p| p.len()).sum::() - offset } @@ -113,12 +110,9 @@ impl Buf for Buffer { match &self.0 { Inner::Contiguous(b) => b.chunk(), Inner::NonContiguous { parts, idx, offset } => { - assert!( - *idx <= parts.len(), - "idx larger than parts length: {:?} <= {:?}", - *idx, - parts.len(), - ); + if *idx >= parts.len() { + return &[]; + } &parts[*idx][*offset..] } @@ -126,28 +120,36 @@ impl Buf for Buffer { } #[inline] - fn advance(&mut self, mut cnt: usize) { - assert!( - cnt <= self.remaining(), - "cannot advance past `remaining`: {:?} <= {:?}", - cnt, - self.remaining(), - ); - + fn advance(&mut self, cnt: usize) { match &mut self.0 { Inner::Contiguous(b) => b.advance(cnt), Inner::NonContiguous { parts, idx, offset } => { - while cnt > 0 { - let remaining = parts[*idx].len() - *offset; - if cnt <= remaining { - *offset += cnt; - return; + let mut new_cnt = cnt; + let mut new_idx = *idx; + let mut new_offset = *offset; + + while new_cnt > 0 { + let remaining = parts[new_idx].len() - new_offset; + if new_cnt < remaining { + new_offset += new_cnt; + new_cnt = 0; + break; } else { - cnt -= remaining; - *idx += 1; - *offset = 0; + new_cnt -= remaining; + new_idx += 1; + new_offset = 0; + if new_idx > parts.len() { + break; + } } } + + if new_cnt == 0 { + *idx = new_idx; + *offset = new_offset; + } else { + panic!("cannot advance past {cnt} bytes") + } } } } diff --git a/core/src/types/reader.rs b/core/src/types/reader.rs index d2cad6732f14..e402f915ea33 100644 --- a/core/src/types/reader.rs +++ b/core/src/types/reader.rs @@ -97,20 +97,17 @@ impl Reader { let mut read = 0; loop { - let bs = self - .inner - // TODO: use service preferred io size instead. - .read_at_dyn(offset, size.unwrap_or(4 * 1024 * 1024) as usize) - .await?; + // TODO: use service preferred io size instead. + let limit = size.unwrap_or(4 * 1024 * 1024) as usize; + let bs = self.inner.read_at_dyn(offset, limit).await?; let n = bs.remaining(); read += n; buf.put(bs); - if n == 0 { + if n < limit { return Ok(read); } offset += n as u64; - size = size.map(|v| v - n as u64); if size == Some(0) { return Ok(read); From 553a297037157e606c83ee55aa6ed5de3d884a42 Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Mon, 25 Mar 2024 20:46:54 +0800 Subject: [PATCH 092/111] Fix sftp support Signed-off-by: Xuanwo --- core/src/services/sftp/backend.rs | 48 +++++++------------ core/src/services/sftp/mod.rs | 1 + core/src/services/sftp/reader.rs | 80 +++++++++++++++++++++++++++++++ 3 files changed, 98 insertions(+), 31 deletions(-) create mode 100644 core/src/services/sftp/reader.rs diff --git a/core/src/services/sftp/backend.rs b/core/src/services/sftp/backend.rs index 936382dd9be7..366e18d4efaa 100644 --- a/core/src/services/sftp/backend.rs +++ b/core/src/services/sftp/backend.rs @@ -22,7 +22,6 @@ use std::path::Path; use std::path::PathBuf; use async_trait::async_trait; -use bytes::Bytes; use futures::StreamExt; use log::debug; use openssh::KnownHosts; @@ -36,6 +35,7 @@ use super::error::is_sftp_protocol_error; use super::error::parse_sftp_error; use super::error::parse_ssh_error; use super::lister::SftpLister; +use super::reader::SftpReader; use super::writer::SftpWriter; use crate::raw::*; use crate::*; @@ -210,7 +210,6 @@ impl Builder for SftpBuilder { key: self.config.key.clone(), known_hosts_strategy, copyable: self.config.enable_copy, - client: tokio::sync::OnceCell::new(), }) } @@ -223,6 +222,7 @@ impl Builder for SftpBuilder { } /// Backend is used to serve `Accessor` support for sftp. +#[derive(Clone)] pub struct SftpBackend { endpoint: String, root: String, @@ -230,7 +230,6 @@ pub struct SftpBackend { key: Option, known_hosts_strategy: KnownHosts, copyable: bool, - client: tokio::sync::OnceCell, } impl Debug for SftpBackend { @@ -241,7 +240,7 @@ impl Debug for SftpBackend { #[async_trait] impl Accessor for SftpBackend { - type Reader = Bytes; + type Reader = SftpReader; type Writer = SftpWriter; type Lister = Option; type BlockingReader = (); @@ -309,18 +308,10 @@ impl Accessor for SftpBackend { } async fn read(&self, path: &str, _: OpRead) -> Result<(RpRead, Self::Reader)> { - let client = self.connect().await?; - - let mut fs = client.fs(); - fs.set_cwd(&self.root); - let path = fs.canonicalize(path).await.map_err(parse_sftp_error)?; - - let _f = client - .open(path.as_path()) - .await - .map_err(parse_sftp_error)?; - - todo!() + Ok(( + RpRead::default(), + SftpReader::new(self.clone(), self.root.clone(), path.to_owned()), + )) } async fn write(&self, path: &str, op: OpWrite) -> Result<(RpWrite, Self::Writer)> { @@ -467,21 +458,16 @@ impl Accessor for SftpBackend { } impl SftpBackend { - async fn connect(&self) -> Result<&Sftp> { - let sftp = self - .client - .get_or_try_init(|| { - Box::pin(connect_sftp( - self.endpoint.as_str(), - self.root.clone(), - self.user.clone(), - self.key.clone(), - self.known_hosts_strategy.clone(), - )) - }) - .await?; - - Ok(sftp) + /// TODO: implement connection pool in the future. + pub async fn connect(&self) -> Result { + connect_sftp( + self.endpoint.as_str(), + self.root.clone(), + self.user.clone(), + self.key.clone(), + self.known_hosts_strategy.clone(), + ) + .await } } diff --git a/core/src/services/sftp/mod.rs b/core/src/services/sftp/mod.rs index fcc74afc2951..001898171b0f 100644 --- a/core/src/services/sftp/mod.rs +++ b/core/src/services/sftp/mod.rs @@ -21,5 +21,6 @@ pub use backend::SftpConfig; mod backend; mod error; mod lister; +mod reader; mod utils; mod writer; diff --git a/core/src/services/sftp/reader.rs b/core/src/services/sftp/reader.rs new file mode 100644 index 000000000000..6f421023fe2d --- /dev/null +++ b/core/src/services/sftp/reader.rs @@ -0,0 +1,80 @@ +// 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 super::backend::SftpBackend; +use super::error::parse_sftp_error; +use crate::raw::*; +use crate::*; +use bytes::BytesMut; +use std::io::SeekFrom; +use tokio::io::AsyncSeekExt; + +pub struct SftpReader { + inner: SftpBackend, + root: String, + path: String, +} + +impl SftpReader { + pub fn new(inner: SftpBackend, root: String, path: String) -> Self { + Self { inner, root, path } + } +} + +impl oio::Read for SftpReader { + async fn read_at(&self, offset: u64, limit: usize) -> Result { + let client = self.inner.connect().await?; + + let mut fs = client.fs(); + fs.set_cwd(&self.root); + + let path = fs + .canonicalize(&self.path) + .await + .map_err(parse_sftp_error)?; + + let mut f = client + .open(path.as_path()) + .await + .map_err(parse_sftp_error)?; + + f.seek(SeekFrom::Start(offset)) + .await + .map_err(new_std_io_error)?; + + let mut size = limit; + if size == 0 { + return Ok(oio::Buffer::new()); + } + + let mut buf = BytesMut::with_capacity(limit); + while size > 0 { + let len = buf.len(); + if let Some(bytes) = f + .read(size as u32, buf.split_off(len)) + .await + .map_err(parse_sftp_error)? + { + size -= bytes.len(); + buf.unsplit(bytes); + } else { + break; + } + } + Ok(oio::Buffer::from(buf.freeze())) + } +} From 256dafe3b9d14ad737041e5af94f686099e2b248 Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Mon, 25 Mar 2024 21:13:26 +0800 Subject: [PATCH 093/111] Protect from limit == 0 Signed-off-by: Xuanwo --- .github/workflows/test_behavior.yml | 9 ++++----- core/src/layers/complete.rs | 22 ++++++++++++++++++++++ 2 files changed, 26 insertions(+), 5 deletions(-) diff --git a/.github/workflows/test_behavior.yml b/.github/workflows/test_behavior.yml index 4c65bbd4e357..5b3b9476acf9 100644 --- a/.github/workflows/test_behavior.yml +++ b/.github/workflows/test_behavior.yml @@ -27,7 +27,6 @@ on: concurrency: group: ${{ github.workflow }}-${{ github.ref }}-${{ github.event_name }} - cancel-in-progress: true jobs: plan: @@ -79,7 +78,7 @@ jobs: test_core: name: core / ${{ matrix.os }} - needs: [plan] + needs: [ plan ] if: fromJson(needs.plan.outputs.plan).components.core secrets: inherit strategy: @@ -92,7 +91,7 @@ jobs: test_binding_java: name: binding_java / ${{ matrix.os }} - needs: [plan] + needs: [ plan ] if: fromJson(needs.plan.outputs.plan).components.binding_java secrets: inherit strategy: @@ -105,7 +104,7 @@ jobs: test_binding_python: name: binding_python / ${{ matrix.os }} - needs: [plan] + needs: [ plan ] if: fromJson(needs.plan.outputs.plan).components.binding_python secrets: inherit strategy: @@ -118,7 +117,7 @@ jobs: test_binding_nodejs: name: binding_nodejs / ${{ matrix.os }} - needs: [plan] + needs: [ plan ] if: fromJson(needs.plan.outputs.plan).components.binding_nodejs secrets: inherit strategy: diff --git a/core/src/layers/complete.rs b/core/src/layers/complete.rs index a48327d0450d..3565f64800d4 100644 --- a/core/src/layers/complete.rs +++ b/core/src/layers/complete.rs @@ -586,6 +586,28 @@ impl LayeredAccessor for CompleteAccessor { pub type CompleteLister = FourWays, P>, PrefixLister

, PrefixLister, P>>>; +pub struct CompleteReader(R); + +impl oio::Read for CompleteReader { + async fn read_at(&self, offset: u64, limit: usize) -> Result { + if limit == 0 { + return Ok(oio::Buffer::new()); + } + + self.0.read_at(offset, limit).await + } +} + +impl oio::BlockingRead for CompleteReader { + fn read_at(&self, offset: u64, limit: usize) -> Result { + if limit == 0 { + return Ok(oio::Buffer::new()); + } + + self.0.read_at(offset, limit) + } +} + pub struct CompleteWriter { inner: Option, } From 6bc85c30295601302884f54f554e8ffab57f79b7 Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Mon, 25 Mar 2024 21:17:56 +0800 Subject: [PATCH 094/111] Fix complete reade Signed-off-by: Xuanwo --- core/src/layers/complete.rs | 13 +++++++++---- 1 file changed, 9 insertions(+), 4 deletions(-) diff --git a/core/src/layers/complete.rs b/core/src/layers/complete.rs index 3565f64800d4..b694c6a6258b 100644 --- a/core/src/layers/complete.rs +++ b/core/src/layers/complete.rs @@ -375,8 +375,8 @@ impl CompleteAccessor { #[cfg_attr(target_arch = "wasm32", async_trait(?Send))] impl LayeredAccessor for CompleteAccessor { type Inner = A; - type Reader = A::Reader; - type BlockingReader = A::BlockingReader; + type Reader = CompleteReader; + type BlockingReader = CompleteReader; type Writer = TwoWays, oio::ExactBufWriter>>; type BlockingWriter = CompleteWriter; @@ -405,7 +405,10 @@ impl LayeredAccessor for CompleteAccessor { if !capability.read { return Err(self.new_unsupported_error(Operation::Read)); } - self.inner.read(path, args).await + self.inner + .read(path, args) + .await + .map(|(rp, r)| (rp, CompleteReader(r))) } async fn write(&self, path: &str, args: OpWrite) -> Result<(RpWrite, Self::Writer)> { @@ -518,7 +521,9 @@ impl LayeredAccessor for CompleteAccessor { if !capability.read || !capability.blocking { return Err(self.new_unsupported_error(Operation::Read)); } - self.inner.blocking_read(path, args) + self.inner + .blocking_read(path, args) + .map(|(rp, r)| (rp, CompleteReader(r))) } fn blocking_write(&self, path: &str, args: OpWrite) -> Result<(RpWrite, Self::BlockingWriter)> { From 6e14e7b7d1891462c9369f1580291e89932c5bbc Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Mon, 25 Mar 2024 22:42:00 +0800 Subject: [PATCH 095/111] Fix alluxio Signed-off-by: Xuanwo --- core/src/services/alluxio/backend.rs | 6 +++++- core/src/services/alluxio/core.rs | 14 ++++++-------- 2 files changed, 11 insertions(+), 9 deletions(-) diff --git a/core/src/services/alluxio/backend.rs b/core/src/services/alluxio/backend.rs index 71669a8e7e89..06ad2f6a4ae1 100644 --- a/core/src/services/alluxio/backend.rs +++ b/core/src/services/alluxio/backend.rs @@ -196,7 +196,11 @@ impl Accessor for AlluxioBackend { .set_native_capability(Capability { stat: true, - read: true, + // FIXME: + // + // alluxio's read support is not implemented correctly + // We need to refactor by use [page_read](https://github.com/Alluxio/alluxio-py/blob/main/alluxio/const.py#L18) + read: false, write: true, write_can_multi: true, diff --git a/core/src/services/alluxio/core.rs b/core/src/services/alluxio/core.rs index 91dc086579d9..eb3edd5bc145 100644 --- a/core/src/services/alluxio/core.rs +++ b/core/src/services/alluxio/core.rs @@ -19,7 +19,6 @@ use std::fmt::Debug; use std::fmt::Formatter; use bytes::Buf; -use http::header::RANGE; use http::Request; use http::Response; use http::StatusCode; @@ -299,16 +298,15 @@ impl AlluxioCore { } } - pub async fn read(&self, stream_id: u64, range: BytesRange) -> Result> { - let mut req = Request::post(format!( + /// TODO: we should implement range support correctly. + /// + /// Please refer to [alluxio-py](https://github.com/Alluxio/alluxio-py/blob/main/alluxio/const.py#L18) + pub async fn read(&self, stream_id: u64, _: BytesRange) -> Result> { + let req = Request::post(format!( "{}/api/v1/streams/{}/read", - self.endpoint, stream_id + self.endpoint, stream_id, )); - if !range.is_full() { - req = req.header(RANGE, range.to_header()); - } - let req = req .body(AsyncBody::Empty) .map_err(new_request_build_error)?; From 02b0cc2dee8a0591fb0a819abfac01a07411f2e6 Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Mon, 25 Mar 2024 22:48:12 +0800 Subject: [PATCH 096/111] Fix logs Signed-off-by: Xuanwo --- core/src/layers/logging.rs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/layers/logging.rs b/core/src/layers/logging.rs index d846315dcda2..c38999db7625 100644 --- a/core/src/layers/logging.rs +++ b/core/src/layers/logging.rs @@ -1003,7 +1003,7 @@ impl oio::Read for LoggingReader { log!( target: LOGGING_TARGET, lvl, - "service={} operation={} path={} read={} -> next failed: {}", + "service={} operation={} path={} read={} -> read failed: {}", self.ctx.scheme, ReadOperation::Read, self.path, @@ -1025,7 +1025,7 @@ impl oio::BlockingRead for LoggingReader { .fetch_add(bs.remaining() as u64, Ordering::Relaxed); trace!( target: LOGGING_TARGET, - "service={} operation={} path={} read={} -> data read {}B", + "service={} operation={} path={} read={} -> read returns {}B", self.ctx.scheme, ReadOperation::BlockingRead, self.path, @@ -1039,7 +1039,7 @@ impl oio::BlockingRead for LoggingReader { log!( target: LOGGING_TARGET, lvl, - "service={} operation={} path={} read={} -> data read failed: {}", + "service={} operation={} path={} read={} -> read failed: {}", self.ctx.scheme, ReadOperation::BlockingRead, self.path, From 0a020a0cec94b1d0e316dd88a494a20e3ef31b51 Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Mon, 25 Mar 2024 22:54:51 +0800 Subject: [PATCH 097/111] Fix seafile Signed-off-by: Xuanwo --- core/src/services/seafile/core.rs | 8 +------- 1 file changed, 1 insertion(+), 7 deletions(-) diff --git a/core/src/services/seafile/core.rs b/core/src/services/seafile/core.rs index 9640b5a3531f..69fc94f3d562 100644 --- a/core/src/services/seafile/core.rs +++ b/core/src/services/seafile/core.rs @@ -232,13 +232,7 @@ impl SeafileCore { .body(AsyncBody::Empty) .map_err(new_request_build_error)?; - let resp = self.send(req).await?; - let status = resp.status(); - - match status { - StatusCode::OK => Ok(resp), - _ => Err(parse_error(resp).await?), - } + self.send(req).await } /// file detail From 94d540fd5a793183061f6623ff67f041194e0753 Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Mon, 25 Mar 2024 23:02:26 +0800 Subject: [PATCH 098/111] Fix sftp Signed-off-by: Xuanwo --- core/src/services/sftp/backend.rs | 54 +++++-------------------------- 1 file changed, 8 insertions(+), 46 deletions(-) diff --git a/core/src/services/sftp/backend.rs b/core/src/services/sftp/backend.rs index 366e18d4efaa..54cc762de88f 100644 --- a/core/src/services/sftp/backend.rs +++ b/core/src/services/sftp/backend.rs @@ -22,7 +22,6 @@ use std::path::Path; use std::path::PathBuf; use async_trait::async_trait; -use futures::StreamExt; use log::debug; use openssh::KnownHosts; use openssh::SessionBuilder; @@ -344,54 +343,17 @@ impl Accessor for SftpBackend { let mut fs = client.fs(); fs.set_cwd(&self.root); - if path.ends_with('/') { - let file_path = format!("./{}", path); - let mut dir = match fs.open_dir(&file_path).await { - Ok(dir) => dir, - Err(e) => { - if is_not_found(&e) { - return Ok(RpDelete::default()); - } else { - return Err(parse_sftp_error(e)); - } - } - } - .read_dir() - .boxed(); - - while let Some(file) = dir.next().await { - let file = file.map_err(parse_sftp_error)?; - let file_name = file.filename().to_str(); - if file_name == Some(".") || file_name == Some("..") { - continue; - } - let file_path = Path::new(&self.root).join(file.filename()); - self.delete( - file_path.to_str().ok_or(Error::new( - ErrorKind::Unexpected, - "unable to convert file path to str", - ))?, - OpDelete::default(), - ) - .await?; - } - - match fs.remove_dir(path).await { - Err(e) if !is_not_found(&e) => { - return Err(parse_sftp_error(e)); - } - _ => {} - } + let res = if path.ends_with('/') { + fs.remove_dir(path).await } else { - match fs.remove_file(path).await { - Err(e) if !is_not_found(&e) => { - return Err(parse_sftp_error(e)); - } - _ => {} - } + fs.remove_file(path).await }; - Ok(RpDelete::default()) + match res { + Ok(()) => Ok(RpDelete::default()), + Err(e) if !is_not_found(&e) => Ok(RpDelete::default()), + Err(e) => Err(parse_sftp_error(e)), + } } async fn list(&self, path: &str, _: OpList) -> Result<(RpList, Self::Lister)> { From 40897bbe8161564b87b404a3478b6bee7c1f4e3c Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Mon, 25 Mar 2024 23:13:19 +0800 Subject: [PATCH 099/111] Set fail fast to false Signed-off-by: Xuanwo --- .github/workflows/test_behavior.yml | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/.github/workflows/test_behavior.yml b/.github/workflows/test_behavior.yml index 5b3b9476acf9..57918c64102e 100644 --- a/.github/workflows/test_behavior.yml +++ b/.github/workflows/test_behavior.yml @@ -27,6 +27,7 @@ on: concurrency: group: ${{ github.workflow }}-${{ github.ref }}-${{ github.event_name }} + cancel-in-progress: true jobs: plan: @@ -82,6 +83,7 @@ jobs: if: fromJson(needs.plan.outputs.plan).components.core secrets: inherit strategy: + fail-fast: false matrix: include: ${{ fromJson(needs.plan.outputs.plan).core }} uses: ./.github/workflows/test_behavior_core.yml @@ -95,6 +97,7 @@ jobs: if: fromJson(needs.plan.outputs.plan).components.binding_java secrets: inherit strategy: + fail-fast: false matrix: include: ${{ fromJson(needs.plan.outputs.plan).binding_java }} uses: ./.github/workflows/test_behavior_binding_java.yml @@ -108,6 +111,7 @@ jobs: if: fromJson(needs.plan.outputs.plan).components.binding_python secrets: inherit strategy: + fail-fast: false matrix: include: ${{ fromJson(needs.plan.outputs.plan).binding_python }} uses: ./.github/workflows/test_behavior_binding_python.yml @@ -121,6 +125,7 @@ jobs: if: fromJson(needs.plan.outputs.plan).components.binding_nodejs secrets: inherit strategy: + fail-fast: false matrix: include: ${{ fromJson(needs.plan.outputs.plan).binding_nodejs }} uses: ./.github/workflows/test_behavior_binding_nodejs.yml From a6fac01803a07e8b5cdda9bf081c862bdfa5437c Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Mon, 25 Mar 2024 23:20:21 +0800 Subject: [PATCH 100/111] Fix typo Signed-off-by: Xuanwo --- core/src/services/sftp/backend.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/services/sftp/backend.rs b/core/src/services/sftp/backend.rs index 54cc762de88f..9fdab5405af4 100644 --- a/core/src/services/sftp/backend.rs +++ b/core/src/services/sftp/backend.rs @@ -351,7 +351,7 @@ impl Accessor for SftpBackend { match res { Ok(()) => Ok(RpDelete::default()), - Err(e) if !is_not_found(&e) => Ok(RpDelete::default()), + Err(e) if is_not_found(&e) => Ok(RpDelete::default()), Err(e) => Err(parse_sftp_error(e)), } } From 4566eb0cb690326594b64a3a23c2fcbeb23d6145 Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Mon, 25 Mar 2024 23:22:11 +0800 Subject: [PATCH 101/111] Fix nodejs test Signed-off-by: Xuanwo --- bindings/nodejs/tests/suites/async.suite.mjs | 12 ++++++------ bindings/nodejs/tests/suites/sync.suite.mjs | 12 ++++++------ 2 files changed, 12 insertions(+), 12 deletions(-) diff --git a/bindings/nodejs/tests/suites/async.suite.mjs b/bindings/nodejs/tests/suites/async.suite.mjs index b1196dda4562..813b4a269dc7 100644 --- a/bindings/nodejs/tests/suites/async.suite.mjs +++ b/bindings/nodejs/tests/suites/async.suite.mjs @@ -17,11 +17,11 @@ * under the License. */ -import { randomUUID } from 'node:crypto' -import { test } from 'vitest' -import { generateBytes, generateFixedBytes } from '../utils.mjs' -import { Readable, Writable } from 'node:stream' -import { finished, pipeline } from 'node:stream/promises' +import {randomUUID} from 'node:crypto' +import {test} from 'vitest' +import {generateBytes, generateFixedBytes} from '../utils.mjs' +import {Readable, Writable} from 'node:stream' +import {finished, pipeline} from 'node:stream/promises' export function run(op) { describe('async tests', () => { @@ -56,7 +56,7 @@ export function run(op) { await op.delete(filename) }) - test.runIf(op.capability().write)('read stream', async () => { + test.runIf(op.capability().read && op.capability().write)('read stream', async () => { let c = generateFixedBytes(3 * 1024 * 1024) const filename = `random_file_${randomUUID()}` diff --git a/bindings/nodejs/tests/suites/sync.suite.mjs b/bindings/nodejs/tests/suites/sync.suite.mjs index 38c186d57d39..b3f205099083 100644 --- a/bindings/nodejs/tests/suites/sync.suite.mjs +++ b/bindings/nodejs/tests/suites/sync.suite.mjs @@ -17,11 +17,11 @@ * under the License. */ -import { randomUUID } from 'node:crypto' -import { test } from 'vitest' -import { WriteStream, ReadStream } from '../../index.js' -import { generateFixedBytes } from '../utils.mjs' -import { Readable } from 'node:stream' +import {randomUUID} from 'node:crypto' +import {test} from 'vitest' +import {WriteStream, ReadStream} from '../../index.js' +import {generateFixedBytes} from '../utils.mjs' +import {Readable} from 'node:stream' export function run(op) { describe.runIf(op.capability().blocking)('sync tests', () => { @@ -59,7 +59,7 @@ export function run(op) { }, ) - test.runIf(op.capability().write)('blocking read stream', async () => { + test.runIf(op.capability().read && op.capability().write)('blocking read stream', async () => { let c = generateFixedBytes(3 * 1024 * 1024) const filename = `random_file_${randomUUID()}` From 93b81d1fe6cdc1541d6f4c0e52748426d7122b8b Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Mon, 25 Mar 2024 23:23:31 +0800 Subject: [PATCH 102/111] Don't fail fast Signed-off-by: Xuanwo --- .github/workflows/test_behavior_binding_java.yml | 1 + .github/workflows/test_behavior_binding_nodejs.yml | 1 + .github/workflows/test_behavior_binding_python.yml | 1 + .github/workflows/test_behavior_core.yml | 1 + 4 files changed, 4 insertions(+) diff --git a/.github/workflows/test_behavior_binding_java.yml b/.github/workflows/test_behavior_binding_java.yml index 52dd117870e9..82143fc7d153 100644 --- a/.github/workflows/test_behavior_binding_java.yml +++ b/.github/workflows/test_behavior_binding_java.yml @@ -32,6 +32,7 @@ jobs: name: ${{ matrix.cases.service }} / ${{ matrix.cases.setup }} runs-on: ${{ inputs.os }} strategy: + fail-fast: false matrix: cases: ${{ fromJson(inputs.cases) }} steps: diff --git a/.github/workflows/test_behavior_binding_nodejs.yml b/.github/workflows/test_behavior_binding_nodejs.yml index cb08366c5fcd..0dfc748226be 100644 --- a/.github/workflows/test_behavior_binding_nodejs.yml +++ b/.github/workflows/test_behavior_binding_nodejs.yml @@ -32,6 +32,7 @@ jobs: name: ${{ matrix.cases.service }} / ${{ matrix.cases.setup }} runs-on: ${{ inputs.os }} strategy: + fail-fast: false matrix: cases: ${{ fromJson(inputs.cases) }} steps: diff --git a/.github/workflows/test_behavior_binding_python.yml b/.github/workflows/test_behavior_binding_python.yml index 42fd1de81831..8f3d4bbaf48a 100644 --- a/.github/workflows/test_behavior_binding_python.yml +++ b/.github/workflows/test_behavior_binding_python.yml @@ -32,6 +32,7 @@ jobs: name: ${{ matrix.cases.service }} / ${{ matrix.cases.setup }} runs-on: ${{ inputs.os }} strategy: + fail-fast: false matrix: cases: ${{ fromJson(inputs.cases) }} steps: diff --git a/.github/workflows/test_behavior_core.yml b/.github/workflows/test_behavior_core.yml index f6831e84919c..ef49723c5ed1 100644 --- a/.github/workflows/test_behavior_core.yml +++ b/.github/workflows/test_behavior_core.yml @@ -32,6 +32,7 @@ jobs: name: ${{ matrix.cases.service }} / ${{ matrix.cases.setup }} runs-on: ${{ inputs.os }} strategy: + fail-fast: false matrix: cases: ${{ fromJson(inputs.cases) }} steps: From 506b07a8156da47eb4041857afbcbdbc079612b7 Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Mon, 25 Mar 2024 23:27:11 +0800 Subject: [PATCH 103/111] FIx format Signed-off-by: Xuanwo --- bindings/nodejs/tests/suites/async.suite.mjs | 10 +++++----- bindings/nodejs/tests/suites/sync.suite.mjs | 10 +++++----- 2 files changed, 10 insertions(+), 10 deletions(-) diff --git a/bindings/nodejs/tests/suites/async.suite.mjs b/bindings/nodejs/tests/suites/async.suite.mjs index 813b4a269dc7..56956dbd9703 100644 --- a/bindings/nodejs/tests/suites/async.suite.mjs +++ b/bindings/nodejs/tests/suites/async.suite.mjs @@ -17,11 +17,11 @@ * under the License. */ -import {randomUUID} from 'node:crypto' -import {test} from 'vitest' -import {generateBytes, generateFixedBytes} from '../utils.mjs' -import {Readable, Writable} from 'node:stream' -import {finished, pipeline} from 'node:stream/promises' +import { randomUUID } from 'node:crypto' +import { test } from 'vitest' +import { generateBytes, generateFixedBytes } from '../utils.mjs' +import { Readable, Writable } from 'node:stream' +import { finished, pipeline } from 'node:stream/promises' export function run(op) { describe('async tests', () => { diff --git a/bindings/nodejs/tests/suites/sync.suite.mjs b/bindings/nodejs/tests/suites/sync.suite.mjs index b3f205099083..a4f287e1fa95 100644 --- a/bindings/nodejs/tests/suites/sync.suite.mjs +++ b/bindings/nodejs/tests/suites/sync.suite.mjs @@ -17,11 +17,11 @@ * under the License. */ -import {randomUUID} from 'node:crypto' -import {test} from 'vitest' -import {WriteStream, ReadStream} from '../../index.js' -import {generateFixedBytes} from '../utils.mjs' -import {Readable} from 'node:stream' +import { randomUUID } from 'node:crypto' +import { test } from 'vitest' +import { WriteStream, ReadStream } from '../../index.js' +import { generateFixedBytes } from '../utils.mjs' +import { Readable } from 'node:stream' export function run(op) { describe.runIf(op.capability().blocking)('sync tests', () => { From be3523039a99b420ebdad15e533d3234063aeebd Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Tue, 26 Mar 2024 17:54:43 +0800 Subject: [PATCH 104/111] Add cap for read Signed-off-by: Xuanwo --- core/tests/behavior/async_write.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/tests/behavior/async_write.rs b/core/tests/behavior/async_write.rs index 9b3124c456dc..79077b73a4bf 100644 --- a/core/tests/behavior/async_write.rs +++ b/core/tests/behavior/async_write.rs @@ -31,7 +31,7 @@ use crate::*; pub fn tests(op: &Operator, tests: &mut Vec) { let cap = op.info().full_capability(); - if cap.write && cap.stat { + if cap.read && cap.write && cap.stat { tests.extend(async_trials!( op, test_write_only, @@ -54,7 +54,7 @@ pub fn tests(op: &Operator, tests: &mut Vec) { )) } - if cap.write && cap.write_can_append && cap.stat { + if cap.read && cap.write && cap.write_can_append && cap.stat { tests.extend(async_trials!( op, test_write_with_append, From 43272de513d3cd470739c8c185f75793f858b230 Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Tue, 26 Mar 2024 18:02:08 +0800 Subject: [PATCH 105/111] Allow ssh error to retry Signed-off-by: Xuanwo --- core/src/services/sftp/error.rs | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/core/src/services/sftp/error.rs b/core/src/services/sftp/error.rs index 4c6fd31beaad..9fd947b21b49 100644 --- a/core/src/services/sftp/error.rs +++ b/core/src/services/sftp/error.rs @@ -34,7 +34,14 @@ pub fn parse_sftp_error(e: SftpClientError) -> Error { _ => ErrorKind::Unexpected, }; - Error::new(kind, "sftp error").set_source(e) + let mut err = Error::new(kind, "sftp error").set_source(e); + + // Mark error as temporary if it's unexpected. + if kind == ErrorKind::Unexpected { + err = err.set_temporary(); + } + + err } pub fn parse_ssh_error(e: SshError) -> Error { From bdc6c2d72a4a0ec572422bc9a8d59dd913ae5b35 Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Tue, 26 Mar 2024 18:22:22 +0800 Subject: [PATCH 106/111] Disable chainsafe Signed-off-by: Xuanwo --- .../chainsafe/chainsafe/{action.yml => disable_action.yml} | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename .github/services/chainsafe/chainsafe/{action.yml => disable_action.yml} (100%) diff --git a/.github/services/chainsafe/chainsafe/action.yml b/.github/services/chainsafe/chainsafe/disable_action.yml similarity index 100% rename from .github/services/chainsafe/chainsafe/action.yml rename to .github/services/chainsafe/chainsafe/disable_action.yml From 6a62b033a39357ac606529e95c4d7b67b353fa43 Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Tue, 26 Mar 2024 19:02:49 +0800 Subject: [PATCH 107/111] Implement connection pool for sftp Signed-off-by: Xuanwo --- .github/services/sftp/sftp/action.yml | 1 + core/Cargo.toml | 2 +- core/src/services/sftp/backend.rs | 171 ++++++++++++++++---------- 3 files changed, 108 insertions(+), 66 deletions(-) diff --git a/.github/services/sftp/sftp/action.yml b/.github/services/sftp/sftp/action.yml index c3586ca3ab0d..08fd7b9aa2db 100644 --- a/.github/services/sftp/sftp/action.yml +++ b/.github/services/sftp/sftp/action.yml @@ -38,3 +38,4 @@ runs: OPENDAL_SFTP_KEY=${{ github.workspace }}/fixtures/sftp/test_ssh_key OPENDAL_SFTP_KNOWN_HOSTS_STRATEGY=accept EOF + diff --git a/core/Cargo.toml b/core/Cargo.toml index ae656920ce3d..b40282fceb8b 100644 --- a/core/Cargo.toml +++ b/core/Cargo.toml @@ -173,7 +173,7 @@ services-s3 = [ "reqsign?/reqwest_request", ] services-seafile = [] -services-sftp = ["dep:openssh", "dep:openssh-sftp-client"] +services-sftp = ["dep:openssh", "dep:openssh-sftp-client", "dep:bb8"] services-sled = ["dep:sled", "internal-tokio-rt"] services-sqlite = ["dep:rusqlite", "dep:r2d2", "internal-tokio-rt"] services-supabase = [] diff --git a/core/src/services/sftp/backend.rs b/core/src/services/sftp/backend.rs index 9fdab5405af4..a855523a64c0 100644 --- a/core/src/services/sftp/backend.rs +++ b/core/src/services/sftp/backend.rs @@ -22,12 +22,14 @@ use std::path::Path; use std::path::PathBuf; use async_trait::async_trait; +use bb8::{PooledConnection, RunError}; use log::debug; use openssh::KnownHosts; use openssh::SessionBuilder; use openssh_sftp_client::Sftp; use openssh_sftp_client::SftpOptions; use serde::Deserialize; +use tokio::sync::OnceCell; use super::error::is_not_found; use super::error::is_sftp_protocol_error; @@ -39,7 +41,7 @@ use super::writer::SftpWriter; use crate::raw::*; use crate::*; -/// Config for Sftpservices support. +/// Config for Sftp Service support. #[derive(Default, Deserialize)] #[serde(default)] #[non_exhaustive] @@ -209,6 +211,8 @@ impl Builder for SftpBuilder { key: self.config.key.clone(), known_hosts_strategy, copyable: self.config.enable_copy, + + client: OnceCell::new(), }) } @@ -223,12 +227,85 @@ impl Builder for SftpBuilder { /// Backend is used to serve `Accessor` support for sftp. #[derive(Clone)] pub struct SftpBackend { + copyable: bool, + endpoint: String, + root: String, + user: Option, + key: Option, + known_hosts_strategy: KnownHosts, + + client: OnceCell>, +} + +pub struct Manager { endpoint: String, root: String, user: Option, key: Option, known_hosts_strategy: KnownHosts, - copyable: bool, +} + +#[async_trait] +impl bb8::ManageConnection for Manager { + type Connection = Sftp; + type Error = Error; + + async fn connect(&self) -> std::result::Result { + let mut session = SessionBuilder::default(); + + if let Some(user) = &self.user { + session.user(user.clone()); + } + + if let Some(key) = &self.key { + session.keyfile(key); + } + + session.known_hosts_check(self.known_hosts_strategy.clone()); + + let session = session + .connect(&self.endpoint) + .await + .map_err(parse_ssh_error)?; + + let sftp = Sftp::from_session(session, SftpOptions::default()) + .await + .map_err(parse_sftp_error)?; + + if !self.root.is_empty() { + let mut fs = sftp.fs(); + + let paths = Path::new(&self.root).components(); + let mut current = PathBuf::new(); + for p in paths { + current.push(p); + let res = fs.create_dir(p).await; + + if let Err(e) = res { + // ignore error if dir already exists + if !is_sftp_protocol_error(&e) { + return Err(parse_sftp_error(e)); + } + } + fs.set_cwd(¤t); + } + } + + debug!("sftp connection created at {}", self.root); + Ok(sftp) + } + + // Check if connect valid by checking the root path. + async fn is_valid(&self, conn: &mut Self::Connection) -> std::result::Result<(), Self::Error> { + let _ = conn.fs().metadata("./").await.map_err(parse_sftp_error)?; + + Ok(()) + } + + /// Always allow reuse conn. + fn has_broken(&self, _: &mut Self::Connection) -> bool { + false + } } impl Debug for SftpBackend { @@ -237,6 +314,33 @@ impl Debug for SftpBackend { } } +impl SftpBackend { + pub async fn connect(&self) -> Result> { + let client = self + .client + .get_or_try_init(|| async { + bb8::Pool::builder() + .max_size(64) + .build(Manager { + endpoint: self.endpoint.clone(), + root: self.root.clone(), + user: self.user.clone(), + key: self.key.clone(), + known_hosts_strategy: self.known_hosts_strategy.clone(), + }) + .await + }) + .await?; + + client.get_owned().await.map_err(|err| match err { + RunError::User(err) => err, + RunError::TimedOut => { + Error::new(ErrorKind::Unexpected, "connection request: timeout").set_temporary() + } + }) + } +} + #[async_trait] impl Accessor for SftpBackend { type Reader = SftpReader; @@ -418,66 +522,3 @@ impl Accessor for SftpBackend { Ok(RpRename::default()) } } - -impl SftpBackend { - /// TODO: implement connection pool in the future. - pub async fn connect(&self) -> Result { - connect_sftp( - self.endpoint.as_str(), - self.root.clone(), - self.user.clone(), - self.key.clone(), - self.known_hosts_strategy.clone(), - ) - .await - } -} - -async fn connect_sftp( - endpoint: &str, - root: String, - user: Option, - key: Option, - known_hosts_strategy: KnownHosts, -) -> Result { - let mut session = SessionBuilder::default(); - - if let Some(user) = user { - session.user(user); - } - - if let Some(key) = &key { - session.keyfile(key); - } - - session.known_hosts_check(known_hosts_strategy); - - let session = session.connect(&endpoint).await.map_err(parse_ssh_error)?; - - let sftp = Sftp::from_session(session, SftpOptions::default()) - .await - .map_err(parse_sftp_error)?; - - if !root.is_empty() { - let mut fs = sftp.fs(); - - let paths = Path::new(&root).components(); - let mut current = PathBuf::new(); - for p in paths { - current.push(p); - let res = fs.create_dir(p).await; - - if let Err(e) = res { - // ignore error if dir already exists - if !is_sftp_protocol_error(&e) { - return Err(parse_sftp_error(e)); - } - } - fs.set_cwd(¤t); - } - } - - debug!("sftp connection created at {}", root); - - Ok(sftp) -} From 4c6208dd89e9614dea8cd06a1d60ee4076fc6b74 Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Tue, 26 Mar 2024 19:14:26 +0800 Subject: [PATCH 108/111] fix retry reader Signed-off-by: Xuanwo --- core/src/layers/retry.rs | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/core/src/layers/retry.rs b/core/src/layers/retry.rs index b2bd2d791fa4..43ed81f8671a 100644 --- a/core/src/layers/retry.rs +++ b/core/src/layers/retry.rs @@ -1044,13 +1044,11 @@ mod tests { Error::new(ErrorKind::Unexpected, "retryable_error from reader") .set_temporary(), ), - 2 => Ok(Bytes::copy_from_slice("Hello, ".as_bytes()).into()), - 3 => Err( + 2 => Err( Error::new(ErrorKind::Unexpected, "retryable_error from reader") .set_temporary(), ), - 4 => Ok(Bytes::copy_from_slice("World!".as_bytes()).into()), - 5 => Ok(Bytes::new().into()), + 3 => Ok(Bytes::copy_from_slice("Hello, World!".as_bytes()).into()), _ => unreachable!(), } } @@ -1116,8 +1114,8 @@ mod tests { .expect("read must succeed"); assert_eq!(size, 13); assert_eq!(content, "Hello, World!".as_bytes()); - // The error is retryable, we should request it 1 + 10 times. - assert_eq!(*builder.attempt.lock().unwrap(), 5); + // The error is retryable, we should request it 3 times. + assert_eq!(*builder.attempt.lock().unwrap(), 3); } #[tokio::test] From 0ee9a56ddc39adb82ed2f2410768f056faebeec0 Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Tue, 26 Mar 2024 19:17:04 +0800 Subject: [PATCH 109/111] Fix nodejs test Signed-off-by: Xuanwo --- bindings/nodejs/tests/suites/services.suite.mjs | 1 - bindings/nodejs/tests/suites/sync.suite.mjs | 2 +- 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/bindings/nodejs/tests/suites/services.suite.mjs b/bindings/nodejs/tests/suites/services.suite.mjs index 031257f9f0a7..004a8d22474b 100644 --- a/bindings/nodejs/tests/suites/services.suite.mjs +++ b/bindings/nodejs/tests/suites/services.suite.mjs @@ -20,7 +20,6 @@ export function run(operator) { test('get capability', () => { assert.ok(operator.capability()) - assert.ok(operator.capability().read) }) test('try to non-exist capability', () => { diff --git a/bindings/nodejs/tests/suites/sync.suite.mjs b/bindings/nodejs/tests/suites/sync.suite.mjs index a4f287e1fa95..43a7224fffcf 100644 --- a/bindings/nodejs/tests/suites/sync.suite.mjs +++ b/bindings/nodejs/tests/suites/sync.suite.mjs @@ -35,7 +35,7 @@ export function run(op) { } }) - test.runIf(op.capability().write && op.capability().writeCanMulti)( + test.runIf(op.capability().read && op.capability().write && op.capability().writeCanMulti)( 'blocking reader/writer stream pipeline', async () => { const filename = `random_file_${randomUUID()}` From 522ac745d654276869be2de659187befea564f91 Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Tue, 26 Mar 2024 19:31:11 +0800 Subject: [PATCH 110/111] Fix doc tests Signed-off-by: Xuanwo --- core/src/types/operator/blocking_operator.rs | 2 +- core/src/types/operator/operator.rs | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/types/operator/blocking_operator.rs b/core/src/types/operator/blocking_operator.rs index 36100de0322c..bfb592443402 100644 --- a/core/src/types/operator/blocking_operator.rs +++ b/core/src/types/operator/blocking_operator.rs @@ -433,7 +433,7 @@ impl BlockingOperator { /// use opendal::EntryMode; /// use opendal::Metakey; /// # fn test(op: BlockingOperator) -> Result<()> { - /// let r = op.reader_with("path/to/file").range(0..10).call()?; + /// let r = op.reader_with("path/to/file").version("version_id").call()?; /// # Ok(()) /// # } /// ``` diff --git a/core/src/types/operator/operator.rs b/core/src/types/operator/operator.rs index 4f25e4a102bf..3f3e8fecc0a5 100644 --- a/core/src/types/operator/operator.rs +++ b/core/src/types/operator/operator.rs @@ -564,7 +564,7 @@ impl Operator { /// # use opendal::Operator; /// # use opendal::Scheme; /// # async fn test(op: Operator) -> Result<()> { - /// let r = op.reader_with("path/to/file").range(0..10).await?; + /// let r = op.reader_with("path/to/file").version("version_id").await?; /// # Ok(()) /// # } /// ``` From eea26dfd8bac583a4efb97f87838ca523ee6de56 Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Tue, 26 Mar 2024 19:37:44 +0800 Subject: [PATCH 111/111] FIx nodejs Signed-off-by: Xuanwo --- bindings/nodejs/tests/suites/async.suite.mjs | 35 +++++++++++--------- 1 file changed, 19 insertions(+), 16 deletions(-) diff --git a/bindings/nodejs/tests/suites/async.suite.mjs b/bindings/nodejs/tests/suites/async.suite.mjs index 56956dbd9703..b021c273d781 100644 --- a/bindings/nodejs/tests/suites/async.suite.mjs +++ b/bindings/nodejs/tests/suites/async.suite.mjs @@ -35,26 +35,29 @@ export function run(op) { } }) - test.runIf(op.capability().write && op.capability().writeCanMulti)('reader/writer stream pipeline', async () => { - const filename = `random_file_${randomUUID()}` - const buf = generateFixedBytes(5 * 1024 * 1024) - const rs = Readable.from(buf, { - highWaterMark: 5 * 1024 * 1024, // to buffer 5MB data to read - }) - const w = await op.writer(filename) - const ws = w.createWriteStream() - await pipeline(rs, ws) + test.runIf(op.capability().read && op.capability().write && op.capability().writeCanMulti)( + 'reader/writer stream pipeline', + async () => { + const filename = `random_file_${randomUUID()}` + const buf = generateFixedBytes(5 * 1024 * 1024) + const rs = Readable.from(buf, { + highWaterMark: 5 * 1024 * 1024, // to buffer 5MB data to read + }) + const w = await op.writer(filename) + const ws = w.createWriteStream() + await pipeline(rs, ws) - await finished(ws) + await finished(ws) - const t = await op.stat(filename) - assert.equal(t.contentLength, buf.length) + const t = await op.stat(filename) + assert.equal(t.contentLength, buf.length) - const content = await op.read(filename) - assert.equal(Buffer.compare(content, buf), 0) // 0 means equal + const content = await op.read(filename) + assert.equal(Buffer.compare(content, buf), 0) // 0 means equal - await op.delete(filename) - }) + await op.delete(filename) + }, + ) test.runIf(op.capability().read && op.capability().write)('read stream', async () => { let c = generateFixedBytes(3 * 1024 * 1024)