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

perf(stream): concurrently lookup remote in temporal join #14666

Closed
wants to merge 14 commits into from
Closed
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
177 changes: 134 additions & 43 deletions src/stream/src/executor/temporal_join.rs
Original file line number Diff line number Diff line change
Expand Up @@ -21,11 +21,11 @@ use std::sync::Arc;

use either::Either;
use futures::stream::{self, PollNext};
use futures::{pin_mut, StreamExt, TryStreamExt};
use futures::{future, pin_mut, StreamExt, TryStreamExt};
use futures_async_stream::try_stream;
use local_stats_alloc::{SharedStatsAlloc, StatsAlloc};
use lru::DefaultHasher;
use risingwave_common::array::{Op, StreamChunk};
use risingwave_common::array::{Op, RowRef, StreamChunk};
use risingwave_common::catalog::Schema;
use risingwave_common::estimate_size::{EstimateSize, KvSize};
use risingwave_common::hash::{HashKey, NullBitmap};
Expand Down Expand Up @@ -149,9 +149,8 @@ struct TemporalSide<K: HashKey, S: StateStore> {
}

impl<K: HashKey, S: StateStore> TemporalSide<K, S> {
/// Lookup the temporal side table and return a `JoinEntry` which could be empty if there are no
/// matched records.
async fn lookup(&mut self, key: &K, epoch: HummockEpoch) -> StreamExecutorResult<JoinEntry> {
/// Lookup the temporal side table cache and return a `Some(JoinEntry)`. If `None` was returned, then the result can't be found in cache.
fn lookup_cache(&mut self, key: &K) -> Option<JoinEntry> {
let table_id_str = self.source.table_id().to_string();
let actor_id_str = self.ctx.id.to_string();
let fragment_id_str = self.ctx.id.to_string();
Expand All @@ -160,47 +159,49 @@ impl<K: HashKey, S: StateStore> TemporalSide<K, S> {
.temporal_join_total_query_cache_count
.with_label_values(&[&table_id_str, &actor_id_str, &fragment_id_str])
.inc();

let res = if self.cache.contains(key) {
if self.cache.contains(key) {
let mut state = self.cache.peek_mut(key).unwrap();
state.take()
Some(state.take())
} else {
// cache miss
self.ctx
.streaming_metrics
.temporal_join_cache_miss_count
.with_label_values(&[&table_id_str, &actor_id_str, &fragment_id_str])
.inc();
None
}
}

let pk_prefix = key.deserialize(&self.join_key_data_types)?;

let iter = self
.source
.batch_iter_with_pk_bounds(
HummockReadEpoch::NoWait(epoch),
&pk_prefix,
..,
false,
PrefetchOptions::default(),
)
.await?;

let mut entry = JoinEntry::default();

pin_mut!(iter);
while let Some(row) = iter.next_row().await? {
entry.insert(
row.as_ref()
.project(&self.table_stream_key_indices)
.into_owned_row(),
row.project(&self.table_output_indices).into_owned_row(),
);
}

entry
};
/// Lookup the temporal side table and return a `JoinEntry` which could be empty if there are no
/// matched records. The lookup will bypass the cache, it's better to call [`Self::lookup_cache`] first.
async fn lookup_table(&self, key: &K, epoch: HummockEpoch) -> StreamExecutorResult<JoinEntry> {
let pk_prefix = key.deserialize(&self.join_key_data_types)?;

let iter = self
.source
.batch_iter_with_pk_bounds(
HummockReadEpoch::NoWait(epoch),
&pk_prefix,
..,
false,
PrefetchOptions::default(),
)
.await?;

let mut entry = JoinEntry::default();

pin_mut!(iter);
while let Some(row) = iter.next_row().await? {
entry.insert(
row.as_ref()
.project(&self.table_stream_key_indices)
.into_owned_row(),
row.project(&self.table_output_indices).into_owned_row(),
);
}

Ok(res)
Ok(entry)
}

fn update(
Expand Down Expand Up @@ -428,12 +429,103 @@ impl<K: HashKey, S: StateStore, const T: JoinTypePrimitive> TemporalJoinExecutor
);
let epoch = prev_epoch.expect("Chunk data should come after some barrier.");
let keys = K::build(&self.left_join_keys, chunk.data_chunk())?;
for (r, key) in chunk.rows_with_holes().zip_eq_debug(keys.into_iter()) {
let Some((op, left_row)) = r else {
continue;
};
if key.null_bitmap().is_subset(&null_matched)
&& let join_entry = self.right_table.lookup(&key, epoch).await?

// Fetch the local cache.
struct FetchingCacheItem<'a, K> {
op: Op,
left_row: RowRef<'a>,
key: K,
inner: FetchingCacheItemInner,
}
enum FetchingCacheItemInner {
NotSubsetOfNullMatched,
Cached(JoinEntry),
NotFoundInCache,
}
let fetching_cache_items: Vec<FetchingCacheItem<'_, K>> = chunk
.rows_with_holes()
.zip_eq_debug(keys.into_iter())
.filter_map(|(r, key)| {
use FetchingCacheItemInner::*;
let Some((op, left_row)) = r else {
return None;
};
if !key.null_bitmap().is_subset(&null_matched) {
return Some(FetchingCacheItem {
op,
left_row,
key,
inner: NotSubsetOfNullMatched,
});
}
let inner = match self.right_table.lookup_cache(&key) {
Some(entry) => Cached(entry),
None => NotFoundInCache,
};
Some(FetchingCacheItem {
op,
left_row,
key,
inner,
})
})
.collect();

// Step 2: Fetch remote.
struct FetchingTableItem<'a, K> {
op: Op,
left_row: RowRef<'a>,
key: K,
inner: FetchingTableItemInner,
}
enum FetchingTableItemInner {
NotSubsetOfNullMatched,
Entry(JoinEntry),
}
let fetching_table_futs = fetching_cache_items.into_iter().map(
|FetchingCacheItem {
op,
left_row,
key,
inner,
}| {
let right_table = &self.right_table;
async move {
let inner = match inner {
FetchingCacheItemInner::NotSubsetOfNullMatched => {
Ok(FetchingTableItemInner::NotSubsetOfNullMatched)
as StreamExecutorResult<_>
}
FetchingCacheItemInner::Cached(entry) => {
Ok(FetchingTableItemInner::Entry(entry))
as StreamExecutorResult<_>
}
FetchingCacheItemInner::NotFoundInCache => right_table
.lookup_table(&key, epoch)
.await
.map(FetchingTableItemInner::Entry),
};
Ok(FetchingTableItem {
op,
left_row,
key,
inner: inner?,
}) as StreamExecutorResult<_>
}
},
);
// Fetching table can be concurrent.
let fetching_table_items: Vec<FetchingTableItem<'_, K>> =
future::try_join_all(fetching_table_futs).await?;

for FetchingTableItem {
op,
left_row,
key,
inner,
} in fetching_table_items
{
if let FetchingTableItemInner::Entry(join_entry) = inner
&& !join_entry.is_empty()
{
for right_row in join_entry.cached.values() {
Expand All @@ -447,7 +539,6 @@ impl<K: HashKey, S: StateStore, const T: JoinTypePrimitive> TemporalJoinExecutor
} else {
true
};

if ok {
if let Some(chunk) = builder.append_row(op, left_row, right_row)
{
Expand Down
Loading