Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

limit parallelization when reading parquet columns. #4283

Merged
merged 1 commit into from
Mar 1, 2022
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
17 changes: 13 additions & 4 deletions query/src/storages/fuse/io/block_reader.rs
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@ use common_arrow::arrow::datatypes::Field;
use common_arrow::arrow::datatypes::Schema as ArrowSchema;
use common_arrow::arrow::io::parquet::read::read_columns_many_async;
use common_arrow::arrow::io::parquet::read::RowGroupDeserializer;
use common_base::tokio::sync::Semaphore;
use common_datablocks::DataBlock;
use common_datavalues::prelude::*;
use common_exception::ErrorCode;
Expand All @@ -28,6 +29,7 @@ use common_tracing::tracing::Instrument;
use futures::future::BoxFuture;
use opendal::Operator;

use super::parallel_async_reader::ParallelAsyncReader;
use crate::storages::fuse::io::meta_readers::BlockMetaReader;

pub struct BlockReader {
Expand Down Expand Up @@ -96,14 +98,21 @@ impl BlockReader {
})
.collect();

// todo(youngsofun): make this a config
let semaphore = Arc::new(Semaphore::new(2));
let factory = || {
let data_accessor = self.data_accessor.clone();
let path = self.path.clone();
let semaphore = semaphore.clone();
Box::pin(async move {
Ok(data_accessor
.object(path.as_str())
.reader()
.total_size(stream_len))
let permit = semaphore.clone().acquire_owned().await.unwrap();
Ok(ParallelAsyncReader::new(
permit,
data_accessor
.object(path.as_str())
.reader()
.total_size(stream_len),
))
}) as BoxFuture<_>
};

Expand Down
1 change: 1 addition & 0 deletions query/src/storages/fuse/io/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@ mod block_stream_writer;
mod block_writer;
mod locations;
mod meta_readers;
mod parallel_async_reader;

pub use block_reader::BlockReader;
pub use block_stream_writer::BlockRegulator;
Expand Down
81 changes: 81 additions & 0 deletions query/src/storages/fuse/io/parallel_async_reader.rs
Original file line number Diff line number Diff line change
@@ -0,0 +1,81 @@
// Copyright 2022 Datafuse Labs.
//
// Licensed 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::IoSliceMut;
use std::io::SeekFrom;
use std::ops::Deref;
use std::ops::DerefMut;
use std::pin::Pin;
use std::task::Context;
use std::task::Poll;

use common_base::tokio::sync::OwnedSemaphorePermit;
use futures::AsyncRead;
use futures::AsyncSeek;

pub struct ParallelAsyncReader<T: ?Sized> {
_permit: OwnedSemaphorePermit,
value: T,
}

impl<T: Sized> ParallelAsyncReader<T> {
pub fn new(permit: OwnedSemaphorePermit, value: T) -> Self {
ParallelAsyncReader {
_permit: permit,
value,
}
}
}

impl<T: ?Sized> Deref for ParallelAsyncReader<T> {
type Target = T;

fn deref(&self) -> &Self::Target {
&self.value
}
}

impl<T: ?Sized> DerefMut for ParallelAsyncReader<T> {
fn deref_mut(&mut self) -> &mut Self::Target {
&mut self.value
}
}

impl<T: ?Sized + AsyncRead + Unpin> AsyncRead for ParallelAsyncReader<T> {
fn poll_read(
mut self: Pin<&mut Self>,
cx: &mut Context<'_>,
buf: &mut [u8],
) -> Poll<std::io::Result<usize>> {
Pin::new(&mut **self).poll_read(cx, buf)
}

fn poll_read_vectored(
mut self: Pin<&mut Self>,
cx: &mut Context<'_>,
bufs: &mut [IoSliceMut<'_>],
) -> Poll<std::io::Result<usize>> {
Pin::new(&mut **self).poll_read_vectored(cx, bufs)
}
}

impl<T: ?Sized + AsyncSeek + Unpin> AsyncSeek for ParallelAsyncReader<T> {
fn poll_seek(
mut self: Pin<&mut Self>,
cx: &mut Context<'_>,
pos: SeekFrom,
) -> Poll<std::io::Result<u64>> {
Pin::new(&mut **self).poll_seek(cx, pos)
}
}