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

refactor(query): csv reader support prefetch #14983

Merged
merged 7 commits into from
Mar 19, 2024
Merged
Show file tree
Hide file tree
Changes from 6 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
3 changes: 3 additions & 0 deletions src/query/pipeline/sources/src/lib.rs
Original file line number Diff line number Diff line change
Expand Up @@ -28,12 +28,15 @@ mod sync_source;
mod sync_source_receiver;

pub mod input_formats;
mod prefetch_async_source;

pub use async_source::AsyncSource;
pub use async_source::AsyncSourcer;
pub use blocks_source::BlocksSource;
pub use empty_source::EmptySource;
pub use one_block_source::OneBlockSource;
pub use prefetch_async_source::PrefetchAsyncSource;
pub use prefetch_async_source::PrefetchAsyncSourcer;
pub use stream_source::AsyncStreamSource;
pub use stream_source::StreamSource;
pub use stream_source::StreamSourceNoSkipEmpty;
Expand Down
142 changes: 142 additions & 0 deletions src/query/pipeline/sources/src/prefetch_async_source.rs
Original file line number Diff line number Diff line change
@@ -0,0 +1,142 @@
// Copyright 2021 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::any::Any;
use std::sync::Arc;

use databend_common_base::base::Progress;
use databend_common_base::base::ProgressValues;
use databend_common_base::runtime::profile::Profile;
use databend_common_base::runtime::profile::ProfileStatisticsName;
use databend_common_catalog::table_context::TableContext;
use databend_common_exception::Result;
use databend_common_expression::DataBlock;
use databend_common_pipeline_core::processors::Event;
use databend_common_pipeline_core::processors::EventCause;
use databend_common_pipeline_core::processors::OutputPort;
use databend_common_pipeline_core::processors::Processor;
use databend_common_pipeline_core::processors::ProcessorPtr;

#[async_trait::async_trait]
pub trait PrefetchAsyncSource: Send {
const NAME: &'static str;
const SKIP_EMPTY_DATA_BLOCK: bool = true;

#[async_trait::unboxed_simple]
async fn generate(&mut self) -> Result<Option<DataBlock>>;
fn is_full(&self, prefetched: &[DataBlock]) -> bool;

fn un_reacted(&self) -> Result<()> {
Ok(())
}
}

// TODO: This can be refactored using proc macros
// TODO: Most of its current code is consistent with sync. We need refactor this with better async
// scheduling after supported expand processors. It will be implemented using a similar dynamic window.
pub struct PrefetchAsyncSourcer<T: 'static + PrefetchAsyncSource> {
is_inner_finish: bool,

inner: T,
output: Arc<OutputPort>,
scan_progress: Arc<Progress>,
generated_data: Vec<DataBlock>,
}

impl<T: 'static + PrefetchAsyncSource> PrefetchAsyncSourcer<T> {
pub fn create(
ctx: Arc<dyn TableContext>,
output: Arc<OutputPort>,
inner: T,
) -> Result<ProcessorPtr> {
let scan_progress = ctx.get_scan_progress();
Ok(ProcessorPtr::create(Box::new(Self {
inner,
output,
scan_progress,
is_inner_finish: false,
generated_data: vec![],
})))
}
}

#[async_trait::async_trait]
impl<T: 'static + PrefetchAsyncSource> Processor for PrefetchAsyncSourcer<T> {
fn name(&self) -> String {
T::NAME.to_string()
}

fn as_any(&mut self) -> &mut dyn Any {
self
}

fn event(&mut self) -> Result<Event> {
if self.is_inner_finish && self.generated_data.is_empty() {
self.output.finish();
return Ok(Event::Finished);
}

if self.output.is_finished() {
return Ok(Event::Finished);
}

if self.output.can_push() {
if let Some(data_block) = self.generated_data.pop() {
self.output.push_data(Ok(data_block));
}
}

if self.is_inner_finish || self.inner.is_full(&self.generated_data) {
Ok(Event::NeedConsume)
} else {
Ok(Event::Async)
}
}

fn un_reacted(&self, cause: EventCause, _id: usize) -> Result<()> {
if let EventCause::Output(output) = cause {
youngsofun marked this conversation as resolved.
Show resolved Hide resolved
self.inner.un_reacted()?;
}

Ok(())
}

