Skip to content

Commit

Permalink
chore(query): row fetch will be fetched by distinct block batch (#17391)
Browse files Browse the repository at this point in the history
* chore(query): row fetch will be fetched by distinct block batch

* chore(query): row fetch will be fetched by distinct block batch

* chore(query): row fetch will be fetched by distinct block batch

* increase permit
  • Loading branch information
sundy-li authored Feb 1, 2025
1 parent e76aabc commit 8dfab90
Show file tree
Hide file tree
Showing 4 changed files with 95 additions and 34 deletions.
4 changes: 2 additions & 2 deletions Cargo.lock

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

117 changes: 85 additions & 32 deletions src/query/storages/fuse/src/operations/read/fuse_rows_fetcher.rs
Original file line number Diff line number Diff line change
Expand Up @@ -12,8 +12,10 @@
// See the License for the specific language governing permissions and
// limitations under the License.

use std::collections::HashSet;
use std::sync::Arc;

use databend_common_catalog::plan::split_row_id;
use databend_common_catalog::plan::DataSourcePlan;
use databend_common_catalog::plan::Projection;
use databend_common_catalog::table_context::TableContext;
Expand Down Expand Up @@ -66,6 +68,7 @@ pub fn row_fetch_processor(
input,
output,
row_id_col_offset,
max_threads,
NativeRowsFetcher::<true>::create(
fuse_table.clone(),
projection.clone(),
Expand All @@ -79,6 +82,7 @@ pub fn row_fetch_processor(
input,
output,
row_id_col_offset,
max_threads,
NativeRowsFetcher::<false>::create(
fuse_table.clone(),
projection.clone(),
Expand All @@ -97,6 +101,7 @@ pub fn row_fetch_processor(
input,
output,
row_id_col_offset,
max_threads,
ParquetRowsFetcher::<true>::create(
fuse_table.clone(),
projection.clone(),
Expand All @@ -111,6 +116,7 @@ pub fn row_fetch_processor(
input,
output,
row_id_col_offset,
max_threads,
ParquetRowsFetcher::<false>::create(
fuse_table.clone(),
projection.clone(),
Expand All @@ -130,13 +136,17 @@ pub fn row_fetch_processor(
pub trait RowsFetcher {
async fn on_start(&mut self) -> Result<()>;
async fn fetch(&mut self, row_ids: &[u64]) -> Result<DataBlock>;
fn clear_cache(&mut self);
}

pub struct TransformRowsFetcher<F: RowsFetcher> {
row_id_col_offset: usize,
max_threads: usize,
fetcher: F,
need_wrap_nullable: bool,
blocks: Vec<DataBlock>,
row_ids: Vec<u64>,
distinct_block_ids: HashSet<u64>,
}

#[async_trait::async_trait]
Expand All @@ -151,26 +161,7 @@ where F: RowsFetcher + Send + Sync + 'static
}

async fn transform(&mut self, data: DataBlock) -> Result<Option<DataBlock>> {
self.blocks.push(data);
Ok(None)
}

#[async_backtrace::framed]
async fn on_finish(&mut self, _output: bool) -> Result<Option<DataBlock>> {
if self.blocks.is_empty() {
return Ok(None);
}

let start_time = std::time::Instant::now();
let num_blocks = self.blocks.len();
let mut data = DataBlock::concat(&self.blocks)?;
self.blocks.clear();

let num_rows = data.num_rows();
if num_rows == 0 {
return Ok(None);
}

let entry = &data.columns()[self.row_id_col_offset];
let value = entry
.value
Expand All @@ -184,24 +175,43 @@ where F: RowsFetcher + Send + Sync + 'static
value.column.into_number().unwrap().into_u_int64().unwrap()
};

let fetched_block = self.fetcher.fetch(&row_id_column).await?;
// Process the row id column in block batch
// Ensure that the same block would be processed in the same batch and threads
let mut consumed_len = num_rows;
for (idx, row_id) in row_id_column.iter().enumerate() {
let (prefix, _) = split_row_id(*row_id);

for col in fetched_block.columns().iter() {
if self.need_wrap_nullable {
data.add_column(wrap_true_validity(col, num_rows));
} else {
data.add_column(col.clone());
// Which means we are full now, new prefix will be processed in next batch
if self.distinct_block_ids.len() >= self.max_threads * 2
&& !self.distinct_block_ids.contains(&prefix)
{
consumed_len = idx;
break;
}
self.distinct_block_ids.insert(prefix);
}

log::info!(
"TransformRowsFetcher on_finish: num_rows: {}, input blocks: {} in {} milliseconds",
num_rows,
num_blocks,
start_time.elapsed().as_millis()
);
self.row_ids
.extend_from_slice(&row_id_column.as_slice()[0..consumed_len]);
self.blocks.push(data.slice(0..consumed_len));

Ok(Some(data))
if consumed_len < num_rows {
let block = self.flush().await;
for row_id in row_id_column.as_slice()[consumed_len..num_rows].iter() {
let (prefix, _) = split_row_id(*row_id);
self.distinct_block_ids.insert(prefix);
self.row_ids.push(*row_id);
}
self.blocks.push(data.slice(consumed_len..num_rows));
block
} else {
Ok(None)
}
}

#[async_backtrace::framed]
async fn on_finish(&mut self, _output: bool) -> Result<Option<DataBlock>> {
self.flush().await
}
}

Expand All @@ -212,16 +222,59 @@ where F: RowsFetcher + Send + Sync + 'static
input: Arc<InputPort>,
output: Arc<OutputPort>,
row_id_col_offset: usize,
max_threads: usize,
fetcher: F,
need_wrap_nullable: bool,
) -> ProcessorPtr {
ProcessorPtr::create(AsyncAccumulatingTransformer::create(input, output, Self {
row_id_col_offset,
max_threads,
fetcher,
need_wrap_nullable,
blocks: vec![],
row_ids: vec![],
distinct_block_ids: HashSet::new(),
}))
}

async fn flush(&mut self) -> Result<Option<DataBlock>> {
let blocks = std::mem::take(&mut self.blocks);
if blocks.is_empty() {
return Ok(None);
}

let start_time = std::time::Instant::now();
let num_blocks = blocks.len();
let mut data = DataBlock::concat(&blocks)?;
let num_rows = data.num_rows();
if num_rows == 0 {
return Ok(None);
}

let row_ids = std::mem::take(&mut self.row_ids);
self.distinct_block_ids.clear();
let fetched_block = self.fetcher.fetch(&row_ids).await?;
// Clear cache after fetch, the block will never be fetched in following batches
// We ensure it in transform method
self.fetcher.clear_cache();

for col in fetched_block.columns().iter() {
if self.need_wrap_nullable {
data.add_column(wrap_true_validity(col, num_rows));
} else {
data.add_column(col.clone());
}
}

log::info!(
"TransformRowsFetcher flush: num_rows: {}, input blocks: {} in {} milliseconds",
num_rows,
num_blocks,
start_time.elapsed().as_millis()
);

Ok(Some(data))
}
}

fn wrap_true_validity(column: &BlockEntry, num_rows: usize) -> BlockEntry {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -64,6 +64,10 @@ impl<const BLOCKING_IO: bool> RowsFetcher for NativeRowsFetcher<BLOCKING_IO> {
Ok(())
}

fn clear_cache(&mut self) {
self.part_map.clear();
}

#[async_backtrace::framed]
async fn fetch(&mut self, row_ids: &[u64]) -> Result<DataBlock> {
self.prepare_part_map(row_ids).await?;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -66,6 +66,10 @@ impl<const BLOCKING_IO: bool> RowsFetcher for ParquetRowsFetcher<BLOCKING_IO> {
Ok(())
}

fn clear_cache(&mut self) {
self.part_map.clear();
}

#[async_backtrace::framed]
async fn fetch(&mut self, row_ids: &[u64]) -> Result<DataBlock> {
self.prepare_part_map(row_ids).await?;
Expand Down

0 comments on commit 8dfab90

Please sign in to comment.