#[async_backtrace::framed]
async fn async_process(&mut self) -> Result<()> {
match self.inner.generate().await? {
None => self.is_inner_finish = true,
Some(data_block) => {
// Don't need to record the scan progress of `MaterializedCteSource`
// Because it reads data from memory.
if !data_block.is_empty() && self.name() != "MaterializedCteSource" {
let progress_values = ProgressValues {
rows: data_block.num_rows(),
bytes: data_block.memory_size(),
};
self.scan_progress.incr(&progress_values);
Profile::record_usize_profile(
ProfileStatisticsName::ScanBytes,
data_block.memory_size(),
);
}

if !T::SKIP_EMPTY_DATA_BLOCK || !data_block.is_empty() {
self.generated_data.push(data_block)
}
}
};

Ok(())
}
}
11 changes: 9 additions & 2 deletions src/query/storages/stage/src/read/row_based/processors/reader.rs
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,7 @@ use databend_common_catalog::table_context::TableContext;
use databend_common_exception::ErrorCode;
use databend_common_exception::Result;
use databend_common_expression::DataBlock;
use databend_common_pipeline_sources::AsyncSource;
use databend_common_pipeline_sources::PrefetchAsyncSource;
use log::debug;
use opendal::Operator;

Expand All @@ -40,19 +40,22 @@ pub struct BytesReader {
op: Operator,
read_batch_size: usize,
file_state: Option<FileState>,
prefetch_num: usize,
}

impl BytesReader {
pub fn try_create(
table_ctx: Arc<dyn TableContext>,
op: Operator,
read_batch_size: usize,
prefetch_num: usize,
) -> Result<Self> {
Ok(Self {
table_ctx,
op,
read_batch_size,
file_state: None,
prefetch_num,
})
}

Expand Down Expand Up @@ -103,11 +106,15 @@ impl BytesReader {
}

#[async_trait::async_trait]
impl AsyncSource for BytesReader {
impl PrefetchAsyncSource for BytesReader {
const NAME: &'static str = "BytesReader";

const SKIP_EMPTY_DATA_BLOCK: bool = false;

fn is_full(&self, prefetched: &[DataBlock]) -> bool {
prefetched.len() >= self.prefetch_num
}

#[async_trait::unboxed_simple]
async fn generate(&mut self) -> Result<Option<DataBlock>> {
if self.file_state.is_none() {
Expand Down
13 changes: 7 additions & 6 deletions src/query/storages/stage/src/read/row_based/read_pipeline.rs
Original file line number Diff line number Diff line change
Expand Up @@ -25,8 +25,8 @@ use databend_common_meta_app::principal::StageFileCompression;
use databend_common_pipeline_core::processors::ProcessorPtr;
use databend_common_pipeline_core::Pipeline;
use databend_common_pipeline_sources::input_formats::InputContext;
use databend_common_pipeline_sources::AsyncSourcer;
use databend_common_pipeline_sources::EmptySource;
use databend_common_pipeline_sources::PrefetchAsyncSourcer;
use databend_common_pipeline_transforms::processors::AccumulatingTransformer;
use databend_common_settings::Settings;
use databend_common_storage::init_stage_operator;
Expand Down Expand Up @@ -55,8 +55,8 @@ impl RowBasedReadPipelineBuilder<'_> {
let batch_size = settings.get_input_read_buffer_size()? as usize;
pipeline.add_source(
|output| {
let reader = BytesReader::try_create(ctx.clone(), operator.clone(), batch_size)?;
AsyncSourcer::create(ctx.clone(), output, reader)
let reader = BytesReader::try_create(ctx.clone(), operator.clone(), batch_size, 1)?;
PrefetchAsyncSourcer::create(ctx.clone(), output, reader)
},
num_threads,
)?;
Expand Down Expand Up @@ -94,8 +94,9 @@ impl RowBasedReadPipelineBuilder<'_> {
let settings = ctx.get_settings();
ctx.set_partitions(plan.parts.clone())?;

let threads = std::cmp::min(settings.get_max_threads()? as usize, plan.parts.len());
self.build_read_stage_source(ctx.clone(), pipeline, &settings, threads)?;
let max_threads = settings.get_max_threads()? as usize;
let num_sources = std::cmp::min(max_threads, plan.parts.len());
self.build_read_stage_source(ctx.clone(), pipeline, &settings, num_sources)?;

let format =
create_row_based_file_format(&self.stage_table_info.stage_info.file_format_params);
Expand Down Expand Up @@ -137,7 +138,7 @@ impl RowBasedReadPipelineBuilder<'_> {
})?;

// todo(youngsofun): no need to resize if it is unlikely to be unbalanced
pipeline.try_resize(threads)?;
pipeline.try_resize(max_threads)?;

pipeline.add_transform(|input, output| {
let transformer = BlockBuilder::create(load_ctx.clone(), &format)?;
Expand Down
Loading