From a2112910ba202b33153d8099fdfc43937e569a8a Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Fri, 9 Jun 2023 18:05:36 +0800 Subject: [PATCH 01/90] add arrangement backfill executor --- .../src/executor/arrangement_backfill.rs | 665 ++++++++++++++++++ src/stream/src/executor/mod.rs | 1 + 2 files changed, 666 insertions(+) create mode 100644 src/stream/src/executor/arrangement_backfill.rs diff --git a/src/stream/src/executor/arrangement_backfill.rs b/src/stream/src/executor/arrangement_backfill.rs new file mode 100644 index 0000000000000..5923a0e2bca1b --- /dev/null +++ b/src/stream/src/executor/arrangement_backfill.rs @@ -0,0 +1,665 @@ +// Copyright 2023 RisingWave 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::cmp::Ordering; +use std::ops::Bound; +use std::pin::pin; +use std::sync::Arc; + +use await_tree::InstrumentAwait; +use either::Either; +use futures::stream::select_with_strategy; +use futures::{pin_mut, stream, StreamExt, TryStreamExt}; +use futures_async_stream::try_stream; +use risingwave_common::array::stream_record::Record; +use risingwave_common::array::{Op, StreamChunk}; +use risingwave_common::buffer::BitmapBuilder; +use risingwave_common::catalog::Schema; +use risingwave_common::hash::VnodeBitmapExt; +use risingwave_common::row::{self, OwnedRow, Row, RowExt}; +use risingwave_common::types::Datum; +use risingwave_common::util::epoch::EpochPair; +use risingwave_common::util::iter_util::ZipEqFast; +use risingwave_common::util::sort_util::{cmp_datum, OrderType}; +use risingwave_hummock_sdk::HummockReadEpoch; +use risingwave_storage::error::StorageResult; +use risingwave_storage::store::PrefetchOptions; +use risingwave_storage::table::batch_table::storage_table::{StorageTable, StorageTableInnerIter}; +use risingwave_storage::table::TableIter; +use risingwave_storage::StateStore; + +use super::error::StreamExecutorError; +use super::{expect_first_barrier, BoxedExecutor, Executor, ExecutorInfo, Message, PkIndicesRef}; +use crate::common::table::state_table::StateTable; +use crate::executor::monitor::StreamingMetrics; +use crate::executor::{PkIndices, StreamExecutorResult, Watermark}; +use crate::task::{ActorId, CreateMviewProgress}; + +pub struct ArrangementBackfillExecutor { + /// Upstream table + upstream_table: StateTable, + + /// Upstream with the same schema with the upstream table. + upstream: BoxedExecutor, + + /// Internal state table for persisting state of backfill state. + state_table: Option>, + + /// The column indices need to be forwarded to the downstream from the upstream and table scan. + output_indices: Vec, + + progress: CreateMviewProgress, + + actor_id: ActorId, + + info: ExecutorInfo, + + metrics: Arc, +} + +const CHUNK_SIZE: usize = 1024; + +impl ArrangementBackfillExecutor +where + S: StateStore, +{ + #[allow(clippy::too_many_arguments)] + pub fn new( + upstream_table: StateTable, + upstream: BoxedExecutor, + state_table: Option>, + output_indices: Vec, + progress: CreateMviewProgress, + schema: Schema, + pk_indices: PkIndices, + metrics: Arc, + ) -> Self { + Self { + info: ExecutorInfo { + schema, + pk_indices, + identity: "ArrangementBackfillExecutor".to_owned(), + }, + upstream_table, + upstream, + state_table, + output_indices, + actor_id: progress.actor_id(), + progress, + metrics, + } + } + + #[try_stream(ok = Message, error = StreamExecutorError)] + async fn execute_inner(mut self) { + // The primary key columns, in the output columns of the upstream_table scan. + // let pk_in_output_indices = self.upstream_table.pk_in_output_indices().unwrap(); + let pk_in_output_indices = self.upstream_table.pk_indices(); + let state_len = pk_in_output_indices.len() + 2; // +1 for backfill_finished, +1 for vnode key. + + let pk_order = self.upstream_table.pk_serde().get_order_types(); + + let upstream_table_id = self.upstream_table.table_id(); + + let mut upstream = self.upstream.execute(); + + // Poll the upstream to get the first barrier. + let first_barrier = expect_first_barrier(&mut upstream).await?; + let init_epoch = first_barrier.epoch.prev; + if let Some(state_table) = self.state_table.as_mut() { + state_table.init_epoch(first_barrier.epoch); + } + + let is_finished = if let Some(state_table) = self.state_table.as_mut() { + let is_finished = Self::check_all_vnode_finished(state_table, state_len).await?; + if is_finished { + assert!(!first_barrier.is_newly_added(self.actor_id)); + } + is_finished + } else { + // Maintain backwards compatibility with no state table + !first_barrier.is_newly_added(self.actor_id) + }; + + // If the snapshot is empty, we don't need to backfill. + // We cannot complete progress now, as we want to persist + // finished state to state store first. + // As such we will wait for next barrier. + let is_snapshot_empty: bool = { + if is_finished { + // It is finished, so just assign a value to avoid accessing storage table again. + false + } else { + let snapshot = Self::snapshot_read(&self.upstream_table, init_epoch, None, false); + pin_mut!(snapshot); + snapshot.try_next().await?.unwrap().is_none() + } + }; + + // | backfill_is_finished | snapshot_empty | need_to_backfill | + // | t | t/f | f | + // | f | t | f | + // | f | f | t | + let to_backfill = !is_finished && !is_snapshot_empty; + + // Current position of the upstream_table storage primary key. + // `None` means it starts from the beginning. + let mut current_pos: Option = None; + + // Use these to persist state. + // They contain the backfill position, + // as well as the progress. + // However, they do not contain the vnode key at index 0. + // That is filled in when we flush the state table. + let mut current_state: Vec = vec![None; state_len]; + let mut old_state: Option> = None; + + // The first barrier message should be propagated. + yield Message::Barrier(first_barrier); + + // If no need backfill, but state was still "unfinished" we need to finish it. + // So we just update the state + progress to meta at the next barrier to finish progress, + // and forward other messages. + // + // Reason for persisting on second barrier rather than first: + // We can't update meta with progress as finished until state_table + // has been updated. + // We also can't update state_table in first epoch, since state_table + // expects to have been initialized in previous epoch. + + // The epoch used to snapshot read upstream mv. + let mut snapshot_read_epoch = init_epoch; + + // Keep track of rows from the snapshot. + let mut total_snapshot_processed_rows: u64 = 0; + + // Backfill Algorithm: + // + // backfill_stream + // / \ + // upstream snapshot + // + // We construct a backfill stream with upstream as its left input and mv snapshot read + // stream as its right input. When a chunk comes from upstream, we will buffer it. + // + // When a barrier comes from upstream: + // - Update the `snapshot_read_epoch`. + // - For each row of the upstream chunk buffer, forward it to downstream if its pk <= + // `current_pos`, otherwise ignore it. + // - reconstruct the whole backfill stream with upstream and new mv snapshot read stream + // with the `snapshot_read_epoch`. + // + // When a chunk comes from snapshot, we forward it to the downstream and raise + // `current_pos`. + // + // When we reach the end of the snapshot read stream, it means backfill has been + // finished. + // + // Once the backfill loop ends, we forward the upstream directly to the downstream. + if to_backfill { + 'backfill_loop: loop { + let mut upstream_chunk_buffer: Vec = vec![]; + + let left_upstream = upstream.by_ref().map(Either::Left); + + let right_snapshot = pin!(Self::snapshot_read( + &self.upstream_table, + snapshot_read_epoch, + current_pos.clone(), + true + ) + .map(Either::Right),); + + // Prefer to select upstream, so we can stop snapshot stream as soon as the barrier + // comes. + let backfill_stream = + select_with_strategy(left_upstream, right_snapshot, |_: &mut ()| { + stream::PollNext::Left + }); + + let mut cur_barrier_snapshot_processed_rows: u64 = 0; + let mut cur_barrier_upstream_processed_rows: u64 = 0; + + #[for_await] + for either in backfill_stream { + match either { + // Upstream + Either::Left(msg) => { + match msg? { + Message::Barrier(barrier) => { + // If it is a barrier, switch snapshot and consume + // upstream buffer chunk + + // Consume upstream buffer chunk + if let Some(current_pos) = ¤t_pos { + for chunk in upstream_chunk_buffer.drain(..) { + cur_barrier_upstream_processed_rows += + chunk.cardinality() as u64; + yield Message::Chunk(Self::mapping_chunk( + Self::mark_chunk( + chunk, + current_pos, + &pk_in_output_indices, + pk_order, + ), + &self.output_indices, + )); + } + } + + self.metrics + .backfill_snapshot_read_row_count + .with_label_values(&[ + upstream_table_id.to_string().as_str(), + self.actor_id.to_string().as_str(), + ]) + .inc_by(cur_barrier_snapshot_processed_rows); + + self.metrics + .backfill_upstream_output_row_count + .with_label_values(&[ + upstream_table_id.to_string().as_str(), + self.actor_id.to_string().as_str(), + ]) + .inc_by(cur_barrier_upstream_processed_rows); + + // Update snapshot read epoch. + snapshot_read_epoch = barrier.epoch.prev; + + self.progress.update( + barrier.epoch.curr, + snapshot_read_epoch, + total_snapshot_processed_rows, + ); + + // Persist state on barrier + Self::persist_state( + barrier.epoch, + &mut self.state_table, + false, + ¤t_pos, + &mut old_state, + &mut current_state, + ) + .await?; + + yield Message::Barrier(barrier); + // Break the for loop and start a new snapshot read stream. + break; + } + Message::Chunk(chunk) => { + // Buffer the upstream chunk. + upstream_chunk_buffer.push(chunk.compact()); + } + Message::Watermark(_) => { + // Ignore watermark during backfill. + } + } + } + // Snapshot read + Either::Right(msg) => { + match msg? { + None => { + // End of the snapshot read stream. + // We should not mark the chunk anymore, + // otherwise, we will ignore some rows + // in the buffer. Here we choose to never mark the chunk. + // Consume with the renaming stream buffer chunk without mark. + for chunk in upstream_chunk_buffer.drain(..) { + let chunk_cardinality = chunk.cardinality() as u64; + cur_barrier_snapshot_processed_rows += chunk_cardinality; + total_snapshot_processed_rows += chunk_cardinality; + yield Message::Chunk(Self::mapping_chunk( + chunk, + &self.output_indices, + )); + } + + break 'backfill_loop; + } + Some(chunk) => { + // Raise the current position. + // As snapshot read streams are ordered by pk, so we can + // just use the last row to update `current_pos`. + current_pos = Self::update_pos(&chunk, &pk_in_output_indices); + + let chunk_cardinality = chunk.cardinality() as u64; + cur_barrier_snapshot_processed_rows += chunk_cardinality; + total_snapshot_processed_rows += chunk_cardinality; + yield Message::Chunk(Self::mapping_chunk( + chunk, + &self.output_indices, + )); + } + } + } + } + } + } + } + + tracing::trace!( + actor = self.actor_id, + "Backfill has already finished and forward messages directly to the downstream" + ); + + // Wait for first barrier to come after backfill is finished. + // So we can update our progress + persist the status. + while let Some(Ok(msg)) = upstream.next().await { + if let Some(msg) = Self::mapping_message(msg, &self.output_indices) { + // If not finished then we need to update state, otherwise no need. + if let Message::Barrier(barrier) = &msg && !is_finished { + // If snapshot was empty, we do not need to backfill, + // but we still need to persist the finished state. + // We currently persist it on the second barrier here rather than first. + // This is because we can't update state table in first epoch, + // since it expects to have been initialized in previous epoch + // (there's no epoch before the first epoch). + if is_snapshot_empty { + current_pos = + Self::construct_initial_finished_state(pk_in_output_indices.len()) + } + + // We will update current_pos at least once, + // since snapshot read has to be non-empty, + // Or snapshot was empty and we construct a placeholder state. + debug_assert_ne!(current_pos, None); + + Self::persist_state( + barrier.epoch, + &mut self.state_table, + true, + ¤t_pos, + &mut old_state, + &mut current_state, + ) + .await?; + self.progress.finish(barrier.epoch.curr); + yield msg; + break; + } + yield msg; + } + } + + // After progress finished + state persisted, + // we can forward messages directly to the downstream, + // as backfill is finished. + #[for_await] + for msg in upstream { + if let Some(msg) = Self::mapping_message(msg?, &self.output_indices) { + if let Some(state_table) = self.state_table.as_mut() && let Message::Barrier(barrier) = &msg { + state_table.commit_no_data_expected(barrier.epoch); + } + yield msg; + } + } + } + + #[try_stream(ok = Option, error = StreamExecutorError)] + async fn snapshot_read( + upstream_table: &StateTable, + epoch: u64, + current_pos: Option, + ordered: bool, + ) { + // `current_pos` is None means it needs to scan from the beginning, so we use Unbounded to + // scan. Otherwise, use Excluded. + let range_bounds: (Bound, Bound) = + if let Some(current_pos) = current_pos { + // If `current_pos` is an empty row which means upstream mv contains only one row + // and it has been consumed. The iter interface doesn't support + // `Excluded(empty_row)` range bound, so we can simply return `None`. + if current_pos.is_empty() { + assert!(upstream_table.pk_indices().is_empty()); + yield None; + return Ok(()); + } + + (Bound::Excluded(current_pos), Bound::Unbounded) + } else { + (Bound::Unbounded, Bound::Unbounded) + }; + // // We use uncommitted read here, because we have already scheduled the + // // `ArrangementBackfillExecutor` together with the upstream mv. + // let iter = upstream_table + // .batch_iter_with_pk_bounds( + // HummockReadEpoch::NoWait(epoch), + // row::empty(), + // range_bounds, + // ordered, + // PrefetchOptions::new_for_exhaust_iter(), + // ) + // .await?; + // + // pin_mut!(iter); + // + // + // while let Some(data_chunk) = iter + // .collect_data_chunk(upstream_table.schema(), Some(CHUNK_SIZE)) + // .instrument_await("backfill_snapshot_read") + // .await? + // { + // if data_chunk.cardinality() != 0 { + // let ops = vec![Op::Insert; data_chunk.capacity()]; + // let stream_chunk = StreamChunk::from_parts(ops, data_chunk); + // yield Some(stream_chunk); + // } + // } + + yield None; + } + + /// Mark chunk: + /// For each row of the chunk, forward it to downstream if its pk <= `current_pos`, otherwise + /// ignore it. We implement it by changing the visibility bitmap. + fn mark_chunk( + chunk: StreamChunk, + current_pos: &OwnedRow, + pk_in_output_indices: PkIndicesRef<'_>, + pk_order: &[OrderType], + ) -> StreamChunk { + let chunk = chunk.compact(); + let (data, ops) = chunk.into_parts(); + let mut new_visibility = BitmapBuilder::with_capacity(ops.len()); + // Use project to avoid allocation. + for v in data.rows().map(|row| { + match row + .project(pk_in_output_indices) + .iter() + .zip_eq_fast(pk_order.iter().copied()) + .cmp_by(current_pos.iter(), |(x, order), y| cmp_datum(x, y, order)) + { + Ordering::Less | Ordering::Equal => true, + Ordering::Greater => false, + } + }) { + new_visibility.append(v); + } + let (columns, _) = data.into_parts(); + StreamChunk::new(ops, columns, Some(new_visibility.finish())) + } + + /// Builds a new stream chunk with `output_indices`. + fn mapping_chunk(chunk: StreamChunk, output_indices: &[usize]) -> StreamChunk { + let (ops, columns, visibility) = chunk.into_inner(); + let mapped_columns = output_indices.iter().map(|&i| columns[i].clone()).collect(); + StreamChunk::new(ops, mapped_columns, visibility) + } + + fn mapping_watermark(watermark: Watermark, upstream_indices: &[usize]) -> Option { + watermark.transform_with_indices(upstream_indices) + } + + fn mapping_message(msg: Message, upstream_indices: &[usize]) -> Option { + match msg { + Message::Barrier(_) => Some(msg), + Message::Watermark(watermark) => { + Self::mapping_watermark(watermark, upstream_indices).map(Message::Watermark) + } + Message::Chunk(chunk) => { + Some(Message::Chunk(Self::mapping_chunk(chunk, upstream_indices))) + } + } + } + + /// Schema + /// | vnode | pk | `backfill_finished` | + /// + /// For `current_pos` and `old_pos` are just pk of upstream. + /// They should be strictly increasing. + async fn persist_state( + epoch: EpochPair, + table: &mut Option>, + is_finished: bool, + current_pos: &Option, + old_state: &mut Option>, + current_state: &mut [Datum], + ) -> StreamExecutorResult<()> { + // Backwards compatibility with no state table in backfill. + let Some(table) = table else { + return Ok(()) + }; + if let Some(current_pos_inner) = current_pos { + // state w/o vnodes. + Self::build_temporary_state(current_state, is_finished, current_pos_inner); + Self::flush_data(table, epoch, old_state, current_state).await?; + *old_state = Some(current_state.into()); + } else { + table.commit_no_data_expected(epoch); + } + Ok(()) + } + + /// Flush the data + async fn flush_data( + table: &mut StateTable, + epoch: EpochPair, + old_state: &mut Option>, + current_partial_state: &mut [Datum], + ) -> StreamExecutorResult<()> { + let vnodes = table.vnodes().clone(); + if let Some(old_state) = old_state { + if old_state[1..] == current_partial_state[1..] { + table.commit_no_data_expected(epoch); + return Ok(()); + } else { + vnodes.iter_vnodes_scalar().for_each(|vnode| { + let datum = Some(vnode.into()); + current_partial_state[0] = datum.clone(); + old_state[0] = datum; + table.write_record(Record::Update { + old_row: &old_state[..], + new_row: &(*current_partial_state), + }) + }); + } + } else { + // No existing state, create a new entry. + vnodes.iter_vnodes_scalar().for_each(|vnode| { + let datum = Some(vnode.into()); + // fill the state + current_partial_state[0] = datum; + table.write_record(Record::Insert { + new_row: &(*current_partial_state), + }) + }); + } + table.commit(epoch).await + } + + // We want to avoid building a row for every vnode. + // Instead we can just modify a single row, and dispatch it to state table to write. + fn build_temporary_state(row_state: &mut [Datum], is_finished: bool, current_pos: &OwnedRow) { + row_state[1..current_pos.len() + 1].clone_from_slice(current_pos.as_inner()); + row_state[current_pos.len() + 1] = Some(is_finished.into()); + } + + fn update_pos(chunk: &StreamChunk, pk_in_output_indices: &[usize]) -> Option { + Some( + chunk + .rows() + .last() + .unwrap() + .1 + .project(pk_in_output_indices) + .into_owned_row(), + ) + } + + // TODO(kwannoel): I'm not sure if ["None" ..] encoding is appropriate + // for the case where upstream snapshot is empty, and we want to persist + // backfill state as "finished". + // Could it be confused with another case where pk position comprised of nulls? + // I don't think it will matter, + // because they both record that backfill is finished. + // We can revisit in future if necessary. + fn construct_initial_finished_state(pos_len: usize) -> Option { + Some(OwnedRow::new(vec![None; pos_len])) + } + + /// All vnodes should be persisted with status finished. + /// TODO: In the future we will support partial backfill recovery. + /// When that is done, this logic may need to be rewritten to handle + /// partially complete states per vnode. + async fn check_all_vnode_finished( + state_table: &StateTable, + state_len: usize, + ) -> StreamExecutorResult { + debug_assert!(!state_table.vnode_bitmap().is_empty()); + let vnodes = state_table.vnodes().iter_vnodes_scalar(); + let mut is_finished = true; + for vnode in vnodes { + let key: &[Datum] = &[Some(vnode.into())]; + let row = state_table.get_row(key).await?; + + // original_backfill_datum_pos = (state_len - 1) + // value indices are set, so we can -1 for the pk (a single vnode). + let backfill_datum_pos = state_len - 2; + let vnode_is_finished = if let Some(row) = row + && let Some(vnode_is_finished) = row.datum_at(backfill_datum_pos) + { + vnode_is_finished.into_bool() + } else { + false + }; + if !vnode_is_finished { + is_finished = false; + break; + } + } + Ok(is_finished) + } +} + +impl Executor for ArrangementBackfillExecutor +where + S: StateStore, +{ + fn execute(self: Box) -> super::BoxedMessageStream { + self.execute_inner().boxed() + } + + fn schema(&self) -> &Schema { + &self.info.schema + } + + fn pk_indices(&self) -> super::PkIndicesRef<'_> { + &self.info.pk_indices + } + + fn identity(&self) -> &str { + &self.info.identity + } +} diff --git a/src/stream/src/executor/mod.rs b/src/stream/src/executor/mod.rs index 069982aab0dd7..8b73b87604d09 100644 --- a/src/stream/src/executor/mod.rs +++ b/src/stream/src/executor/mod.rs @@ -55,6 +55,7 @@ pub mod monitor; pub mod agg_common; pub mod aggregation; +mod arrangement_backfill; mod barrier_recv; mod batch_query; mod chain; From 72a50c697e4dfdaac7523679a74e8d3379e7a6e3 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Fri, 9 Jun 2023 18:28:54 +0800 Subject: [PATCH 02/90] iter over pk bounds of state table --- .../src/executor/arrangement_backfill.rs | 44 ++++++++----------- 1 file changed, 18 insertions(+), 26 deletions(-) diff --git a/src/stream/src/executor/arrangement_backfill.rs b/src/stream/src/executor/arrangement_backfill.rs index 5923a0e2bca1b..687e59508d995 100644 --- a/src/stream/src/executor/arrangement_backfill.rs +++ b/src/stream/src/executor/arrangement_backfill.rs @@ -26,7 +26,7 @@ use risingwave_common::array::stream_record::Record; use risingwave_common::array::{Op, StreamChunk}; use risingwave_common::buffer::BitmapBuilder; use risingwave_common::catalog::Schema; -use risingwave_common::hash::VnodeBitmapExt; +use risingwave_common::hash::{VirtualNode, VnodeBitmapExt}; use risingwave_common::row::{self, OwnedRow, Row, RowExt}; use risingwave_common::types::Datum; use risingwave_common::util::epoch::EpochPair; @@ -431,32 +431,24 @@ where } else { (Bound::Unbounded, Bound::Unbounded) }; - // // We use uncommitted read here, because we have already scheduled the - // // `ArrangementBackfillExecutor` together with the upstream mv. - // let iter = upstream_table - // .batch_iter_with_pk_bounds( - // HummockReadEpoch::NoWait(epoch), - // row::empty(), - // range_bounds, - // ordered, - // PrefetchOptions::new_for_exhaust_iter(), - // ) - // .await?; - // + // TODO: iter over all vnodes of this state table. + let iter = upstream_table + .iter_with_pk_range(&range_bounds, VirtualNode::ZERO, Default::default()) + .await?; // pin_mut!(iter); - // - // - // while let Some(data_chunk) = iter - // .collect_data_chunk(upstream_table.schema(), Some(CHUNK_SIZE)) - // .instrument_await("backfill_snapshot_read") - // .await? - // { - // if data_chunk.cardinality() != 0 { - // let ops = vec![Op::Insert; data_chunk.capacity()]; - // let stream_chunk = StreamChunk::from_parts(ops, data_chunk); - // yield Some(stream_chunk); - // } - // } + + // TODO: these are rows instead... + #[for_await] + for data_chunk in iter + // .collect_data_chunk(upstream_table.schema(), Some(CHUNK_SIZE)) + // .instrument_await("backfill_snapshot_read") + { + if data_chunk.cardinality() != 0 { + let ops = vec![Op::Insert; data_chunk.capacity()]; + let stream_chunk = StreamChunk::from_parts(ops, data_chunk); + yield Some(stream_chunk); + } + } yield None; } From 6e573c2cda5c14b572812e5ef5dce7eea45abf3c Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Tue, 13 Jun 2023 14:29:14 +0800 Subject: [PATCH 03/90] add ordered iter stub --- src/stream/src/common/table/state_table.rs | 14 ++++++++ .../src/executor/arrangement_backfill.rs | 32 +++++++++---------- 2 files changed, 30 insertions(+), 16 deletions(-) diff --git a/src/stream/src/common/table/state_table.rs b/src/stream/src/common/table/state_table.rs index 93ef6898acffa..2d48f879cf4e6 100644 --- a/src/stream/src/common/table/state_table.rs +++ b/src/stream/src/common/table/state_table.rs @@ -936,6 +936,20 @@ where .map(get_second)) } + pub async fn iter_ordered_with_pk_range( + &self, + pk_range: &(Bound, Bound), + // Optional vnode that returns an iterator only over the given range under that vnode. + // For now, we require this parameter, and will panic. In the future, when `None`, we can + // iterate over each vnode that the `StateTableInner` owns. + prefetch_options: PrefetchOptions, + ) -> StreamExecutorResult> { + Ok(self + .iter_key_and_val_with_pk_range(pk_range, VirtualNode::ZERO, prefetch_options) + .await? + .map(get_second)) + } + pub async fn iter_key_and_val_with_pk_range( &self, pk_range: &(Bound, Bound), diff --git a/src/stream/src/executor/arrangement_backfill.rs b/src/stream/src/executor/arrangement_backfill.rs index 687e59508d995..5f4bdc3711648 100644 --- a/src/stream/src/executor/arrangement_backfill.rs +++ b/src/stream/src/executor/arrangement_backfill.rs @@ -431,24 +431,24 @@ where } else { (Bound::Unbounded, Bound::Unbounded) }; - // TODO: iter over all vnodes of this state table. + // // TODO: iter over all vnodes of this state table. let iter = upstream_table - .iter_with_pk_range(&range_bounds, VirtualNode::ZERO, Default::default()) + .iter_ordered_with_pk_range(&range_bounds, Default::default()) .await?; - // pin_mut!(iter); - - // TODO: these are rows instead... - #[for_await] - for data_chunk in iter - // .collect_data_chunk(upstream_table.schema(), Some(CHUNK_SIZE)) - // .instrument_await("backfill_snapshot_read") - { - if data_chunk.cardinality() != 0 { - let ops = vec![Op::Insert; data_chunk.capacity()]; - let stream_chunk = StreamChunk::from_parts(ops, data_chunk); - yield Some(stream_chunk); - } - } + // // pin_mut!(iter); + // + // // TODO: these are rows instead... + // #[for_await] + // for data_chunk in iter + // // .collect_data_chunk(upstream_table.schema(), Some(CHUNK_SIZE)) + // // .instrument_await("backfill_snapshot_read") + // { + // if data_chunk.cardinality() != 0 { + // let ops = vec![Op::Insert; data_chunk.capacity()]; + // let stream_chunk = StreamChunk::from_parts(ops, data_chunk); + // yield Some(stream_chunk); + // } + // } yield None; } From 8c6dc8460df73b833ff1c478ec0f79337f7ff967 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Tue, 13 Jun 2023 14:32:25 +0800 Subject: [PATCH 04/90] add reference to schema --- src/stream/src/executor/arrangement_backfill.rs | 16 +++++++++++++--- 1 file changed, 13 insertions(+), 3 deletions(-) diff --git a/src/stream/src/executor/arrangement_backfill.rs b/src/stream/src/executor/arrangement_backfill.rs index 5f4bdc3711648..a7dedd10290d7 100644 --- a/src/stream/src/executor/arrangement_backfill.rs +++ b/src/stream/src/executor/arrangement_backfill.rs @@ -112,6 +112,8 @@ where let upstream_table_id = self.upstream_table.table_id(); + let schema = Arc::new(self.upstream.schema().clone()); + let mut upstream = self.upstream.execute(); // Poll the upstream to get the first barrier. @@ -141,7 +143,13 @@ where // It is finished, so just assign a value to avoid accessing storage table again. false } else { - let snapshot = Self::snapshot_read(&self.upstream_table, init_epoch, None, false); + let snapshot = Self::snapshot_read( + schema.clone(), + &self.upstream_table, + init_epoch, + None, + false, + ); pin_mut!(snapshot); snapshot.try_next().await?.unwrap().is_none() } @@ -214,6 +222,7 @@ where let left_upstream = upstream.by_ref().map(Either::Left); let right_snapshot = pin!(Self::snapshot_read( + schema.clone(), &self.upstream_table, snapshot_read_epoch, current_pos.clone(), @@ -409,6 +418,7 @@ where #[try_stream(ok = Option, error = StreamExecutorError)] async fn snapshot_read( + schema: Arc, upstream_table: &StateTable, epoch: u64, current_pos: Option, @@ -431,11 +441,11 @@ where } else { (Bound::Unbounded, Bound::Unbounded) }; - // // TODO: iter over all vnodes of this state table. + // TODO: iter over all vnodes of this state table. let iter = upstream_table .iter_ordered_with_pk_range(&range_bounds, Default::default()) .await?; - // // pin_mut!(iter); + // pin_mut!(iter); // // // TODO: these are rows instead... // #[for_await] From d6bc636c3e7ea24b45fa3e42ca02f378ceb076b4 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Tue, 13 Jun 2023 15:07:28 +0800 Subject: [PATCH 05/90] duplicate collect_data_chunk for now --- src/stream/src/common/table/state_table.rs | 47 ++++++++++++++++++++-- 1 file changed, 43 insertions(+), 4 deletions(-) diff --git a/src/stream/src/common/table/state_table.rs b/src/stream/src/common/table/state_table.rs index 2d48f879cf4e6..f9eec51c720eb 100644 --- a/src/stream/src/common/table/state_table.rs +++ b/src/stream/src/common/table/state_table.rs @@ -20,10 +20,12 @@ use bytes::{BufMut, Bytes, BytesMut}; use futures::{Stream, StreamExt}; use itertools::{izip, Itertools}; use risingwave_common::array::stream_record::Record; -use risingwave_common::array::{Op, StreamChunk, Vis}; +use risingwave_common::array::{DataChunk, Op, StreamChunk, Vis}; use risingwave_common::buffer::Bitmap; use risingwave_common::cache::CachePriority; -use risingwave_common::catalog::{get_dist_key_in_pk_indices, ColumnDesc, TableId, TableOption}; +use risingwave_common::catalog::{ + get_dist_key_in_pk_indices, ColumnDesc, Schema, TableId, TableOption, +}; use risingwave_common::hash::{VirtualNode, VnodeBitmapExt}; use risingwave_common::row::{self, CompactedRow, OwnedRow, Row, RowExt}; use risingwave_common::types::ScalarImpl; @@ -36,7 +38,7 @@ use risingwave_hummock_sdk::key::{ end_bound_of_prefix, next_key, prefixed_range, range_of_prefix, start_bound_of_excluded_prefix, }; use risingwave_pb::catalog::Table; -use risingwave_storage::error::StorageError; +use risingwave_storage::error::{StorageError, StorageResult}; use risingwave_storage::hummock::CachePolicy; use risingwave_storage::mem_table::MemTableError; use risingwave_storage::row_serde::row_serde_util::{ @@ -45,7 +47,7 @@ use risingwave_storage::row_serde::row_serde_util::{ use risingwave_storage::store::{ LocalStateStore, NewLocalOptions, PrefetchOptions, ReadOptions, StateStoreIterItemStream, }; -use risingwave_storage::table::{compute_chunk_vnode, compute_vnode, Distribution}; +use risingwave_storage::table::{compute_chunk_vnode, compute_vnode, Distribution, TableIter}; use risingwave_storage::StateStore; use tracing::trace; @@ -1093,6 +1095,43 @@ where } } +// FIXME: Try merge this with the impl in `src/table/mod.rs`. +async fn collect_data_chunk( + mut stream: impl Stream> + Unpin, + schema: &Schema, + chunk_size: Option, +) -> StreamExecutorResult> { + let mut builders = schema.create_array_builders(chunk_size.unwrap_or(0)); + + let mut row_count = 0; + for _ in 0..chunk_size.unwrap_or(usize::MAX) { + match stream.next().await.transpose()? { + Some(row) => { + for (datum, builder) in row.iter().zip_eq_fast(builders.iter_mut()) { + builder.append(datum); + } + } + None => break, + } + + row_count += 1; + } + + let chunk = { + let columns: Vec<_> = builders + .into_iter() + .map(|builder| builder.finish().into()) + .collect(); + DataChunk::new(columns, row_count) + }; + + if chunk.cardinality() == 0 { + Ok(None) + } else { + Ok(Some(chunk)) + } +} + pub type RowStream<'a, S: StateStore, SD: ValueRowSerde + 'a> = impl Stream> + 'a; pub type RowStreamWithPk<'a, S: StateStore, SD: ValueRowSerde + 'a> = From f9c688d2e7c62969b3094af9a6ce88b19625f081 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Tue, 13 Jun 2023 15:21:42 +0800 Subject: [PATCH 06/90] enable collect_data_chunk --- src/stream/src/common/table/state_table.rs | 2 +- .../src/executor/arrangement_backfill.rs | 28 ++++++++----------- 2 files changed, 13 insertions(+), 17 deletions(-) diff --git a/src/stream/src/common/table/state_table.rs b/src/stream/src/common/table/state_table.rs index f9eec51c720eb..07027abc18add 100644 --- a/src/stream/src/common/table/state_table.rs +++ b/src/stream/src/common/table/state_table.rs @@ -1096,7 +1096,7 @@ where } // FIXME: Try merge this with the impl in `src/table/mod.rs`. -async fn collect_data_chunk( +pub async fn collect_data_chunk( mut stream: impl Stream> + Unpin, schema: &Schema, chunk_size: Option, diff --git a/src/stream/src/executor/arrangement_backfill.rs b/src/stream/src/executor/arrangement_backfill.rs index a7dedd10290d7..5aeecd6802a33 100644 --- a/src/stream/src/executor/arrangement_backfill.rs +++ b/src/stream/src/executor/arrangement_backfill.rs @@ -41,7 +41,7 @@ use risingwave_storage::StateStore; use super::error::StreamExecutorError; use super::{expect_first_barrier, BoxedExecutor, Executor, ExecutorInfo, Message, PkIndicesRef}; -use crate::common::table::state_table::StateTable; +use crate::common::table::state_table::{collect_data_chunk, StateTable}; use crate::executor::monitor::StreamingMetrics; use crate::executor::{PkIndices, StreamExecutorResult, Watermark}; use crate::task::{ActorId, CreateMviewProgress}; @@ -441,24 +441,20 @@ where } else { (Bound::Unbounded, Bound::Unbounded) }; - // TODO: iter over all vnodes of this state table. - let iter = upstream_table + let mut iter = upstream_table .iter_ordered_with_pk_range(&range_bounds, Default::default()) .await?; - // pin_mut!(iter); - // + pin_mut!(iter); // // TODO: these are rows instead... - // #[for_await] - // for data_chunk in iter - // // .collect_data_chunk(upstream_table.schema(), Some(CHUNK_SIZE)) - // // .instrument_await("backfill_snapshot_read") - // { - // if data_chunk.cardinality() != 0 { - // let ops = vec![Op::Insert; data_chunk.capacity()]; - // let stream_chunk = StreamChunk::from_parts(ops, data_chunk); - // yield Some(stream_chunk); - // } - // } + for data_chunk in collect_data_chunk(iter, &schema, Some(CHUNK_SIZE)).await? + // .instrument_await("arrangement_backfill_snapshot_read") + { + if data_chunk.cardinality() != 0 { + let ops = vec![Op::Insert; data_chunk.capacity()]; + let stream_chunk = StreamChunk::from_parts(ops, data_chunk); + yield Some(stream_chunk); + } + } yield None; } From 022710f66c244a89188a589e6d0c77e8ee862937 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Tue, 13 Jun 2023 15:30:49 +0800 Subject: [PATCH 07/90] add back instrumentation --- src/stream/src/executor/arrangement_backfill.rs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/stream/src/executor/arrangement_backfill.rs b/src/stream/src/executor/arrangement_backfill.rs index 5aeecd6802a33..56ba18a6b1df5 100644 --- a/src/stream/src/executor/arrangement_backfill.rs +++ b/src/stream/src/executor/arrangement_backfill.rs @@ -445,9 +445,9 @@ where .iter_ordered_with_pk_range(&range_bounds, Default::default()) .await?; pin_mut!(iter); - // // TODO: these are rows instead... - for data_chunk in collect_data_chunk(iter, &schema, Some(CHUNK_SIZE)).await? - // .instrument_await("arrangement_backfill_snapshot_read") + for data_chunk in collect_data_chunk(iter, &schema, Some(CHUNK_SIZE)) + .instrument_await("arrangement_backfill_snapshot_read") + .await? { if data_chunk.cardinality() != 0 { let ops = vec![Op::Insert; data_chunk.capacity()]; From 2f37d1ba7492a36497e96777bb4b29c17ae30b5e Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Tue, 13 Jun 2023 16:39:16 +0800 Subject: [PATCH 08/90] checkpoint the work on merge_sort --- src/stream/src/common/table/state_table.rs | 37 ++++++++++++++++++++-- 1 file changed, 35 insertions(+), 2 deletions(-) diff --git a/src/stream/src/common/table/state_table.rs b/src/stream/src/common/table/state_table.rs index 07027abc18add..0c2dbce373ec9 100644 --- a/src/stream/src/common/table/state_table.rs +++ b/src/stream/src/common/table/state_table.rs @@ -12,12 +12,15 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::collections::binary_heap::PeekMut; +use std::collections::BinaryHeap; use std::ops::Bound; use std::ops::Bound::*; use std::sync::Arc; use bytes::{BufMut, Bytes, BytesMut}; use futures::{Stream, StreamExt}; +use futures_async_stream::try_stream; use itertools::{izip, Itertools}; use risingwave_common::array::stream_record::Record; use risingwave_common::array::{DataChunk, Op, StreamChunk, Vis}; @@ -946,10 +949,12 @@ where // iterate over each vnode that the `StateTableInner` owns. prefetch_options: PrefetchOptions, ) -> StreamExecutorResult> { - Ok(self + let stream = self .iter_key_and_val_with_pk_range(pk_range, VirtualNode::ZERO, prefetch_options) .await? - .map(get_second)) + .map(get_second); + let stream = merge_sort(vec![stream]).await; + Ok(stream) } pub async fn iter_key_and_val_with_pk_range( @@ -1095,6 +1100,34 @@ where } } +// FIXME: Try merge this with impl from batch/iter_utils.rs + +/// Merge multiple streams of primary key and rows into a single stream, sorted by primary key. +/// We should ensure that the primary key from different streams are unique. +#[try_stream(ok = OwnedRow, error = StreamExecutorError)] +pub(super) async fn merge_sort( + streams: Vec> + Unpin>, +) { + let mut heap = BinaryHeap::with_capacity(streams.len()); + for mut stream in streams { + if let Some(peeked) = stream.next().await.transpose()? { + heap.push(Node { stream, peeked }); + } + } + + while let Some(mut node) = heap.peek_mut() { + // Note: If the `next` returns `Err`, we'll fail to yield the previous item. + // This is acceptable since we're not going to handle errors from cell-based table + // iteration, so where to fail does not matter. Or we need an `Option` for this. + yield match node.stream.next().await.transpose()? { + // There still remains data in the stream, take and update the peeked value. + Some(new_peeked) => std::mem::replace(&mut node.peeked, new_peeked), + // This stream is exhausted, remove it from the heap. + None => PeekMut::pop(node).peeked, + }; + } +} + // FIXME: Try merge this with the impl in `src/table/mod.rs`. pub async fn collect_data_chunk( mut stream: impl Stream> + Unpin, From 6f8a97ea875fc4f4a63de61994b9cac0ed2615f2 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Wed, 14 Jun 2023 15:39:29 +0800 Subject: [PATCH 09/90] make compile --- src/stream/src/common/table/state_table.rs | 98 ++++++++++++------- .../src/executor/arrangement_backfill.rs | 4 +- 2 files changed, 61 insertions(+), 41 deletions(-) diff --git a/src/stream/src/common/table/state_table.rs b/src/stream/src/common/table/state_table.rs index 0c2dbce373ec9..08ae1bf0edfa8 100644 --- a/src/stream/src/common/table/state_table.rs +++ b/src/stream/src/common/table/state_table.rs @@ -16,6 +16,7 @@ use std::collections::binary_heap::PeekMut; use std::collections::BinaryHeap; use std::ops::Bound; use std::ops::Bound::*; +use std::pin::Pin; use std::sync::Arc; use bytes::{BufMut, Bytes, BytesMut}; @@ -61,6 +62,38 @@ use crate::executor::{StreamExecutorError, StreamExecutorResult}; /// This num is arbitrary and we may want to improve this choice in the future. const STATE_CLEANING_PERIOD_EPOCH: usize = 5; +// FIXME: This is duplicated from batch iter utils. +struct Node<'a, S: StateStore, SD: ValueRowSerde + 'a> { + stream: RowStreamWithPk<'a, S, SD>, + + /// The next item polled from `stream` previously. Since the `eq` and `cmp` must be synchronous + /// functions, we need to implement peeking manually. + peeked: (Bytes, OwnedRow), +} + +impl<'a, S: StateStore, SD: ValueRowSerde + 'a> PartialEq for Node<'a, S, SD> { + fn eq(&self, other: &Self) -> bool { + match self.peeked.0 == other.peeked.0 { + true => unreachable!("primary key from different iters should be unique"), + false => false, + } + } +} +impl<'a, S: StateStore, SD: ValueRowSerde + 'a> Eq for Node<'a, S, SD> {} + +impl<'a, S: StateStore, SD: ValueRowSerde + 'a> PartialOrd for Node<'a, S, SD> { + fn partial_cmp(&self, other: &Self) -> Option { + Some(self.cmp(other)) + } +} + +impl<'a, S: StateStore, SD: ValueRowSerde + 'a> Ord for Node<'a, S, SD> { + fn cmp(&self, other: &Self) -> std::cmp::Ordering { + // The heap is a max heap, so we need to reverse the order. + self.peeked.0.cmp(&other.peeked.0).reverse() + } +} + /// `StateTableInner` is the interface accessing relational data in KV(`StateStore`) with /// row-based encoding. #[derive(Clone)] @@ -941,20 +974,37 @@ where .map(get_second)) } + #[try_stream(ok=OwnedRow, error=StreamExecutorError)] pub async fn iter_ordered_with_pk_range( &self, pk_range: &(Bound, Bound), - // Optional vnode that returns an iterator only over the given range under that vnode. - // For now, we require this parameter, and will panic. In the future, when `None`, we can - // iterate over each vnode that the `StateTableInner` owns. prefetch_options: PrefetchOptions, - ) -> StreamExecutorResult> { - let stream = self - .iter_key_and_val_with_pk_range(pk_range, VirtualNode::ZERO, prefetch_options) - .await? - .map(get_second); - let stream = merge_sort(vec![stream]).await; - Ok(stream) + ) { + yield todo!() + // // FIXME: Just iterate over all vnode ranges using: + // // self.vnodes.vnode_ranges() + // let stream = self + // .iter_key_and_val_with_pk_range(pk_range, VirtualNode::ZERO, prefetch_options) + // .await?; + // let streams = vec![Box::pin(stream)]; + // let mut heap = BinaryHeap::with_capacity(streams.len()); + // for mut stream in streams { + // if let Some(peeked) = stream.next().await.transpose()? { + // heap.push(Node { stream, peeked }); + // } + // } + // + // while let Some(mut node) = heap.peek_mut() { + // // Note: If the `next` returns `Err`, we'll fail to yield the previous item. + // // This is acceptable since we're not going to handle errors from cell-based table + // // iteration, so where to fail does not matter. Or we need an `Option` for this. + // yield match node.stream.next().await.transpose()? { + // // There still remains data in the stream, take and update the peeked value. + // Some(new_peeked) => std::mem::replace(&mut node.peeked, new_peeked).1, + // // This stream is exhausted, remove it from the heap. + // None => PeekMut::pop(node).peeked.1, + // }; + // } } pub async fn iter_key_and_val_with_pk_range( @@ -1100,34 +1150,6 @@ where } } -// FIXME: Try merge this with impl from batch/iter_utils.rs - -/// Merge multiple streams of primary key and rows into a single stream, sorted by primary key. -/// We should ensure that the primary key from different streams are unique. -#[try_stream(ok = OwnedRow, error = StreamExecutorError)] -pub(super) async fn merge_sort( - streams: Vec> + Unpin>, -) { - let mut heap = BinaryHeap::with_capacity(streams.len()); - for mut stream in streams { - if let Some(peeked) = stream.next().await.transpose()? { - heap.push(Node { stream, peeked }); - } - } - - while let Some(mut node) = heap.peek_mut() { - // Note: If the `next` returns `Err`, we'll fail to yield the previous item. - // This is acceptable since we're not going to handle errors from cell-based table - // iteration, so where to fail does not matter. Or we need an `Option` for this. - yield match node.stream.next().await.transpose()? { - // There still remains data in the stream, take and update the peeked value. - Some(new_peeked) => std::mem::replace(&mut node.peeked, new_peeked), - // This stream is exhausted, remove it from the heap. - None => PeekMut::pop(node).peeked, - }; - } -} - // FIXME: Try merge this with the impl in `src/table/mod.rs`. pub async fn collect_data_chunk( mut stream: impl Stream> + Unpin, diff --git a/src/stream/src/executor/arrangement_backfill.rs b/src/stream/src/executor/arrangement_backfill.rs index 56ba18a6b1df5..7b958c470c5d7 100644 --- a/src/stream/src/executor/arrangement_backfill.rs +++ b/src/stream/src/executor/arrangement_backfill.rs @@ -441,9 +441,7 @@ where } else { (Bound::Unbounded, Bound::Unbounded) }; - let mut iter = upstream_table - .iter_ordered_with_pk_range(&range_bounds, Default::default()) - .await?; + let mut iter = upstream_table.iter_ordered_with_pk_range(&range_bounds, Default::default()); pin_mut!(iter); for data_chunk in collect_data_chunk(iter, &schema, Some(CHUNK_SIZE)) .instrument_await("arrangement_backfill_snapshot_read") From f55d37577e638bc6d66d8a8c244c99c6552dc68e Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Wed, 14 Jun 2023 16:13:59 +0800 Subject: [PATCH 10/90] tmp commit --- src/stream/src/common/table/state_table.rs | 62 +++++++++++++--------- 1 file changed, 36 insertions(+), 26 deletions(-) diff --git a/src/stream/src/common/table/state_table.rs b/src/stream/src/common/table/state_table.rs index 08ae1bf0edfa8..dc6de4370b4ca 100644 --- a/src/stream/src/common/table/state_table.rs +++ b/src/stream/src/common/table/state_table.rs @@ -975,36 +975,19 @@ where } #[try_stream(ok=OwnedRow, error=StreamExecutorError)] - pub async fn iter_ordered_with_pk_range( + pub async fn iter_all_with_pk_range( &self, pk_range: &(Bound, Bound), prefetch_options: PrefetchOptions, ) { - yield todo!() - // // FIXME: Just iterate over all vnode ranges using: - // // self.vnodes.vnode_ranges() - // let stream = self - // .iter_key_and_val_with_pk_range(pk_range, VirtualNode::ZERO, prefetch_options) - // .await?; - // let streams = vec![Box::pin(stream)]; - // let mut heap = BinaryHeap::with_capacity(streams.len()); - // for mut stream in streams { - // if let Some(peeked) = stream.next().await.transpose()? { - // heap.push(Node { stream, peeked }); - // } - // } - // - // while let Some(mut node) = heap.peek_mut() { - // // Note: If the `next` returns `Err`, we'll fail to yield the previous item. - // // This is acceptable since we're not going to handle errors from cell-based table - // // iteration, so where to fail does not matter. Or we need an `Option` for this. - // yield match node.stream.next().await.transpose()? { - // // There still remains data in the stream, take and update the peeked value. - // Some(new_peeked) => std::mem::replace(&mut node.peeked, new_peeked).1, - // // This stream is exhausted, remove it from the heap. - // None => PeekMut::pop(node).peeked.1, - // }; - // } + let stream = self + .iter_key_and_val_with_pk_range(pk_range, VirtualNode::ZERO, prefetch_options) + .await?; + #[for_await] + for value in stream { + let v = value?; + yield v.1; + } } pub async fn iter_key_and_val_with_pk_range( @@ -1248,3 +1231,30 @@ fn to_memcomparable( } } } + +// /// Merge multiple streams of primary key and rows into a single stream, sorted by primary key. +// /// We should ensure that the primary key from different streams are unique. +// #[try_stream(ok = (Vec, OwnedRow), error = StorageError)] +// pub(super) async fn merge_sort(streams: Vec) +// where +// S: PkAndRowStream + Unpin, +// { +// let mut heap = BinaryHeap::with_capacity(streams.len()); +// for mut stream in streams { +// if let Some(peeked) = stream.next().await.transpose()? { +// heap.push(Node { stream, peeked }); +// } +// } +// +// while let Some(mut node) = heap.peek_mut() { +// // Note: If the `next` returns `Err`, we'll fail to yield the previous item. +// // This is acceptable since we're not going to handle errors from cell-based table +// // iteration, so where to fail does not matter. Or we need an `Option` for this. +// yield match node.stream.next().await.transpose()? { +// // There still remains data in the stream, take and update the peeked value. +// Some(new_peeked) => std::mem::replace(&mut node.peeked, new_peeked), +// // This stream is exhausted, remove it from the heap. +// None => PeekMut::pop(node).peeked, +// }; +// } +// } From 098bedfcb10e30e88b7fdab61b378999781ea453 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Wed, 14 Jun 2023 18:25:16 +0800 Subject: [PATCH 11/90] hack --- src/stream/src/common/table/state_table.rs | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/src/stream/src/common/table/state_table.rs b/src/stream/src/common/table/state_table.rs index dc6de4370b4ca..d20c7f701256c 100644 --- a/src/stream/src/common/table/state_table.rs +++ b/src/stream/src/common/table/state_table.rs @@ -975,9 +975,10 @@ where } #[try_stream(ok=OwnedRow, error=StreamExecutorError)] - pub async fn iter_all_with_pk_range( - &self, - pk_range: &(Bound, Bound), + pub async fn iter_ordered_with_pk_range( + // FIXME + &'_async0 self, + pk_range: &(Bound, Bound), prefetch_options: PrefetchOptions, ) { let stream = self From 2b827c57d8b45ad20b66da26d939d278f4db0cd5 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Wed, 14 Jun 2023 23:04:20 +0800 Subject: [PATCH 12/90] ok works lol --- src/stream/src/common/table/state_table.rs | 29 ++++++++++++++++------ 1 file changed, 22 insertions(+), 7 deletions(-) diff --git a/src/stream/src/common/table/state_table.rs b/src/stream/src/common/table/state_table.rs index d20c7f701256c..b50aae48bf009 100644 --- a/src/stream/src/common/table/state_table.rs +++ b/src/stream/src/common/table/state_table.rs @@ -21,7 +21,7 @@ use std::sync::Arc; use bytes::{BufMut, Bytes, BytesMut}; use futures::{Stream, StreamExt}; -use futures_async_stream::try_stream; +use futures_async_stream::{for_await, try_stream}; use itertools::{izip, Itertools}; use risingwave_common::array::stream_record::Record; use risingwave_common::array::{DataChunk, Op, StreamChunk, Vis}; @@ -63,15 +63,15 @@ use crate::executor::{StreamExecutorError, StreamExecutorResult}; const STATE_CLEANING_PERIOD_EPOCH: usize = 5; // FIXME: This is duplicated from batch iter utils. -struct Node<'a, S: StateStore, SD: ValueRowSerde + 'a> { - stream: RowStreamWithPk<'a, S, SD>, +pub struct Node { + stream: S, /// The next item polled from `stream` previously. Since the `eq` and `cmp` must be synchronous /// functions, we need to implement peeking manually. peeked: (Bytes, OwnedRow), } -impl<'a, S: StateStore, SD: ValueRowSerde + 'a> PartialEq for Node<'a, S, SD> { +impl PartialEq for Node { fn eq(&self, other: &Self) -> bool { match self.peeked.0 == other.peeked.0 { true => unreachable!("primary key from different iters should be unique"), @@ -79,15 +79,15 @@ impl<'a, S: StateStore, SD: ValueRowSerde + 'a> PartialEq for Node<'a, S, SD> { } } } -impl<'a, S: StateStore, SD: ValueRowSerde + 'a> Eq for Node<'a, S, SD> {} +impl Eq for Node {} -impl<'a, S: StateStore, SD: ValueRowSerde + 'a> PartialOrd for Node<'a, S, SD> { +impl PartialOrd for Node { fn partial_cmp(&self, other: &Self) -> Option { Some(self.cmp(other)) } } -impl<'a, S: StateStore, SD: ValueRowSerde + 'a> Ord for Node<'a, S, SD> { +impl Ord for Node { fn cmp(&self, other: &Self) -> std::cmp::Ordering { // The heap is a max heap, so we need to reverse the order. self.peeked.0.cmp(&other.peeked.0).reverse() @@ -991,6 +991,21 @@ where } } + pub async fn collect_heap<'a, R>(streams: Vec) -> StreamExecutorResult>> + where + S: 'a, + SD: 'a, + R: Stream> + 'a + Unpin, + { + let mut heap = BinaryHeap::new(); + for mut stream in streams { + if let Some(peeked) = stream.next().await.transpose()? { + heap.push(Node { stream, peeked }); + } + } + Ok(heap) + } + pub async fn iter_key_and_val_with_pk_range( &self, pk_range: &(Bound, Bound), From c9ed209369f65bc3146bb4f671ba682203ec37da Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Wed, 14 Jun 2023 23:07:41 +0800 Subject: [PATCH 13/90] finally last part of merge sort --- src/stream/src/common/table/state_table.rs | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) diff --git a/src/stream/src/common/table/state_table.rs b/src/stream/src/common/table/state_table.rs index b50aae48bf009..05bd0ae75c899 100644 --- a/src/stream/src/common/table/state_table.rs +++ b/src/stream/src/common/table/state_table.rs @@ -1006,6 +1006,24 @@ where Ok(heap) } + #[try_stream(ok=(Bytes, OwnedRow), error=StreamExecutorError)] + pub async fn iter_heap(mut heap: BinaryHeap>) + where + R: Stream> + Unpin, + { + while let Some(mut node) = heap.peek_mut() { + // Note: If the `next` returns `Err`, we'll fail to yield the previous item. + // This is acceptable since we're not going to handle errors from cell-based table + // iteration, so where to fail does not matter. Or we need an `Option` for this. + yield match node.stream.next().await.transpose()? { + // There still remains data in the stream, take and update the peeked value. + Some(new_peeked) => std::mem::replace(&mut node.peeked, new_peeked), + // This stream is exhausted, remove it from the heap. + None => PeekMut::pop(node).peeked, + }; + } + } + pub async fn iter_key_and_val_with_pk_range( &self, pk_range: &(Bound, Bound), From 57f1eddc75d9f917ab574ea5a14a724b245abf2c Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Thu, 15 Jun 2023 11:36:49 +0800 Subject: [PATCH 14/90] make it compile with arrangement backfill too --- src/stream/src/common/table/state_table.rs | 145 ++++++++++++------ .../src/executor/arrangement_backfill.rs | 9 +- 2 files changed, 104 insertions(+), 50 deletions(-) diff --git a/src/stream/src/common/table/state_table.rs b/src/stream/src/common/table/state_table.rs index 05bd0ae75c899..da6eac06adcb3 100644 --- a/src/stream/src/common/table/state_table.rs +++ b/src/stream/src/common/table/state_table.rs @@ -974,55 +974,36 @@ where .map(get_second)) } - #[try_stream(ok=OwnedRow, error=StreamExecutorError)] - pub async fn iter_ordered_with_pk_range( - // FIXME - &'_async0 self, - pk_range: &(Bound, Bound), + pub async fn iter_all_vnode_ranges( + &self, + pk_range: &(Bound, Bound), prefetch_options: PrefetchOptions, - ) { - let stream = self - .iter_key_and_val_with_pk_range(pk_range, VirtualNode::ZERO, prefetch_options) - .await?; - #[for_await] - for value in stream { - let v = value?; - yield v.1; - } - } - - pub async fn collect_heap<'a, R>(streams: Vec) -> StreamExecutorResult>> - where - S: 'a, - SD: 'a, - R: Stream> + 'a + Unpin, - { - let mut heap = BinaryHeap::new(); - for mut stream in streams { - if let Some(peeked) = stream.next().await.transpose()? { - heap.push(Node { stream, peeked }); - } + ) -> StreamExecutorResult>> { + let mut vec = Vec::with_capacity(self.vnodes.count_ones()); + for vnode in self.vnodes.iter_vnodes() { + vec.push( + self.iter_key_and_val_with_pk_range(pk_range, vnode, prefetch_options) + .await?, + ) } - Ok(heap) + Ok(vec) } - #[try_stream(ok=(Bytes, OwnedRow), error=StreamExecutorError)] - pub async fn iter_heap(mut heap: BinaryHeap>) - where - R: Stream> + Unpin, - { - while let Some(mut node) = heap.peek_mut() { - // Note: If the `next` returns `Err`, we'll fail to yield the previous item. - // This is acceptable since we're not going to handle errors from cell-based table - // iteration, so where to fail does not matter. Or we need an `Option` for this. - yield match node.stream.next().await.transpose()? { - // There still remains data in the stream, take and update the peeked value. - Some(new_peeked) => std::mem::replace(&mut node.peeked, new_peeked), - // This stream is exhausted, remove it from the heap. - None => PeekMut::pop(node).peeked, - }; - } - } + // #[try_stream(ok=OwnedRow, error=StreamExecutorError)] + // pub async fn iter_ordered_with_pk_ranges( + // &self, + // pk_range: &(Bound, Bound), + // prefetch_options: PrefetchOptions, + // ) { + // let iterators = self + // .iter_all_vnode_ranges(pk_range, prefetch_options) + // .await?; + // let stream = Self::merge_sort(iterators); + // #[for_await] + // for row in stream { + // yield row?; + // } + // } pub async fn iter_key_and_val_with_pk_range( &self, @@ -1168,11 +1149,14 @@ where } // FIXME: Try merge this with the impl in `src/table/mod.rs`. -pub async fn collect_data_chunk( - mut stream: impl Stream> + Unpin, +pub async fn collect_data_chunk( + mut stream: S, schema: &Schema, chunk_size: Option, -) -> StreamExecutorResult> { +) -> StreamExecutorResult> +where + S: Stream> + Unpin, +{ let mut builders = schema.create_array_builders(chunk_size.unwrap_or(0)); let mut row_count = 0; @@ -1292,3 +1276,68 @@ fn to_memcomparable( // }; // } // } +// +// pub async fn merge_sort<'a, R, V>(streams: Vec) -> StreamExecutorResult +// where +// R: Stream> + 'a + Unpin, +// V: Stream> + 'a + Unpin, +// { +// let mut heap = collect_heap(streams).await?; +// let s = iter_heap(heap); +// Ok(s) +// } +// +// pub async fn collect_heap<'a, R>(streams: Vec) -> StreamExecutorResult>> +// where +// R: Stream> + 'a + Unpin, +// { +// let mut heap = BinaryHeap::new(); +// for mut stream in streams { +// if let Some(peeked) = stream.next().await.transpose()? { +// heap.push(Node { stream, peeked }); +// } +// } +// Ok(heap) +// } +// +// #[try_stream(ok=OwnedRow, error=StreamExecutorError)] +// pub async fn iter_heap(mut heap: BinaryHeap>) +// where +// R: Stream> + Unpin, +// { +// while let Some(mut node) = heap.peek_mut() { +// // Note: If the `next` returns `Err`, we'll fail to yield the previous item. +// // This is acceptable since we're not going to handle errors from cell-based table +// // iteration, so where to fail does not matter. Or we need an `Option` for this. +// yield match node.stream.next().await.transpose()? { +// // There still remains data in the stream, take and update the peeked value. +// Some(new_peeked) => std::mem::replace(&mut node.peeked, new_peeked).1, +// // This stream is exhausted, remove it from the heap. +// None => PeekMut::pop(node).peeked.1, +// }; +// } +// } + +#[try_stream(ok=OwnedRow, error=StreamExecutorError)] +pub async fn merge_sort<'a, R>(streams: Vec) +where + R: Stream> + 'a + Unpin, +{ + let mut heap = BinaryHeap::new(); + for mut stream in streams { + if let Some(peeked) = stream.next().await.transpose()? { + heap.push(Node { stream, peeked }); + } + } + while let Some(mut node) = heap.peek_mut() { + // Note: If the `next` returns `Err`, we'll fail to yield the previous item. + // This is acceptable since we're not going to handle errors from cell-based table + // iteration, so where to fail does not matter. Or we need an `Option` for this. + yield match node.stream.next().await.transpose()? { + // There still remains data in the stream, take and update the peeked value. + Some(new_peeked) => std::mem::replace(&mut node.peeked, new_peeked).1, + // This stream is exhausted, remove it from the heap. + None => PeekMut::pop(node).peeked.1, + }; + } +} diff --git a/src/stream/src/executor/arrangement_backfill.rs b/src/stream/src/executor/arrangement_backfill.rs index 7b958c470c5d7..4d89d120217af 100644 --- a/src/stream/src/executor/arrangement_backfill.rs +++ b/src/stream/src/executor/arrangement_backfill.rs @@ -22,6 +22,7 @@ use either::Either; use futures::stream::select_with_strategy; use futures::{pin_mut, stream, StreamExt, TryStreamExt}; use futures_async_stream::try_stream; +use itertools::Itertools; use risingwave_common::array::stream_record::Record; use risingwave_common::array::{Op, StreamChunk}; use risingwave_common::buffer::BitmapBuilder; @@ -41,7 +42,7 @@ use risingwave_storage::StateStore; use super::error::StreamExecutorError; use super::{expect_first_barrier, BoxedExecutor, Executor, ExecutorInfo, Message, PkIndicesRef}; -use crate::common::table::state_table::{collect_data_chunk, StateTable}; +use crate::common::table::state_table::{collect_data_chunk, merge_sort, StateTable}; use crate::executor::monitor::StreamingMetrics; use crate::executor::{PkIndices, StreamExecutorResult, Watermark}; use crate::task::{ActorId, CreateMviewProgress}; @@ -441,7 +442,11 @@ where } else { (Bound::Unbounded, Bound::Unbounded) }; - let mut iter = upstream_table.iter_ordered_with_pk_range(&range_bounds, Default::default()); + let iterators = upstream_table + .iter_all_vnode_ranges(&range_bounds, Default::default()) + .await?; + let pinned_iter: Vec<_> = iterators.into_iter().map(Box::pin).collect_vec(); + let iter = merge_sort(pinned_iter); pin_mut!(iter); for data_chunk in collect_data_chunk(iter, &schema, Some(CHUNK_SIZE)) .instrument_await("arrangement_backfill_snapshot_read") From 7ea85036b480d6c940eb0144a6daf97861730ddf Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Thu, 15 Jun 2023 15:31:42 +0800 Subject: [PATCH 15/90] refactor merge_sort to separate module --- .../src/table/batch_table/storage_table.rs | 2 +- src/storage/src/table/mod.rs | 6 +- src/stream/src/common/table/iter_utils.rs | 125 +++++++++++++++ src/stream/src/common/table/mod.rs | 1 + src/stream/src/common/table/state_table.rs | 151 ++---------------- .../src/executor/arrangement_backfill.rs | 3 +- 6 files changed, 143 insertions(+), 145 deletions(-) create mode 100644 src/stream/src/common/table/iter_utils.rs diff --git a/src/storage/src/table/batch_table/storage_table.rs b/src/storage/src/table/batch_table/storage_table.rs index 1e1d2d7af60b4..69c17c7a9c335 100644 --- a/src/storage/src/table/batch_table/storage_table.rs +++ b/src/storage/src/table/batch_table/storage_table.rs @@ -402,7 +402,7 @@ pub trait PkAndRowStream = Stream, OwnedRow)>> + S pub type StorageTableInnerIter = impl PkAndRowStream; #[async_trait::async_trait] -impl TableIter for S { +impl TableIter for S { async fn next_row(&mut self) -> StorageResult> { self.next() .await diff --git a/src/storage/src/table/mod.rs b/src/storage/src/table/mod.rs index a03defe9922c8..e43487ece3773 100644 --- a/src/storage/src/table/mod.rs +++ b/src/storage/src/table/mod.rs @@ -79,14 +79,14 @@ impl Distribution { // TODO: GAT-ify this trait or remove this trait #[async_trait::async_trait] -pub trait TableIter: Send { - async fn next_row(&mut self) -> StorageResult>; +pub trait TableIter: Send { + async fn next_row(&mut self) -> Result, E>; async fn collect_data_chunk( &mut self, schema: &Schema, chunk_size: Option, - ) -> StorageResult> { + ) -> Result, E> { let mut builders = schema.create_array_builders(chunk_size.unwrap_or(0)); let mut row_count = 0; diff --git a/src/stream/src/common/table/iter_utils.rs b/src/stream/src/common/table/iter_utils.rs new file mode 100644 index 0000000000000..c735705aace9a --- /dev/null +++ b/src/stream/src/common/table/iter_utils.rs @@ -0,0 +1,125 @@ +// Copyright 2023 RisingWave 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. + +//! FIXME: This is duplicated from batch iter utils, +//! and slightly modified to change the return error + the public key type. +//! Seems difficult to unify them, since batch uses `StorageError` for its iterators, +//! whereas stream uses `StreamExecutorError` for its iterators. +//! The underlying error type should be `StorageError` as well for stream. +//! Additionally since we use `futures_async_stream`'s `try_stream` macro, +//! the return type has to be concrete, so it can't be parameterized on the PK type. +//! Since batch and stream have different PK types (one is Bytes, other is Vec), +//! this also means we have to find some other mechanism to parameterize them, perhaps a macro. + +use std::collections::binary_heap::PeekMut; +use std::collections::BinaryHeap; +use std::ops::Bound; +use std::ops::Bound::*; +use std::pin::Pin; +use std::sync::Arc; + +use bytes::{BufMut, Bytes, BytesMut}; +use futures::{Stream, StreamExt}; +use futures_async_stream::{for_await, try_stream}; +use itertools::{izip, Itertools}; +use risingwave_common::array::stream_record::Record; +use risingwave_common::array::{DataChunk, Op, StreamChunk, Vis}; +use risingwave_common::buffer::Bitmap; +use risingwave_common::cache::CachePriority; +use risingwave_common::catalog::{ + get_dist_key_in_pk_indices, ColumnDesc, Schema, TableId, TableOption, +}; +use risingwave_common::hash::{VirtualNode, VnodeBitmapExt}; +use risingwave_common::row::{self, CompactedRow, OwnedRow, Row, RowExt}; +use risingwave_common::types::ScalarImpl; +use risingwave_common::util::epoch::EpochPair; +use risingwave_common::util::iter_util::{ZipEqDebug, ZipEqFast}; +use risingwave_common::util::row_serde::OrderedRowSerde; +use risingwave_common::util::sort_util::OrderType; +use risingwave_common::util::value_encoding::{BasicSerde, ValueRowSerde}; +use risingwave_hummock_sdk::key::{ + end_bound_of_prefix, next_key, prefixed_range, range_of_prefix, start_bound_of_excluded_prefix, +}; +use risingwave_pb::catalog::Table; +use risingwave_storage::error::{StorageError, StorageResult}; +use risingwave_storage::hummock::CachePolicy; +use risingwave_storage::mem_table::MemTableError; +use risingwave_storage::row_serde::row_serde_util::{ + deserialize_pk_with_vnode, serialize_pk, serialize_pk_with_vnode, +}; +use risingwave_storage::store::{ + LocalStateStore, NewLocalOptions, PrefetchOptions, ReadOptions, StateStoreIterItemStream, +}; +use risingwave_storage::table::{compute_chunk_vnode, compute_vnode, Distribution, TableIter}; +use risingwave_storage::StateStore; +use tracing::trace; + +use super::watermark::{WatermarkBufferByEpoch, WatermarkBufferStrategy}; +use crate::cache::cache_may_stale; +use crate::executor::{StreamExecutorError, StreamExecutorResult}; + +pub struct Node { + stream: S, + + /// The next item polled from `stream` previously. Since the `eq` and `cmp` must be synchronous + /// functions, we need to implement peeking manually. + peeked: (Bytes, OwnedRow), +} + +impl PartialEq for Node { + fn eq(&self, other: &Self) -> bool { + match self.peeked.0 == other.peeked.0 { + true => unreachable!("primary key from different iters should be unique"), + false => false, + } + } +} +impl Eq for Node {} + +impl PartialOrd for Node { + fn partial_cmp(&self, other: &Self) -> Option { + Some(self.cmp(other)) + } +} + +impl Ord for Node { + fn cmp(&self, other: &Self) -> std::cmp::Ordering { + // The heap is a max heap, so we need to reverse the order. + self.peeked.0.cmp(&other.peeked.0).reverse() + } +} + +#[try_stream(ok=OwnedRow, error=StreamExecutorError)] +pub async fn merge_sort<'a, R>(streams: Vec) +where + R: Stream> + 'a + Unpin, +{ + let mut heap = BinaryHeap::new(); + for mut stream in streams { + if let Some(peeked) = stream.next().await.transpose()? { + heap.push(Node { stream, peeked }); + } + } + while let Some(mut node) = heap.peek_mut() { + // Note: If the `next` returns `Err`, we'll fail to yield the previous item. + // This is acceptable since we're not going to handle errors from cell-based table + // iteration, so where to fail does not matter. Or we need an `Option` for this. + yield match node.stream.next().await.transpose()? { + // There still remains data in the stream, take and update the peeked value. + Some(new_peeked) => std::mem::replace(&mut node.peeked, new_peeked).1, + // This stream is exhausted, remove it from the heap. + None => PeekMut::pop(node).peeked.1, + }; + } +} diff --git a/src/stream/src/common/table/mod.rs b/src/stream/src/common/table/mod.rs index 613ebb3436b02..ab509d5f44e1c 100644 --- a/src/stream/src/common/table/mod.rs +++ b/src/stream/src/common/table/mod.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +pub mod iter_utils; pub mod state_table; mod watermark; diff --git a/src/stream/src/common/table/state_table.rs b/src/stream/src/common/table/state_table.rs index da6eac06adcb3..f2db3c5551a79 100644 --- a/src/stream/src/common/table/state_table.rs +++ b/src/stream/src/common/table/state_table.rs @@ -62,38 +62,6 @@ use crate::executor::{StreamExecutorError, StreamExecutorResult}; /// This num is arbitrary and we may want to improve this choice in the future. const STATE_CLEANING_PERIOD_EPOCH: usize = 5; -// FIXME: This is duplicated from batch iter utils. -pub struct Node { - stream: S, - - /// The next item polled from `stream` previously. Since the `eq` and `cmp` must be synchronous - /// functions, we need to implement peeking manually. - peeked: (Bytes, OwnedRow), -} - -impl PartialEq for Node { - fn eq(&self, other: &Self) -> bool { - match self.peeked.0 == other.peeked.0 { - true => unreachable!("primary key from different iters should be unique"), - false => false, - } - } -} -impl Eq for Node {} - -impl PartialOrd for Node { - fn partial_cmp(&self, other: &Self) -> Option { - Some(self.cmp(other)) - } -} - -impl Ord for Node { - fn cmp(&self, other: &Self) -> std::cmp::Ordering { - // The heap is a max heap, so we need to reverse the order. - self.peeked.0.cmp(&other.peeked.0).reverse() - } -} - /// `StateTableInner` is the interface accessing relational data in KV(`StateStore`) with /// row-based encoding. #[derive(Clone)] @@ -989,22 +957,6 @@ where Ok(vec) } - // #[try_stream(ok=OwnedRow, error=StreamExecutorError)] - // pub async fn iter_ordered_with_pk_ranges( - // &self, - // pk_range: &(Bound, Bound), - // prefetch_options: PrefetchOptions, - // ) { - // let iterators = self - // .iter_all_vnode_ranges(pk_range, prefetch_options) - // .await?; - // let stream = Self::merge_sort(iterators); - // #[for_await] - // for row in stream { - // yield row?; - // } - // } - pub async fn iter_key_and_val_with_pk_range( &self, pk_range: &(Bound, Bound), @@ -1192,6 +1144,17 @@ pub type RowStream<'a, S: StateStore, SD: ValueRowSerde + 'a> = impl Stream> + 'a; pub type RowStreamWithPk<'a, S: StateStore, SD: ValueRowSerde + 'a> = impl Stream> + 'a; +// #[async_trait::async_trait] +// impl> + Unpin> +// TableIter for S +// { +// async fn next_row(&mut self) -> StreamExecutorResult> { +// self.next() +// .await +// .transpose() +// .map(|r| r.map(|(_pk, row)| row)) +// } +// } fn deserialize_row_stream<'a>( stream: impl StateStoreIterItemStream + 'a, @@ -1249,95 +1212,3 @@ fn to_memcomparable( } } } - -// /// Merge multiple streams of primary key and rows into a single stream, sorted by primary key. -// /// We should ensure that the primary key from different streams are unique. -// #[try_stream(ok = (Vec, OwnedRow), error = StorageError)] -// pub(super) async fn merge_sort(streams: Vec) -// where -// S: PkAndRowStream + Unpin, -// { -// let mut heap = BinaryHeap::with_capacity(streams.len()); -// for mut stream in streams { -// if let Some(peeked) = stream.next().await.transpose()? { -// heap.push(Node { stream, peeked }); -// } -// } -// -// while let Some(mut node) = heap.peek_mut() { -// // Note: If the `next` returns `Err`, we'll fail to yield the previous item. -// // This is acceptable since we're not going to handle errors from cell-based table -// // iteration, so where to fail does not matter. Or we need an `Option` for this. -// yield match node.stream.next().await.transpose()? { -// // There still remains data in the stream, take and update the peeked value. -// Some(new_peeked) => std::mem::replace(&mut node.peeked, new_peeked), -// // This stream is exhausted, remove it from the heap. -// None => PeekMut::pop(node).peeked, -// }; -// } -// } -// -// pub async fn merge_sort<'a, R, V>(streams: Vec) -> StreamExecutorResult -// where -// R: Stream> + 'a + Unpin, -// V: Stream> + 'a + Unpin, -// { -// let mut heap = collect_heap(streams).await?; -// let s = iter_heap(heap); -// Ok(s) -// } -// -// pub async fn collect_heap<'a, R>(streams: Vec) -> StreamExecutorResult>> -// where -// R: Stream> + 'a + Unpin, -// { -// let mut heap = BinaryHeap::new(); -// for mut stream in streams { -// if let Some(peeked) = stream.next().await.transpose()? { -// heap.push(Node { stream, peeked }); -// } -// } -// Ok(heap) -// } -// -// #[try_stream(ok=OwnedRow, error=StreamExecutorError)] -// pub async fn iter_heap(mut heap: BinaryHeap>) -// where -// R: Stream> + Unpin, -// { -// while let Some(mut node) = heap.peek_mut() { -// // Note: If the `next` returns `Err`, we'll fail to yield the previous item. -// // This is acceptable since we're not going to handle errors from cell-based table -// // iteration, so where to fail does not matter. Or we need an `Option` for this. -// yield match node.stream.next().await.transpose()? { -// // There still remains data in the stream, take and update the peeked value. -// Some(new_peeked) => std::mem::replace(&mut node.peeked, new_peeked).1, -// // This stream is exhausted, remove it from the heap. -// None => PeekMut::pop(node).peeked.1, -// }; -// } -// } - -#[try_stream(ok=OwnedRow, error=StreamExecutorError)] -pub async fn merge_sort<'a, R>(streams: Vec) -where - R: Stream> + 'a + Unpin, -{ - let mut heap = BinaryHeap::new(); - for mut stream in streams { - if let Some(peeked) = stream.next().await.transpose()? { - heap.push(Node { stream, peeked }); - } - } - while let Some(mut node) = heap.peek_mut() { - // Note: If the `next` returns `Err`, we'll fail to yield the previous item. - // This is acceptable since we're not going to handle errors from cell-based table - // iteration, so where to fail does not matter. Or we need an `Option` for this. - yield match node.stream.next().await.transpose()? { - // There still remains data in the stream, take and update the peeked value. - Some(new_peeked) => std::mem::replace(&mut node.peeked, new_peeked).1, - // This stream is exhausted, remove it from the heap. - None => PeekMut::pop(node).peeked.1, - }; - } -} diff --git a/src/stream/src/executor/arrangement_backfill.rs b/src/stream/src/executor/arrangement_backfill.rs index 4d89d120217af..e7bcd9629d2cb 100644 --- a/src/stream/src/executor/arrangement_backfill.rs +++ b/src/stream/src/executor/arrangement_backfill.rs @@ -42,7 +42,8 @@ use risingwave_storage::StateStore; use super::error::StreamExecutorError; use super::{expect_first_barrier, BoxedExecutor, Executor, ExecutorInfo, Message, PkIndicesRef}; -use crate::common::table::state_table::{collect_data_chunk, merge_sort, StateTable}; +use crate::common::table::iter_utils::merge_sort; +use crate::common::table::state_table::{collect_data_chunk, StateTable}; use crate::executor::monitor::StreamingMetrics; use crate::executor::{PkIndices, StreamExecutorResult, Watermark}; use crate::task::{ActorId, CreateMviewProgress}; From b7b5173a9d25719b80d122577dea9070c8b89771 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Thu, 15 Jun 2023 15:56:28 +0800 Subject: [PATCH 16/90] unified collect_chunk somewhat --- src/batch/src/executor/row_seq_scan.rs | 8 +- .../src/table/batch_table/storage_table.rs | 1 - src/storage/src/table/mod.rs | 109 ++++++++++++------ src/stream/src/common/table/state_table.rs | 98 +++++++--------- .../src/executor/arrangement_backfill.rs | 8 +- src/stream/src/executor/backfill.rs | 15 +-- src/stream/src/executor/batch_query.rs | 15 +-- 7 files changed, 142 insertions(+), 112 deletions(-) diff --git a/src/batch/src/executor/row_seq_scan.rs b/src/batch/src/executor/row_seq_scan.rs index 3b1a477407b57..26c3a13e90556 100644 --- a/src/batch/src/executor/row_seq_scan.rs +++ b/src/batch/src/executor/row_seq_scan.rs @@ -34,7 +34,7 @@ use risingwave_pb::common::BatchQueryEpoch; use risingwave_pb::plan_common::StorageTableDesc; use risingwave_storage::store::PrefetchOptions; use risingwave_storage::table::batch_table::storage_table::StorageTable; -use risingwave_storage::table::{Distribution, TableIter}; +use risingwave_storage::table::{collect_data_chunk, get_second, Distribution, TableIter}; use risingwave_storage::{dispatch_state_store, StateStore}; use crate::executor::{ @@ -416,14 +416,14 @@ impl RowSeqScanExecutor { ordered, PrefetchOptions::new_for_exhaust_iter(), ) - .await?; + .await? + .map(get_second); pin_mut!(iter); loop { let timer = histogram.as_ref().map(|histogram| histogram.start_timer()); - let chunk = iter - .collect_data_chunk(table.schema(), Some(chunk_size)) + let chunk = collect_data_chunk(&mut iter, table.schema(), Some(chunk_size)) .await .map_err(RwError::from)?; diff --git a/src/storage/src/table/batch_table/storage_table.rs b/src/storage/src/table/batch_table/storage_table.rs index 69c17c7a9c335..82ce2cf1874fe 100644 --- a/src/storage/src/table/batch_table/storage_table.rs +++ b/src/storage/src/table/batch_table/storage_table.rs @@ -400,7 +400,6 @@ pub trait PkAndRowStream = Stream, OwnedRow)>> + S /// The row iterator of the storage table. /// The wrapper of [`StorageTableInnerIter`] if pk is not persisted. pub type StorageTableInnerIter = impl PkAndRowStream; - #[async_trait::async_trait] impl TableIter for S { async fn next_row(&mut self) -> StorageResult> { diff --git a/src/storage/src/table/mod.rs b/src/storage/src/table/mod.rs index e43487ece3773..ad15ebecf1db9 100644 --- a/src/storage/src/table/mod.rs +++ b/src/storage/src/table/mod.rs @@ -16,6 +16,7 @@ pub mod batch_table; use std::sync::{Arc, LazyLock}; +use futures::{Stream, StreamExt}; use itertools::Itertools; use risingwave_common::array::DataChunk; use risingwave_common::buffer::{Bitmap, BitmapBuilder}; @@ -77,45 +78,87 @@ impl Distribution { } } -// TODO: GAT-ify this trait or remove this trait -#[async_trait::async_trait] -pub trait TableIter: Send { - async fn next_row(&mut self) -> Result, E>; +pub fn get_second(arg: Result<(T, U), E>) -> Result { + arg.map(|x| x.1) +} - async fn collect_data_chunk( - &mut self, - schema: &Schema, - chunk_size: Option, - ) -> Result, E> { - let mut builders = schema.create_array_builders(chunk_size.unwrap_or(0)); - - let mut row_count = 0; - for _ in 0..chunk_size.unwrap_or(usize::MAX) { - match self.next_row().await? { - Some(row) => { - for (datum, builder) in row.iter().zip_eq_fast(builders.iter_mut()) { - builder.append(datum); - } - row_count += 1; +// FIXME: Try merge this with the impl in `src/table/mod.rs`. +pub async fn collect_data_chunk( + stream: &mut S, + schema: &Schema, + chunk_size: Option, +) -> Result, E> +where + S: Stream> + Unpin, +{ + let mut builders = schema.create_array_builders(chunk_size.unwrap_or(0)); + + let mut row_count = 0; + for _ in 0..chunk_size.unwrap_or(usize::MAX) { + match stream.next().await.transpose()? { + Some(row) => { + for (datum, builder) in row.iter().zip_eq_fast(builders.iter_mut()) { + builder.append(datum); } - None => break, } + None => break, } - let chunk = { - let columns: Vec<_> = builders - .into_iter() - .map(|builder| builder.finish().into()) - .collect(); - DataChunk::new(columns, row_count) - }; - - if chunk.cardinality() == 0 { - Ok(None) - } else { - Ok(Some(chunk)) - } + row_count += 1; } + + let chunk = { + let columns: Vec<_> = builders + .into_iter() + .map(|builder| builder.finish().into()) + .collect(); + DataChunk::new(columns, row_count) + }; + + if chunk.cardinality() == 0 { + Ok(None) + } else { + Ok(Some(chunk)) + } +} +// TODO: GAT-ify this trait or remove this trait +#[async_trait::async_trait] +pub trait TableIter: Send { + async fn next_row(&mut self) -> Result, E>; + // async fn collect_data_chunk( + // &mut self, + // schema: &Schema, + // chunk_size: Option, + // ) -> Result, E> { + // let mut builders = schema.create_array_builders(chunk_size.unwrap_or(0)); + // + // let mut row_count = 0; + // for _ in 0..chunk_size.unwrap_or(usize::MAX) { + // match self.next_row().await? { + // Some(row) => { + // for (datum, builder) in row.iter().zip_eq_fast(builders.iter_mut()) { + // builder.append(datum); + // } + // row_count += 1; + // } + // None => break, + // } + // } + // + // let chunk = { + // let columns: Vec<_> = builders + // .into_iter() + // .map(|builder| builder.finish().into()) + // .collect(); + // DataChunk::new(columns, row_count) + // }; + // + // if chunk.cardinality() == 0 { + // Ok(None) + // } else { + // Ok(Some(chunk)) + // } + // } } /// Get vnode value with `indices` on the given `row`. diff --git a/src/stream/src/common/table/state_table.rs b/src/stream/src/common/table/state_table.rs index f2db3c5551a79..32ad813984727 100644 --- a/src/stream/src/common/table/state_table.rs +++ b/src/stream/src/common/table/state_table.rs @@ -51,7 +51,9 @@ use risingwave_storage::row_serde::row_serde_util::{ use risingwave_storage::store::{ LocalStateStore, NewLocalOptions, PrefetchOptions, ReadOptions, StateStoreIterItemStream, }; -use risingwave_storage::table::{compute_chunk_vnode, compute_vnode, Distribution, TableIter}; +use risingwave_storage::table::{ + compute_chunk_vnode, compute_vnode, get_second, Distribution, TableIter, +}; use risingwave_storage::StateStore; use tracing::trace; @@ -874,10 +876,6 @@ where } } -fn get_second(arg: StreamExecutorResult<(T, U)>) -> StreamExecutorResult { - arg.map(|x| x.1) -} - // Iterator functions impl StateTableInner where @@ -1099,62 +1097,50 @@ where .map_err(Into::into) } } - -// FIXME: Try merge this with the impl in `src/table/mod.rs`. -pub async fn collect_data_chunk( - mut stream: S, - schema: &Schema, - chunk_size: Option, -) -> StreamExecutorResult> -where - S: Stream> + Unpin, -{ - let mut builders = schema.create_array_builders(chunk_size.unwrap_or(0)); - - let mut row_count = 0; - for _ in 0..chunk_size.unwrap_or(usize::MAX) { - match stream.next().await.transpose()? { - Some(row) => { - for (datum, builder) in row.iter().zip_eq_fast(builders.iter_mut()) { - builder.append(datum); - } - } - None => break, - } - - row_count += 1; - } - - let chunk = { - let columns: Vec<_> = builders - .into_iter() - .map(|builder| builder.finish().into()) - .collect(); - DataChunk::new(columns, row_count) - }; - - if chunk.cardinality() == 0 { - Ok(None) - } else { - Ok(Some(chunk)) - } -} +// // FIXME: Try merge this with the impl in `src/table/mod.rs`. +// pub async fn collect_data_chunk( +// mut stream: S, +// schema: &Schema, +// chunk_size: Option, +// ) -> StreamExecutorResult> +// where +// S: Stream> + Unpin, +// { +// let mut builders = schema.create_array_builders(chunk_size.unwrap_or(0)); +// +// let mut row_count = 0; +// for _ in 0..chunk_size.unwrap_or(usize::MAX) { +// match stream.next().await.transpose()? { +// Some(row) => { +// for (datum, builder) in row.iter().zip_eq_fast(builders.iter_mut()) { +// builder.append(datum); +// } +// } +// None => break, +// } +// +// row_count += 1; +// } +// +// let chunk = { +// let columns: Vec<_> = builders +// .into_iter() +// .map(|builder| builder.finish().into()) +// .collect(); +// DataChunk::new(columns, row_count) +// }; +// +// if chunk.cardinality() == 0 { +// Ok(None) +// } else { +// Ok(Some(chunk)) +// } +// } pub type RowStream<'a, S: StateStore, SD: ValueRowSerde + 'a> = impl Stream> + 'a; pub type RowStreamWithPk<'a, S: StateStore, SD: ValueRowSerde + 'a> = impl Stream> + 'a; -// #[async_trait::async_trait] -// impl> + Unpin> -// TableIter for S -// { -// async fn next_row(&mut self) -> StreamExecutorResult> { -// self.next() -// .await -// .transpose() -// .map(|r| r.map(|(_pk, row)| row)) -// } -// } fn deserialize_row_stream<'a>( stream: impl StateStoreIterItemStream + 'a, diff --git a/src/stream/src/executor/arrangement_backfill.rs b/src/stream/src/executor/arrangement_backfill.rs index e7bcd9629d2cb..42b2c4328abbf 100644 --- a/src/stream/src/executor/arrangement_backfill.rs +++ b/src/stream/src/executor/arrangement_backfill.rs @@ -37,13 +37,13 @@ use risingwave_hummock_sdk::HummockReadEpoch; use risingwave_storage::error::StorageResult; use risingwave_storage::store::PrefetchOptions; use risingwave_storage::table::batch_table::storage_table::{StorageTable, StorageTableInnerIter}; -use risingwave_storage::table::TableIter; +use risingwave_storage::table::{collect_data_chunk, TableIter}; use risingwave_storage::StateStore; use super::error::StreamExecutorError; use super::{expect_first_barrier, BoxedExecutor, Executor, ExecutorInfo, Message, PkIndicesRef}; use crate::common::table::iter_utils::merge_sort; -use crate::common::table::state_table::{collect_data_chunk, StateTable}; +use crate::common::table::state_table::StateTable; use crate::executor::monitor::StreamingMetrics; use crate::executor::{PkIndices, StreamExecutorResult, Watermark}; use crate::task::{ActorId, CreateMviewProgress}; @@ -447,9 +447,9 @@ where .iter_all_vnode_ranges(&range_bounds, Default::default()) .await?; let pinned_iter: Vec<_> = iterators.into_iter().map(Box::pin).collect_vec(); - let iter = merge_sort(pinned_iter); + let mut iter = merge_sort(pinned_iter); pin_mut!(iter); - for data_chunk in collect_data_chunk(iter, &schema, Some(CHUNK_SIZE)) + for data_chunk in collect_data_chunk(&mut iter, &schema, Some(CHUNK_SIZE)) .instrument_await("arrangement_backfill_snapshot_read") .await? { diff --git a/src/stream/src/executor/backfill.rs b/src/stream/src/executor/backfill.rs index 7c7602231aa0a..e17f89097b42a 100644 --- a/src/stream/src/executor/backfill.rs +++ b/src/stream/src/executor/backfill.rs @@ -35,7 +35,7 @@ use risingwave_common::util::sort_util::{cmp_datum, OrderType}; use risingwave_hummock_sdk::HummockReadEpoch; use risingwave_storage::store::PrefetchOptions; use risingwave_storage::table::batch_table::storage_table::StorageTable; -use risingwave_storage::table::TableIter; +use risingwave_storage::table::{collect_data_chunk, get_second, TableIter}; use risingwave_storage::StateStore; use super::error::StreamExecutorError; @@ -450,7 +450,7 @@ where }; // We use uncommitted read here, because we have already scheduled the `BackfillExecutor` // together with the upstream mv. - let iter = upstream_table + let mut iter = upstream_table .batch_iter_with_pk_bounds( HummockReadEpoch::NoWait(epoch), row::empty(), @@ -458,14 +458,15 @@ where ordered, PrefetchOptions::new_for_exhaust_iter(), ) - .await?; + .await? + .map(get_second); pin_mut!(iter); - while let Some(data_chunk) = iter - .collect_data_chunk(upstream_table.schema(), Some(CHUNK_SIZE)) - .instrument_await("backfill_snapshot_read") - .await? + while let Some(data_chunk) = + collect_data_chunk(&mut iter, upstream_table.schema(), Some(CHUNK_SIZE)) + .instrument_await("backfill_snapshot_read") + .await? { if data_chunk.cardinality() != 0 { let ops = vec![Op::Insert; data_chunk.capacity()]; diff --git a/src/stream/src/executor/batch_query.rs b/src/stream/src/executor/batch_query.rs index 09c539a5ae2bd..3cfd5265f959d 100644 --- a/src/stream/src/executor/batch_query.rs +++ b/src/stream/src/executor/batch_query.rs @@ -20,7 +20,7 @@ use risingwave_common::catalog::Schema; use risingwave_hummock_sdk::HummockReadEpoch; use risingwave_storage::store::PrefetchOptions; use risingwave_storage::table::batch_table::storage_table::StorageTable; -use risingwave_storage::table::TableIter; +use risingwave_storage::table::{collect_data_chunk, get_second, TableIter}; use risingwave_storage::StateStore; use super::error::StreamExecutorError; @@ -51,20 +51,21 @@ where #[try_stream(ok = Message, error = StreamExecutorError)] async fn execute_inner(self, epoch: u64) { - let iter = self + let mut iter = self .table .batch_iter( HummockReadEpoch::Committed(epoch), false, PrefetchOptions::new_for_exhaust_iter(), ) - .await?; + .await? + .map(get_second); pin_mut!(iter); - while let Some(data_chunk) = iter - .collect_data_chunk(self.schema(), Some(self.batch_size)) - .instrument_await("batch_query_executor_collect_chunk") - .await? + while let Some(data_chunk) = + collect_data_chunk(&mut iter, self.schema(), Some(self.batch_size)) + .instrument_await("batch_query_executor_collect_chunk") + .await? { let ops = vec![Op::Insert; data_chunk.capacity()]; let stream_chunk = StreamChunk::from_parts(ops, data_chunk); From b540eb75a8fb924fc21cbf54b95adcb9d9847a03 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Thu, 15 Jun 2023 16:03:14 +0800 Subject: [PATCH 17/90] make the diff better --- src/storage/src/table/mod.rs | 45 +++++------------------------------- 1 file changed, 6 insertions(+), 39 deletions(-) diff --git a/src/storage/src/table/mod.rs b/src/storage/src/table/mod.rs index ad15ebecf1db9..3d15f34f92b2f 100644 --- a/src/storage/src/table/mod.rs +++ b/src/storage/src/table/mod.rs @@ -78,6 +78,12 @@ impl Distribution { } } +// TODO: GAT-ify this trait or remove this trait +#[async_trait::async_trait] +pub trait TableIter: Send { + async fn next_row(&mut self) -> Result, E>; +} + pub fn get_second(arg: Result<(T, U), E>) -> Result { arg.map(|x| x.1) } @@ -121,45 +127,6 @@ where Ok(Some(chunk)) } } -// TODO: GAT-ify this trait or remove this trait -#[async_trait::async_trait] -pub trait TableIter: Send { - async fn next_row(&mut self) -> Result, E>; - // async fn collect_data_chunk( - // &mut self, - // schema: &Schema, - // chunk_size: Option, - // ) -> Result, E> { - // let mut builders = schema.create_array_builders(chunk_size.unwrap_or(0)); - // - // let mut row_count = 0; - // for _ in 0..chunk_size.unwrap_or(usize::MAX) { - // match self.next_row().await? { - // Some(row) => { - // for (datum, builder) in row.iter().zip_eq_fast(builders.iter_mut()) { - // builder.append(datum); - // } - // row_count += 1; - // } - // None => break, - // } - // } - // - // let chunk = { - // let columns: Vec<_> = builders - // .into_iter() - // .map(|builder| builder.finish().into()) - // .collect(); - // DataChunk::new(columns, row_count) - // }; - // - // if chunk.cardinality() == 0 { - // Ok(None) - // } else { - // Ok(Some(chunk)) - // } - // } -} /// Get vnode value with `indices` on the given `row`. pub fn compute_vnode(row: impl Row, indices: &[usize], vnodes: &Bitmap) -> VirtualNode { From 7fdcb2ff3fc5052dcf2524306a90acaeb8d007b5 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Thu, 15 Jun 2023 16:03:48 +0800 Subject: [PATCH 18/90] remove outdated comment --- src/storage/src/table/mod.rs | 1 - 1 file changed, 1 deletion(-) diff --git a/src/storage/src/table/mod.rs b/src/storage/src/table/mod.rs index 3d15f34f92b2f..3834f85eb0df9 100644 --- a/src/storage/src/table/mod.rs +++ b/src/storage/src/table/mod.rs @@ -88,7 +88,6 @@ pub fn get_second(arg: Result<(T, U), E>) -> Result { arg.map(|x| x.1) } -// FIXME: Try merge this with the impl in `src/table/mod.rs`. pub async fn collect_data_chunk( stream: &mut S, schema: &Schema, From faf7c5041e14c8a1e0cc1ab0979269fc534623d6 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Thu, 15 Jun 2023 16:06:19 +0800 Subject: [PATCH 19/90] fix outdated comment --- src/stream/src/common/table/iter_utils.rs | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/stream/src/common/table/iter_utils.rs b/src/stream/src/common/table/iter_utils.rs index c735705aace9a..f089b84b94f5e 100644 --- a/src/stream/src/common/table/iter_utils.rs +++ b/src/stream/src/common/table/iter_utils.rs @@ -113,8 +113,6 @@ where } while let Some(mut node) = heap.peek_mut() { // Note: If the `next` returns `Err`, we'll fail to yield the previous item. - // This is acceptable since we're not going to handle errors from cell-based table - // iteration, so where to fail does not matter. Or we need an `Option` for this. yield match node.stream.next().await.transpose()? { // There still remains data in the stream, take and update the peeked value. Some(new_peeked) => std::mem::replace(&mut node.peeked, new_peeked).1, From cbb2127b2b0a55fe2b4c40ef89ef06f3ae1d4c44 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Thu, 15 Jun 2023 16:13:16 +0800 Subject: [PATCH 20/90] risedev cf --- src/batch/src/executor/row_seq_scan.rs | 2 +- src/storage/src/table/mod.rs | 2 +- src/stream/src/common/table/iter_utils.rs | 70 +++++++++---------- src/stream/src/common/table/state_table.rs | 16 ++--- .../src/executor/arrangement_backfill.rs | 24 +++---- src/stream/src/executor/backfill.rs | 4 +- src/stream/src/executor/batch_query.rs | 4 +- 7 files changed, 58 insertions(+), 64 deletions(-) diff --git a/src/batch/src/executor/row_seq_scan.rs b/src/batch/src/executor/row_seq_scan.rs index 26c3a13e90556..b4aa5e4fa4786 100644 --- a/src/batch/src/executor/row_seq_scan.rs +++ b/src/batch/src/executor/row_seq_scan.rs @@ -34,7 +34,7 @@ use risingwave_pb::common::BatchQueryEpoch; use risingwave_pb::plan_common::StorageTableDesc; use risingwave_storage::store::PrefetchOptions; use risingwave_storage::table::batch_table::storage_table::StorageTable; -use risingwave_storage::table::{collect_data_chunk, get_second, Distribution, TableIter}; +use risingwave_storage::table::{collect_data_chunk, get_second, Distribution}; use risingwave_storage::{dispatch_state_store, StateStore}; use crate::executor::{ diff --git a/src/storage/src/table/mod.rs b/src/storage/src/table/mod.rs index 3834f85eb0df9..902b727e1850f 100644 --- a/src/storage/src/table/mod.rs +++ b/src/storage/src/table/mod.rs @@ -25,7 +25,7 @@ use risingwave_common::hash::VirtualNode; use risingwave_common::row::{OwnedRow, Row}; use risingwave_common::util::iter_util::ZipEqFast; -use crate::error::StorageResult; + /// For tables without distribution (singleton), the `DEFAULT_VNODE` is encoded. pub const DEFAULT_VNODE: VirtualNode = VirtualNode::ZERO; diff --git a/src/stream/src/common/table/iter_utils.rs b/src/stream/src/common/table/iter_utils.rs index f089b84b94f5e..dc61c6230036f 100644 --- a/src/stream/src/common/table/iter_utils.rs +++ b/src/stream/src/common/table/iter_utils.rs @@ -24,49 +24,43 @@ use std::collections::binary_heap::PeekMut; use std::collections::BinaryHeap; -use std::ops::Bound; -use std::ops::Bound::*; -use std::pin::Pin; -use std::sync::Arc; -use bytes::{BufMut, Bytes, BytesMut}; + + + + +use bytes::{Bytes}; use futures::{Stream, StreamExt}; -use futures_async_stream::{for_await, try_stream}; -use itertools::{izip, Itertools}; -use risingwave_common::array::stream_record::Record; -use risingwave_common::array::{DataChunk, Op, StreamChunk, Vis}; -use risingwave_common::buffer::Bitmap; -use risingwave_common::cache::CachePriority; -use risingwave_common::catalog::{ - get_dist_key_in_pk_indices, ColumnDesc, Schema, TableId, TableOption, -}; -use risingwave_common::hash::{VirtualNode, VnodeBitmapExt}; -use risingwave_common::row::{self, CompactedRow, OwnedRow, Row, RowExt}; -use risingwave_common::types::ScalarImpl; -use risingwave_common::util::epoch::EpochPair; -use risingwave_common::util::iter_util::{ZipEqDebug, ZipEqFast}; -use risingwave_common::util::row_serde::OrderedRowSerde; -use risingwave_common::util::sort_util::OrderType; -use risingwave_common::util::value_encoding::{BasicSerde, ValueRowSerde}; -use risingwave_hummock_sdk::key::{ - end_bound_of_prefix, next_key, prefixed_range, range_of_prefix, start_bound_of_excluded_prefix, -}; -use risingwave_pb::catalog::Table; -use risingwave_storage::error::{StorageError, StorageResult}; -use risingwave_storage::hummock::CachePolicy; -use risingwave_storage::mem_table::MemTableError; -use risingwave_storage::row_serde::row_serde_util::{ - deserialize_pk_with_vnode, serialize_pk, serialize_pk_with_vnode, -}; +use futures_async_stream::{try_stream}; + + + + + + + +use risingwave_common::row::{OwnedRow}; + + + + + + + + + + + + use risingwave_storage::store::{ - LocalStateStore, NewLocalOptions, PrefetchOptions, ReadOptions, StateStoreIterItemStream, + StateStoreIterItemStream, }; -use risingwave_storage::table::{compute_chunk_vnode, compute_vnode, Distribution, TableIter}; -use risingwave_storage::StateStore; -use tracing::trace; -use super::watermark::{WatermarkBufferByEpoch, WatermarkBufferStrategy}; -use crate::cache::cache_may_stale; + + + + + use crate::executor::{StreamExecutorError, StreamExecutorResult}; pub struct Node { diff --git a/src/stream/src/common/table/state_table.rs b/src/stream/src/common/table/state_table.rs index 32ad813984727..4dde313b07c05 100644 --- a/src/stream/src/common/table/state_table.rs +++ b/src/stream/src/common/table/state_table.rs @@ -12,23 +12,23 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::collections::binary_heap::PeekMut; -use std::collections::BinaryHeap; + + use std::ops::Bound; use std::ops::Bound::*; -use std::pin::Pin; + use std::sync::Arc; use bytes::{BufMut, Bytes, BytesMut}; use futures::{Stream, StreamExt}; -use futures_async_stream::{for_await, try_stream}; + use itertools::{izip, Itertools}; use risingwave_common::array::stream_record::Record; -use risingwave_common::array::{DataChunk, Op, StreamChunk, Vis}; +use risingwave_common::array::{Op, StreamChunk, Vis}; use risingwave_common::buffer::Bitmap; use risingwave_common::cache::CachePriority; use risingwave_common::catalog::{ - get_dist_key_in_pk_indices, ColumnDesc, Schema, TableId, TableOption, + get_dist_key_in_pk_indices, ColumnDesc, TableId, TableOption, }; use risingwave_common::hash::{VirtualNode, VnodeBitmapExt}; use risingwave_common::row::{self, CompactedRow, OwnedRow, Row, RowExt}; @@ -42,7 +42,7 @@ use risingwave_hummock_sdk::key::{ end_bound_of_prefix, next_key, prefixed_range, range_of_prefix, start_bound_of_excluded_prefix, }; use risingwave_pb::catalog::Table; -use risingwave_storage::error::{StorageError, StorageResult}; +use risingwave_storage::error::{StorageError}; use risingwave_storage::hummock::CachePolicy; use risingwave_storage::mem_table::MemTableError; use risingwave_storage::row_serde::row_serde_util::{ @@ -52,7 +52,7 @@ use risingwave_storage::store::{ LocalStateStore, NewLocalOptions, PrefetchOptions, ReadOptions, StateStoreIterItemStream, }; use risingwave_storage::table::{ - compute_chunk_vnode, compute_vnode, get_second, Distribution, TableIter, + compute_chunk_vnode, compute_vnode, get_second, Distribution, }; use risingwave_storage::StateStore; use tracing::trace; diff --git a/src/stream/src/executor/arrangement_backfill.rs b/src/stream/src/executor/arrangement_backfill.rs index 42b2c4328abbf..899aec74d2405 100644 --- a/src/stream/src/executor/arrangement_backfill.rs +++ b/src/stream/src/executor/arrangement_backfill.rs @@ -27,17 +27,17 @@ use risingwave_common::array::stream_record::Record; use risingwave_common::array::{Op, StreamChunk}; use risingwave_common::buffer::BitmapBuilder; use risingwave_common::catalog::Schema; -use risingwave_common::hash::{VirtualNode, VnodeBitmapExt}; -use risingwave_common::row::{self, OwnedRow, Row, RowExt}; +use risingwave_common::hash::{VnodeBitmapExt}; +use risingwave_common::row::{OwnedRow, Row, RowExt}; use risingwave_common::types::Datum; use risingwave_common::util::epoch::EpochPair; use risingwave_common::util::iter_util::ZipEqFast; use risingwave_common::util::sort_util::{cmp_datum, OrderType}; -use risingwave_hummock_sdk::HummockReadEpoch; -use risingwave_storage::error::StorageResult; -use risingwave_storage::store::PrefetchOptions; -use risingwave_storage::table::batch_table::storage_table::{StorageTable, StorageTableInnerIter}; -use risingwave_storage::table::{collect_data_chunk, TableIter}; + + + + +use risingwave_storage::table::{collect_data_chunk}; use risingwave_storage::StateStore; use super::error::StreamExecutorError; @@ -261,7 +261,7 @@ where Self::mark_chunk( chunk, current_pos, - &pk_in_output_indices, + pk_in_output_indices, pk_order, ), &self.output_indices, @@ -343,7 +343,7 @@ where // Raise the current position. // As snapshot read streams are ordered by pk, so we can // just use the last row to update `current_pos`. - current_pos = Self::update_pos(&chunk, &pk_in_output_indices); + current_pos = Self::update_pos(&chunk, pk_in_output_indices); let chunk_cardinality = chunk.cardinality() as u64; cur_barrier_snapshot_processed_rows += chunk_cardinality; @@ -422,9 +422,9 @@ where async fn snapshot_read( schema: Arc, upstream_table: &StateTable, - epoch: u64, + _epoch: u64, current_pos: Option, - ordered: bool, + _ordered: bool, ) { // `current_pos` is None means it needs to scan from the beginning, so we use Unbounded to // scan. Otherwise, use Excluded. @@ -447,7 +447,7 @@ where .iter_all_vnode_ranges(&range_bounds, Default::default()) .await?; let pinned_iter: Vec<_> = iterators.into_iter().map(Box::pin).collect_vec(); - let mut iter = merge_sort(pinned_iter); + let iter = merge_sort(pinned_iter); pin_mut!(iter); for data_chunk in collect_data_chunk(&mut iter, &schema, Some(CHUNK_SIZE)) .instrument_await("arrangement_backfill_snapshot_read") diff --git a/src/stream/src/executor/backfill.rs b/src/stream/src/executor/backfill.rs index e17f89097b42a..7381409a4f662 100644 --- a/src/stream/src/executor/backfill.rs +++ b/src/stream/src/executor/backfill.rs @@ -35,7 +35,7 @@ use risingwave_common::util::sort_util::{cmp_datum, OrderType}; use risingwave_hummock_sdk::HummockReadEpoch; use risingwave_storage::store::PrefetchOptions; use risingwave_storage::table::batch_table::storage_table::StorageTable; -use risingwave_storage::table::{collect_data_chunk, get_second, TableIter}; +use risingwave_storage::table::{collect_data_chunk, get_second}; use risingwave_storage::StateStore; use super::error::StreamExecutorError; @@ -450,7 +450,7 @@ where }; // We use uncommitted read here, because we have already scheduled the `BackfillExecutor` // together with the upstream mv. - let mut iter = upstream_table + let iter = upstream_table .batch_iter_with_pk_bounds( HummockReadEpoch::NoWait(epoch), row::empty(), diff --git a/src/stream/src/executor/batch_query.rs b/src/stream/src/executor/batch_query.rs index 3cfd5265f959d..5929702d2dd1d 100644 --- a/src/stream/src/executor/batch_query.rs +++ b/src/stream/src/executor/batch_query.rs @@ -20,7 +20,7 @@ use risingwave_common::catalog::Schema; use risingwave_hummock_sdk::HummockReadEpoch; use risingwave_storage::store::PrefetchOptions; use risingwave_storage::table::batch_table::storage_table::StorageTable; -use risingwave_storage::table::{collect_data_chunk, get_second, TableIter}; +use risingwave_storage::table::{collect_data_chunk, get_second}; use risingwave_storage::StateStore; use super::error::StreamExecutorError; @@ -51,7 +51,7 @@ where #[try_stream(ok = Message, error = StreamExecutorError)] async fn execute_inner(self, epoch: u64) { - let mut iter = self + let iter = self .table .batch_iter( HummockReadEpoch::Committed(epoch), From 120bb70cb417713a830d6ab7dfb7ff1455627806 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Thu, 15 Jun 2023 16:34:58 +0800 Subject: [PATCH 21/90] some more refactoring + run on ci to make sure nothing broke --- src/storage/src/table/mod.rs | 10 ++--- src/stream/src/common/table/iter_utils.rs | 37 ++----------------- src/stream/src/common/table/state_table.rs | 14 ++----- .../src/executor/arrangement_backfill.rs | 11 ++---- 4 files changed, 14 insertions(+), 58 deletions(-) diff --git a/src/storage/src/table/mod.rs b/src/storage/src/table/mod.rs index 902b727e1850f..19c431d0d861d 100644 --- a/src/storage/src/table/mod.rs +++ b/src/storage/src/table/mod.rs @@ -25,8 +25,6 @@ use risingwave_common::hash::VirtualNode; use risingwave_common::row::{OwnedRow, Row}; use risingwave_common::util::iter_util::ZipEqFast; - - /// For tables without distribution (singleton), the `DEFAULT_VNODE` is encoded. pub const DEFAULT_VNODE: VirtualNode = VirtualNode::ZERO; @@ -84,10 +82,6 @@ pub trait TableIter: Send { async fn next_row(&mut self) -> Result, E>; } -pub fn get_second(arg: Result<(T, U), E>) -> Result { - arg.map(|x| x.1) -} - pub async fn collect_data_chunk( stream: &mut S, schema: &Schema, @@ -127,6 +121,10 @@ where } } +pub fn get_second(arg: Result<(T, U), E>) -> Result { + arg.map(|x| x.1) +} + /// Get vnode value with `indices` on the given `row`. pub fn compute_vnode(row: impl Row, indices: &[usize], vnodes: &Bitmap) -> VirtualNode { let vnode = if indices.is_empty() { diff --git a/src/stream/src/common/table/iter_utils.rs b/src/stream/src/common/table/iter_utils.rs index dc61c6230036f..c2a7728c992a8 100644 --- a/src/stream/src/common/table/iter_utils.rs +++ b/src/stream/src/common/table/iter_utils.rs @@ -25,41 +25,10 @@ use std::collections::binary_heap::PeekMut; use std::collections::BinaryHeap; - - - - -use bytes::{Bytes}; +use bytes::Bytes; use futures::{Stream, StreamExt}; -use futures_async_stream::{try_stream}; - - - - - - - -use risingwave_common::row::{OwnedRow}; - - - - - - - - - - - - -use risingwave_storage::store::{ - StateStoreIterItemStream, -}; - - - - - +use futures_async_stream::try_stream; +use risingwave_common::row::OwnedRow; use crate::executor::{StreamExecutorError, StreamExecutorResult}; diff --git a/src/stream/src/common/table/state_table.rs b/src/stream/src/common/table/state_table.rs index 4dde313b07c05..63f82c40cff7c 100644 --- a/src/stream/src/common/table/state_table.rs +++ b/src/stream/src/common/table/state_table.rs @@ -12,24 +12,18 @@ // See the License for the specific language governing permissions and // limitations under the License. - - use std::ops::Bound; use std::ops::Bound::*; - use std::sync::Arc; use bytes::{BufMut, Bytes, BytesMut}; use futures::{Stream, StreamExt}; - use itertools::{izip, Itertools}; use risingwave_common::array::stream_record::Record; use risingwave_common::array::{Op, StreamChunk, Vis}; use risingwave_common::buffer::Bitmap; use risingwave_common::cache::CachePriority; -use risingwave_common::catalog::{ - get_dist_key_in_pk_indices, ColumnDesc, TableId, TableOption, -}; +use risingwave_common::catalog::{get_dist_key_in_pk_indices, ColumnDesc, TableId, TableOption}; use risingwave_common::hash::{VirtualNode, VnodeBitmapExt}; use risingwave_common::row::{self, CompactedRow, OwnedRow, Row, RowExt}; use risingwave_common::types::ScalarImpl; @@ -42,7 +36,7 @@ use risingwave_hummock_sdk::key::{ end_bound_of_prefix, next_key, prefixed_range, range_of_prefix, start_bound_of_excluded_prefix, }; use risingwave_pb::catalog::Table; -use risingwave_storage::error::{StorageError}; +use risingwave_storage::error::StorageError; use risingwave_storage::hummock::CachePolicy; use risingwave_storage::mem_table::MemTableError; use risingwave_storage::row_serde::row_serde_util::{ @@ -51,9 +45,7 @@ use risingwave_storage::row_serde::row_serde_util::{ use risingwave_storage::store::{ LocalStateStore, NewLocalOptions, PrefetchOptions, ReadOptions, StateStoreIterItemStream, }; -use risingwave_storage::table::{ - compute_chunk_vnode, compute_vnode, get_second, Distribution, -}; +use risingwave_storage::table::{compute_chunk_vnode, compute_vnode, get_second, Distribution}; use risingwave_storage::StateStore; use tracing::trace; diff --git a/src/stream/src/executor/arrangement_backfill.rs b/src/stream/src/executor/arrangement_backfill.rs index 899aec74d2405..554b26fe8de42 100644 --- a/src/stream/src/executor/arrangement_backfill.rs +++ b/src/stream/src/executor/arrangement_backfill.rs @@ -27,17 +27,13 @@ use risingwave_common::array::stream_record::Record; use risingwave_common::array::{Op, StreamChunk}; use risingwave_common::buffer::BitmapBuilder; use risingwave_common::catalog::Schema; -use risingwave_common::hash::{VnodeBitmapExt}; +use risingwave_common::hash::VnodeBitmapExt; use risingwave_common::row::{OwnedRow, Row, RowExt}; use risingwave_common::types::Datum; use risingwave_common::util::epoch::EpochPair; use risingwave_common::util::iter_util::ZipEqFast; use risingwave_common::util::sort_util::{cmp_datum, OrderType}; - - - - -use risingwave_storage::table::{collect_data_chunk}; +use risingwave_storage::table::collect_data_chunk; use risingwave_storage::StateStore; use super::error::StreamExecutorError; @@ -77,6 +73,7 @@ where S: StateStore, { #[allow(clippy::too_many_arguments)] + #[allow(dead_code)] pub fn new( upstream_table: StateTable, upstream: BoxedExecutor, @@ -449,7 +446,7 @@ where let pinned_iter: Vec<_> = iterators.into_iter().map(Box::pin).collect_vec(); let iter = merge_sort(pinned_iter); pin_mut!(iter); - for data_chunk in collect_data_chunk(&mut iter, &schema, Some(CHUNK_SIZE)) + while let Some(data_chunk) = collect_data_chunk(&mut iter, &schema, Some(CHUNK_SIZE)) .instrument_await("arrangement_backfill_snapshot_read") .await? { From 5b3e0347bd4b202d9b5ece7f2d442355cd310a89 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Fri, 16 Jun 2023 11:57:09 +0800 Subject: [PATCH 22/90] remove outdated code --- src/stream/src/common/table/state_table.rs | 39 ---------------------- 1 file changed, 39 deletions(-) diff --git a/src/stream/src/common/table/state_table.rs b/src/stream/src/common/table/state_table.rs index 63f82c40cff7c..2d61de02f87b9 100644 --- a/src/stream/src/common/table/state_table.rs +++ b/src/stream/src/common/table/state_table.rs @@ -1089,45 +1089,6 @@ where .map_err(Into::into) } } -// // FIXME: Try merge this with the impl in `src/table/mod.rs`. -// pub async fn collect_data_chunk( -// mut stream: S, -// schema: &Schema, -// chunk_size: Option, -// ) -> StreamExecutorResult> -// where -// S: Stream> + Unpin, -// { -// let mut builders = schema.create_array_builders(chunk_size.unwrap_or(0)); -// -// let mut row_count = 0; -// for _ in 0..chunk_size.unwrap_or(usize::MAX) { -// match stream.next().await.transpose()? { -// Some(row) => { -// for (datum, builder) in row.iter().zip_eq_fast(builders.iter_mut()) { -// builder.append(datum); -// } -// } -// None => break, -// } -// -// row_count += 1; -// } -// -// let chunk = { -// let columns: Vec<_> = builders -// .into_iter() -// .map(|builder| builder.finish().into()) -// .collect(); -// DataChunk::new(columns, row_count) -// }; -// -// if chunk.cardinality() == 0 { -// Ok(None) -// } else { -// Ok(Some(chunk)) -// } -// } pub type RowStream<'a, S: StateStore, SD: ValueRowSerde + 'a> = impl Stream> + 'a; From bd2cf4a1ad663344afbcc218004e7f09f117bcdf Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Fri, 16 Jun 2023 11:57:19 +0800 Subject: [PATCH 23/90] borrow chunk when flushing to downstream --- src/stream/src/executor/arrangement_backfill.rs | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/stream/src/executor/arrangement_backfill.rs b/src/stream/src/executor/arrangement_backfill.rs index 554b26fe8de42..fc31c32aee77f 100644 --- a/src/stream/src/executor/arrangement_backfill.rs +++ b/src/stream/src/executor/arrangement_backfill.rs @@ -256,7 +256,7 @@ where chunk.cardinality() as u64; yield Message::Chunk(Self::mapping_chunk( Self::mark_chunk( - chunk, + &chunk, current_pos, pk_in_output_indices, pk_order, @@ -464,11 +464,12 @@ where /// For each row of the chunk, forward it to downstream if its pk <= `current_pos`, otherwise /// ignore it. We implement it by changing the visibility bitmap. fn mark_chunk( - chunk: StreamChunk, + chunk: &StreamChunk, current_pos: &OwnedRow, pk_in_output_indices: PkIndicesRef<'_>, pk_order: &[OrderType], ) -> StreamChunk { + let chunk = chunk.clone(); // FIXME: Temporary workaround. let chunk = chunk.compact(); let (data, ops) = chunk.into_parts(); let mut new_visibility = BitmapBuilder::with_capacity(ops.len()); From 16ece5b4fc5df3e69cf131438ced0586ffe1daa9 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Fri, 16 Jun 2023 17:37:07 +0800 Subject: [PATCH 24/90] finish implementing buffered chunk logic --- .../src/executor/arrangement_backfill.rs | 32 +++++++++---------- 1 file changed, 16 insertions(+), 16 deletions(-) diff --git a/src/stream/src/executor/arrangement_backfill.rs b/src/stream/src/executor/arrangement_backfill.rs index fc31c32aee77f..856af7499b23a 100644 --- a/src/stream/src/executor/arrangement_backfill.rs +++ b/src/stream/src/executor/arrangement_backfill.rs @@ -104,12 +104,11 @@ where async fn execute_inner(mut self) { // The primary key columns, in the output columns of the upstream_table scan. // let pk_in_output_indices = self.upstream_table.pk_in_output_indices().unwrap(); - let pk_in_output_indices = self.upstream_table.pk_indices(); + let pk_in_output_indices = self.upstream_table.pk_indices().to_vec(); let state_len = pk_in_output_indices.len() + 2; // +1 for backfill_finished, +1 for vnode key. - - let pk_order = self.upstream_table.pk_serde().get_order_types(); - + let pk_order = self.upstream_table.pk_serde().get_order_types().to_vec(); let upstream_table_id = self.upstream_table.table_id(); + let mut upstream_table = self.upstream_table; let schema = Arc::new(self.upstream.schema().clone()); @@ -142,13 +141,8 @@ where // It is finished, so just assign a value to avoid accessing storage table again. false } else { - let snapshot = Self::snapshot_read( - schema.clone(), - &self.upstream_table, - init_epoch, - None, - false, - ); + let snapshot = + Self::snapshot_read(schema.clone(), &upstream_table, init_epoch, None, false); pin_mut!(snapshot); snapshot.try_next().await?.unwrap().is_none() } @@ -215,14 +209,20 @@ where // // Once the backfill loop ends, we forward the upstream directly to the downstream. if to_backfill { + let mut upstream_chunk_buffer: Vec = vec![]; 'backfill_loop: loop { - let mut upstream_chunk_buffer: Vec = vec![]; + // Each time we break out of the backfill loop, we need to flush + if let Some(_current_pos) = ¤t_pos { + for chunk in upstream_chunk_buffer.drain(..) { + upstream_table.write_chunk(chunk); + } + } let left_upstream = upstream.by_ref().map(Either::Left); let right_snapshot = pin!(Self::snapshot_read( schema.clone(), - &self.upstream_table, + &upstream_table, snapshot_read_epoch, current_pos.clone(), true @@ -258,8 +258,8 @@ where Self::mark_chunk( &chunk, current_pos, - pk_in_output_indices, - pk_order, + &pk_in_output_indices, + &pk_order, ), &self.output_indices, )); @@ -340,7 +340,7 @@ where // Raise the current position. // As snapshot read streams are ordered by pk, so we can // just use the last row to update `current_pos`. - current_pos = Self::update_pos(&chunk, pk_in_output_indices); + current_pos = Self::update_pos(&chunk, &pk_in_output_indices); let chunk_cardinality = chunk.cardinality() as u64; cur_barrier_snapshot_processed_rows += chunk_cardinality; From 1f525475fabb7a07f6169b3794574c157073eee1 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Sat, 17 Jun 2023 00:06:08 +0800 Subject: [PATCH 25/90] fix pk_in_output_indices --- src/stream/src/executor/arrangement_backfill.rs | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/src/stream/src/executor/arrangement_backfill.rs b/src/stream/src/executor/arrangement_backfill.rs index 856af7499b23a..74c08da23b56f 100644 --- a/src/stream/src/executor/arrangement_backfill.rs +++ b/src/stream/src/executor/arrangement_backfill.rs @@ -104,7 +104,13 @@ where async fn execute_inner(mut self) { // The primary key columns, in the output columns of the upstream_table scan. // let pk_in_output_indices = self.upstream_table.pk_in_output_indices().unwrap(); - let pk_in_output_indices = self.upstream_table.pk_indices().to_vec(); + let pk_in_output_indices = self + .upstream_table + .pk_indices() + .iter() + .map(|&i| self.output_indices.iter().position(|&j| i == j)) + .collect::>>() + .unwrap(); let state_len = pk_in_output_indices.len() + 2; // +1 for backfill_finished, +1 for vnode key. let pk_order = self.upstream_table.pk_serde().get_order_types().to_vec(); let upstream_table_id = self.upstream_table.table_id(); From 17cad18b161c189d6e5a85e35913906ffb0c240c Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Mon, 19 Jun 2023 17:53:37 +0800 Subject: [PATCH 26/90] minor --- src/storage/src/table/batch_table/storage_table.rs | 2 +- src/storage/src/table/mod.rs | 6 ++++-- src/stream/src/common/table/state_table.rs | 2 +- src/stream/src/executor/arrangement_backfill.rs | 2 +- 4 files changed, 7 insertions(+), 5 deletions(-) diff --git a/src/storage/src/table/batch_table/storage_table.rs b/src/storage/src/table/batch_table/storage_table.rs index 82ce2cf1874fe..8929c7f566bac 100644 --- a/src/storage/src/table/batch_table/storage_table.rs +++ b/src/storage/src/table/batch_table/storage_table.rs @@ -401,7 +401,7 @@ pub trait PkAndRowStream = Stream, OwnedRow)>> + S /// The wrapper of [`StorageTableInnerIter`] if pk is not persisted. pub type StorageTableInnerIter = impl PkAndRowStream; #[async_trait::async_trait] -impl TableIter for S { +impl TableIter for S { async fn next_row(&mut self) -> StorageResult> { self.next() .await diff --git a/src/storage/src/table/mod.rs b/src/storage/src/table/mod.rs index 19c431d0d861d..628612200730f 100644 --- a/src/storage/src/table/mod.rs +++ b/src/storage/src/table/mod.rs @@ -25,6 +25,8 @@ use risingwave_common::hash::VirtualNode; use risingwave_common::row::{OwnedRow, Row}; use risingwave_common::util::iter_util::ZipEqFast; +use crate::error::StorageResult; + /// For tables without distribution (singleton), the `DEFAULT_VNODE` is encoded. pub const DEFAULT_VNODE: VirtualNode = VirtualNode::ZERO; @@ -78,8 +80,8 @@ impl Distribution { // TODO: GAT-ify this trait or remove this trait #[async_trait::async_trait] -pub trait TableIter: Send { - async fn next_row(&mut self) -> Result, E>; +pub trait TableIter: Send { + async fn next_row(&mut self) -> StorageResult>; } pub async fn collect_data_chunk( diff --git a/src/stream/src/common/table/state_table.rs b/src/stream/src/common/table/state_table.rs index 2d61de02f87b9..c9123f4b1bc5d 100644 --- a/src/stream/src/common/table/state_table.rs +++ b/src/stream/src/common/table/state_table.rs @@ -932,7 +932,7 @@ where .map(get_second)) } - pub async fn iter_all_vnode_ranges( + pub async fn iter_all_vnodes_with_pk_range( &self, pk_range: &(Bound, Bound), prefetch_options: PrefetchOptions, diff --git a/src/stream/src/executor/arrangement_backfill.rs b/src/stream/src/executor/arrangement_backfill.rs index 74c08da23b56f..5a8a3deb9cc52 100644 --- a/src/stream/src/executor/arrangement_backfill.rs +++ b/src/stream/src/executor/arrangement_backfill.rs @@ -447,7 +447,7 @@ where (Bound::Unbounded, Bound::Unbounded) }; let iterators = upstream_table - .iter_all_vnode_ranges(&range_bounds, Default::default()) + .iter_all_vnodes_with_pk_range(&range_bounds, Default::default()) .await?; let pinned_iter: Vec<_> = iterators.into_iter().map(Box::pin).collect_vec(); let iter = merge_sort(pinned_iter); From 650837881aba8e5c25ac81f5dbc17a371b90661d Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Mon, 19 Jun 2023 17:56:12 +0800 Subject: [PATCH 27/90] doc --- src/storage/src/table/mod.rs | 1 + 1 file changed, 1 insertion(+) diff --git a/src/storage/src/table/mod.rs b/src/storage/src/table/mod.rs index 628612200730f..135122f390e53 100644 --- a/src/storage/src/table/mod.rs +++ b/src/storage/src/table/mod.rs @@ -84,6 +84,7 @@ pub trait TableIter: Send { async fn next_row(&mut self) -> StorageResult>; } +/// Collects data chunks from stream of rows. pub async fn collect_data_chunk( stream: &mut S, schema: &Schema, From 3ff06bcb5a915d86a6777de571068f5573a6375b Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Mon, 19 Jun 2023 17:59:47 +0800 Subject: [PATCH 28/90] do not drain upstream_chunk_buffer, it needs to be flushed to state_table --- src/stream/src/executor/arrangement_backfill.rs | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/src/stream/src/executor/arrangement_backfill.rs b/src/stream/src/executor/arrangement_backfill.rs index 5a8a3deb9cc52..99af3d15d6a01 100644 --- a/src/stream/src/executor/arrangement_backfill.rs +++ b/src/stream/src/executor/arrangement_backfill.rs @@ -103,7 +103,6 @@ where #[try_stream(ok = Message, error = StreamExecutorError)] async fn execute_inner(mut self) { // The primary key columns, in the output columns of the upstream_table scan. - // let pk_in_output_indices = self.upstream_table.pk_in_output_indices().unwrap(); let pk_in_output_indices = self .upstream_table .pk_indices() @@ -257,12 +256,12 @@ where // Consume upstream buffer chunk if let Some(current_pos) = ¤t_pos { - for chunk in upstream_chunk_buffer.drain(..) { + for chunk in &upstream_chunk_buffer { cur_barrier_upstream_processed_rows += chunk.cardinality() as u64; yield Message::Chunk(Self::mapping_chunk( Self::mark_chunk( - &chunk, + chunk, current_pos, &pk_in_output_indices, &pk_order, From 3cd524d1f6dd065e9476e9c34f155dcd24868791 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Mon, 19 Jun 2023 18:02:37 +0800 Subject: [PATCH 29/90] add more docs --- src/stream/src/executor/arrangement_backfill.rs | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/src/stream/src/executor/arrangement_backfill.rs b/src/stream/src/executor/arrangement_backfill.rs index 99af3d15d6a01..9a5f5787a4aa0 100644 --- a/src/stream/src/executor/arrangement_backfill.rs +++ b/src/stream/src/executor/arrangement_backfill.rs @@ -217,6 +217,8 @@ where let mut upstream_chunk_buffer: Vec = vec![]; 'backfill_loop: loop { // Each time we break out of the backfill loop, we need to flush + // We can't do it while processing the stream barrier, since the immutable reference to + // upstream_table is live. So we have to do it here. if let Some(_current_pos) = ¤t_pos { for chunk in upstream_chunk_buffer.drain(..) { upstream_table.write_chunk(chunk); @@ -254,8 +256,10 @@ where // If it is a barrier, switch snapshot and consume // upstream buffer chunk - // Consume upstream buffer chunk if let Some(current_pos) = ¤t_pos { + // NOTE: We do not consume the chunk here. + // We still need to flush it to the `state_table`, + // to ensure the upstream state is replicated. for chunk in &upstream_chunk_buffer { cur_barrier_upstream_processed_rows += chunk.cardinality() as u64; From e49f1f32a29b675d1aed95c69345b3d00044f278 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Mon, 19 Jun 2023 18:38:30 +0800 Subject: [PATCH 30/90] remove unused args --- .../src/executor/arrangement_backfill.rs | 25 +++++++------------ 1 file changed, 9 insertions(+), 16 deletions(-) diff --git a/src/stream/src/executor/arrangement_backfill.rs b/src/stream/src/executor/arrangement_backfill.rs index 9a5f5787a4aa0..2cff667422190 100644 --- a/src/stream/src/executor/arrangement_backfill.rs +++ b/src/stream/src/executor/arrangement_backfill.rs @@ -121,7 +121,6 @@ where // Poll the upstream to get the first barrier. let first_barrier = expect_first_barrier(&mut upstream).await?; - let init_epoch = first_barrier.epoch.prev; if let Some(state_table) = self.state_table.as_mut() { state_table.init_epoch(first_barrier.epoch); } @@ -146,8 +145,7 @@ where // It is finished, so just assign a value to avoid accessing storage table again. false } else { - let snapshot = - Self::snapshot_read(schema.clone(), &upstream_table, init_epoch, None, false); + let snapshot = Self::snapshot_read(schema.clone(), &upstream_table, None); pin_mut!(snapshot); snapshot.try_next().await?.unwrap().is_none() } @@ -185,7 +183,7 @@ where // expects to have been initialized in previous epoch. // The epoch used to snapshot read upstream mv. - let mut snapshot_read_epoch = init_epoch; + let mut snapshot_read_epoch; // Keep track of rows from the snapshot. let mut total_snapshot_processed_rows: u64 = 0; @@ -217,8 +215,8 @@ where let mut upstream_chunk_buffer: Vec = vec![]; 'backfill_loop: loop { // Each time we break out of the backfill loop, we need to flush - // We can't do it while processing the stream barrier, since the immutable reference to - // upstream_table is live. So we have to do it here. + // We can't do it while processing the stream barrier, since the immutable reference + // to upstream_table is live. So we have to do it here. if let Some(_current_pos) = ¤t_pos { for chunk in upstream_chunk_buffer.drain(..) { upstream_table.write_chunk(chunk); @@ -227,14 +225,11 @@ where let left_upstream = upstream.by_ref().map(Either::Left); - let right_snapshot = pin!(Self::snapshot_read( - schema.clone(), - &upstream_table, - snapshot_read_epoch, - current_pos.clone(), - true - ) - .map(Either::Right),); + let right_snapshot = + pin!( + Self::snapshot_read(schema.clone(), &upstream_table, current_pos.clone(),) + .map(Either::Right), + ); // Prefer to select upstream, so we can stop snapshot stream as soon as the barrier // comes. @@ -428,9 +423,7 @@ where async fn snapshot_read( schema: Arc, upstream_table: &StateTable, - _epoch: u64, current_pos: Option, - _ordered: bool, ) { // `current_pos` is None means it needs to scan from the beginning, so we use Unbounded to // scan. Otherwise, use Excluded. From 1a4a25c96f018bc22d3a550c12fe91dbf67bbce6 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Mon, 19 Jun 2023 19:06:54 +0800 Subject: [PATCH 31/90] fix logic --- .../src/executor/arrangement_backfill.rs | 138 +++++++++--------- 1 file changed, 72 insertions(+), 66 deletions(-) diff --git a/src/stream/src/executor/arrangement_backfill.rs b/src/stream/src/executor/arrangement_backfill.rs index 2cff667422190..dbad1a6b3cfdb 100644 --- a/src/stream/src/executor/arrangement_backfill.rs +++ b/src/stream/src/executor/arrangement_backfill.rs @@ -41,7 +41,7 @@ use super::{expect_first_barrier, BoxedExecutor, Executor, ExecutorInfo, Message use crate::common::table::iter_utils::merge_sort; use crate::common::table::state_table::StateTable; use crate::executor::monitor::StreamingMetrics; -use crate::executor::{PkIndices, StreamExecutorResult, Watermark}; +use crate::executor::{Barrier, PkIndices, StreamExecutorResult, Watermark}; use crate::task::{ActorId, CreateMviewProgress}; pub struct ArrangementBackfillExecutor { @@ -213,14 +213,76 @@ where // Once the backfill loop ends, we forward the upstream directly to the downstream. if to_backfill { let mut upstream_chunk_buffer: Vec = vec![]; + let mut pending_barrier: Option = None; 'backfill_loop: loop { - // Each time we break out of the backfill loop, we need to flush - // We can't do it while processing the stream barrier, since the immutable reference - // to upstream_table is live. So we have to do it here. - if let Some(_current_pos) = ¤t_pos { + let mut cur_barrier_snapshot_processed_rows: u64 = 0; + let mut cur_barrier_upstream_processed_rows: u64 = 0; + + // Process barrier + // Each time we break out of the backfill_stream loop, + // it is typically due to barrier (except the first time). + // So we should immediately process barrier if there's one pending. + if let Some(barrier) = pending_barrier.clone() { + // If it is a barrier, switch snapshot and consume + // upstream buffer chunk + + if let Some(current_pos) = ¤t_pos { + // NOTE: We do not consume the chunk here. + // We still need to flush it to the `state_table`, + // to ensure the upstream state is replicated. + for chunk in &upstream_chunk_buffer { + cur_barrier_upstream_processed_rows += chunk.cardinality() as u64; + yield Message::Chunk(Self::mapping_chunk( + Self::mark_chunk( + chunk, + current_pos, + &pk_in_output_indices, + &pk_order, + ), + &self.output_indices, + )); + } + } for chunk in upstream_chunk_buffer.drain(..) { upstream_table.write_chunk(chunk); } + self.metrics + .backfill_snapshot_read_row_count + .with_label_values(&[ + upstream_table_id.to_string().as_str(), + self.actor_id.to_string().as_str(), + ]) + .inc_by(cur_barrier_snapshot_processed_rows); + + self.metrics + .backfill_upstream_output_row_count + .with_label_values(&[ + upstream_table_id.to_string().as_str(), + self.actor_id.to_string().as_str(), + ]) + .inc_by(cur_barrier_upstream_processed_rows); + + // Update snapshot read epoch. + snapshot_read_epoch = barrier.epoch.prev; + + self.progress.update( + barrier.epoch.curr, + snapshot_read_epoch, + total_snapshot_processed_rows, + ); + + // Persist state on barrier + Self::persist_state( + barrier.epoch, + &mut self.state_table, + false, + ¤t_pos, + &mut old_state, + &mut current_state, + ) + .await?; + + yield Message::Barrier(barrier); } let left_upstream = upstream.by_ref().map(Either::Left); @@ -238,9 +300,6 @@ where stream::PollNext::Left }); - let mut cur_barrier_snapshot_processed_rows: u64 = 0; - let mut cur_barrier_upstream_processed_rows: u64 = 0; - #[for_await] for either in backfill_stream { match either { @@ -248,65 +307,12 @@ where Either::Left(msg) => { match msg? { Message::Barrier(barrier) => { - // If it is a barrier, switch snapshot and consume - // upstream buffer chunk - - if let Some(current_pos) = ¤t_pos { - // NOTE: We do not consume the chunk here. - // We still need to flush it to the `state_table`, - // to ensure the upstream state is replicated. - for chunk in &upstream_chunk_buffer { - cur_barrier_upstream_processed_rows += - chunk.cardinality() as u64; - yield Message::Chunk(Self::mapping_chunk( - Self::mark_chunk( - chunk, - current_pos, - &pk_in_output_indices, - &pk_order, - ), - &self.output_indices, - )); - } - } + // We have to process the barrier outside of the loop. + // This is because our state_table reference is still live + // here, we have to break the loop to drop it, + // so we can do replication of upstream state_table. + pending_barrier = Some(barrier); - self.metrics - .backfill_snapshot_read_row_count - .with_label_values(&[ - upstream_table_id.to_string().as_str(), - self.actor_id.to_string().as_str(), - ]) - .inc_by(cur_barrier_snapshot_processed_rows); - - self.metrics - .backfill_upstream_output_row_count - .with_label_values(&[ - upstream_table_id.to_string().as_str(), - self.actor_id.to_string().as_str(), - ]) - .inc_by(cur_barrier_upstream_processed_rows); - - // Update snapshot read epoch. - snapshot_read_epoch = barrier.epoch.prev; - - self.progress.update( - barrier.epoch.curr, - snapshot_read_epoch, - total_snapshot_processed_rows, - ); - - // Persist state on barrier - Self::persist_state( - barrier.epoch, - &mut self.state_table, - false, - ¤t_pos, - &mut old_state, - &mut current_state, - ) - .await?; - - yield Message::Barrier(barrier); // Break the for loop and start a new snapshot read stream. break; } From dd5f7c2321135e79c3ab0d005c89be00262d470e Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Mon, 19 Jun 2023 20:47:33 +0800 Subject: [PATCH 32/90] add notes on further optimization --- src/stream/src/executor/arrangement_backfill.rs | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/stream/src/executor/arrangement_backfill.rs b/src/stream/src/executor/arrangement_backfill.rs index dbad1a6b3cfdb..f3277694ea5f5 100644 --- a/src/stream/src/executor/arrangement_backfill.rs +++ b/src/stream/src/executor/arrangement_backfill.rs @@ -477,7 +477,9 @@ where pk_in_output_indices: PkIndicesRef<'_>, pk_order: &[OrderType], ) -> StreamChunk { - let chunk = chunk.clone(); // FIXME: Temporary workaround. + // FIXME: Temporary workaround, we can optimize it later when benchmarking. + // We can do so with a `chunk.compact_ref()` instead of `chunk.compact()`. + let chunk = chunk.clone(); let chunk = chunk.compact(); let (data, ops) = chunk.into_parts(); let mut new_visibility = BitmapBuilder::with_capacity(ops.len()); From 87a01de7523c5610179dd9c7bdd355269e41739d Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Mon, 19 Jun 2023 21:00:11 +0800 Subject: [PATCH 33/90] commit to state table too --- src/stream/src/executor/arrangement_backfill.rs | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/src/stream/src/executor/arrangement_backfill.rs b/src/stream/src/executor/arrangement_backfill.rs index f3277694ea5f5..fb174c5b58e42 100644 --- a/src/stream/src/executor/arrangement_backfill.rs +++ b/src/stream/src/executor/arrangement_backfill.rs @@ -243,9 +243,13 @@ where )); } } - for chunk in upstream_chunk_buffer.drain(..) { - upstream_table.write_chunk(chunk); + if !upstream_chunk_buffer.is_empty() { + for chunk in upstream_chunk_buffer.drain(..) { + upstream_table.write_chunk(chunk); + } + upstream_table.commit(barrier.epoch); } + self.metrics .backfill_snapshot_read_row_count .with_label_values(&[ From 633e18445e2982a5dee8f153d0708fda08e5c361 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Mon, 19 Jun 2023 21:06:44 +0800 Subject: [PATCH 34/90] fix --- src/stream/src/executor/arrangement_backfill.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/stream/src/executor/arrangement_backfill.rs b/src/stream/src/executor/arrangement_backfill.rs index fb174c5b58e42..175b747dd7231 100644 --- a/src/stream/src/executor/arrangement_backfill.rs +++ b/src/stream/src/executor/arrangement_backfill.rs @@ -247,7 +247,7 @@ where for chunk in upstream_chunk_buffer.drain(..) { upstream_table.write_chunk(chunk); } - upstream_table.commit(barrier.epoch); + upstream_table.commit(barrier.epoch).await?; } self.metrics From a82bcbd3e3a5b702eec5f1ace4cf889422ab07ca Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Tue, 20 Jun 2023 15:36:47 +0800 Subject: [PATCH 35/90] remove unrelated changes --- src/storage/src/table/batch_table/storage_table.rs | 1 + 1 file changed, 1 insertion(+) diff --git a/src/storage/src/table/batch_table/storage_table.rs b/src/storage/src/table/batch_table/storage_table.rs index 8929c7f566bac..1e1d2d7af60b4 100644 --- a/src/storage/src/table/batch_table/storage_table.rs +++ b/src/storage/src/table/batch_table/storage_table.rs @@ -400,6 +400,7 @@ pub trait PkAndRowStream = Stream, OwnedRow)>> + S /// The row iterator of the storage table. /// The wrapper of [`StorageTableInnerIter`] if pk is not persisted. pub type StorageTableInnerIter = impl PkAndRowStream; + #[async_trait::async_trait] impl TableIter for S { async fn next_row(&mut self) -> StorageResult> { From 8eec53aae94d17759b7412e36fbda1c2a3f02dde Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Tue, 20 Jun 2023 15:50:17 +0800 Subject: [PATCH 36/90] remove unnecessary backwards compat code for arrangement backfill --- .../src/executor/arrangement_backfill.rs | 34 ++++++------------- 1 file changed, 11 insertions(+), 23 deletions(-) diff --git a/src/stream/src/executor/arrangement_backfill.rs b/src/stream/src/executor/arrangement_backfill.rs index 175b747dd7231..925c0b3ff6879 100644 --- a/src/stream/src/executor/arrangement_backfill.rs +++ b/src/stream/src/executor/arrangement_backfill.rs @@ -52,7 +52,7 @@ pub struct ArrangementBackfillExecutor { upstream: BoxedExecutor, /// Internal state table for persisting state of backfill state. - state_table: Option>, + state_table: StateTable, /// The column indices need to be forwarded to the downstream from the upstream and table scan. output_indices: Vec, @@ -77,7 +77,7 @@ where pub fn new( upstream_table: StateTable, upstream: BoxedExecutor, - state_table: Option>, + state_table: StateTable, output_indices: Vec, progress: CreateMviewProgress, schema: Schema, @@ -121,20 +121,12 @@ where // Poll the upstream to get the first barrier. let first_barrier = expect_first_barrier(&mut upstream).await?; - if let Some(state_table) = self.state_table.as_mut() { - state_table.init_epoch(first_barrier.epoch); - } + self.state_table.init_epoch(first_barrier.epoch); - let is_finished = if let Some(state_table) = self.state_table.as_mut() { - let is_finished = Self::check_all_vnode_finished(state_table, state_len).await?; - if is_finished { - assert!(!first_barrier.is_newly_added(self.actor_id)); - } - is_finished - } else { - // Maintain backwards compatibility with no state table - !first_barrier.is_newly_added(self.actor_id) - }; + let is_finished = Self::check_all_vnode_finished(&self.state_table, state_len).await?; + if is_finished { + assert!(!first_barrier.is_newly_added(self.actor_id)); + } // If the snapshot is empty, we don't need to backfill. // We cannot complete progress now, as we want to persist @@ -421,9 +413,9 @@ where #[for_await] for msg in upstream { if let Some(msg) = Self::mapping_message(msg?, &self.output_indices) { - if let Some(state_table) = self.state_table.as_mut() && let Message::Barrier(barrier) = &msg { - state_table.commit_no_data_expected(barrier.epoch); - } + if let Message::Barrier(barrier) = &msg { + self.state_table.commit_no_data_expected(barrier.epoch); + } yield msg; } } @@ -535,16 +527,12 @@ where /// They should be strictly increasing. async fn persist_state( epoch: EpochPair, - table: &mut Option>, + table: &mut StateTable, is_finished: bool, current_pos: &Option, old_state: &mut Option>, current_state: &mut [Datum], ) -> StreamExecutorResult<()> { - // Backwards compatibility with no state table in backfill. - let Some(table) = table else { - return Ok(()) - }; if let Some(current_pos_inner) = current_pos { // state w/o vnodes. Self::build_temporary_state(current_state, is_finished, current_pos_inner); From 372ed2a39ba1a0ce82ee56b9872f31df37c3f67f Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Tue, 20 Jun 2023 16:29:26 +0800 Subject: [PATCH 37/90] fix double for-loop --- .../src/executor/arrangement_backfill.rs | 27 +++++++++---------- 1 file changed, 13 insertions(+), 14 deletions(-) diff --git a/src/stream/src/executor/arrangement_backfill.rs b/src/stream/src/executor/arrangement_backfill.rs index 925c0b3ff6879..da35d6aa788c8 100644 --- a/src/stream/src/executor/arrangement_backfill.rs +++ b/src/stream/src/executor/arrangement_backfill.rs @@ -215,18 +215,16 @@ where // it is typically due to barrier (except the first time). // So we should immediately process barrier if there's one pending. if let Some(barrier) = pending_barrier.clone() { - // If it is a barrier, switch snapshot and consume - // upstream buffer chunk - - if let Some(current_pos) = ¤t_pos { - // NOTE: We do not consume the chunk here. - // We still need to flush it to the `state_table`, - // to ensure the upstream state is replicated. - for chunk in &upstream_chunk_buffer { - cur_barrier_upstream_processed_rows += chunk.cardinality() as u64; + let upstream_chunk_buffer_is_empty = upstream_chunk_buffer.is_empty(); + for chunk in upstream_chunk_buffer.drain(..) { + cur_barrier_upstream_processed_rows += chunk.cardinality() as u64; + // Flush downstream. + // If no current_pos, means no snapshot processed yet. + // Also means we don't need propagate any updates <= current_pos. + if let Some(current_pos) = ¤t_pos { yield Message::Chunk(Self::mapping_chunk( Self::mark_chunk( - chunk, + &chunk, current_pos, &pk_in_output_indices, &pk_order, @@ -234,11 +232,12 @@ where &self.output_indices, )); } + // Replicate + upstream_table.write_chunk(chunk); } - if !upstream_chunk_buffer.is_empty() { - for chunk in upstream_chunk_buffer.drain(..) { - upstream_table.write_chunk(chunk); - } + if upstream_chunk_buffer_is_empty { + upstream_table.commit_no_data_expected(barrier.epoch) + } else { upstream_table.commit(barrier.epoch).await?; } From 02a71997667633a99aa1810bd05975eda82e1317 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Tue, 20 Jun 2023 19:36:29 +0800 Subject: [PATCH 38/90] add merge_sort to iterator --- src/stream/src/common/table/iter_utils.rs | 6 +++--- src/stream/src/common/table/state_table.rs | 20 ++++++++++++++++--- .../src/executor/arrangement_backfill.rs | 8 ++------ 3 files changed, 22 insertions(+), 12 deletions(-) diff --git a/src/stream/src/common/table/iter_utils.rs b/src/stream/src/common/table/iter_utils.rs index c2a7728c992a8..142674c3c32ee 100644 --- a/src/stream/src/common/table/iter_utils.rs +++ b/src/stream/src/common/table/iter_utils.rs @@ -63,7 +63,7 @@ impl Ord for Node { } } -#[try_stream(ok=OwnedRow, error=StreamExecutorError)] +#[try_stream(ok=(Bytes, OwnedRow), error=StreamExecutorError)] pub async fn merge_sort<'a, R>(streams: Vec) where R: Stream> + 'a + Unpin, @@ -78,9 +78,9 @@ where // Note: If the `next` returns `Err`, we'll fail to yield the previous item. yield match node.stream.next().await.transpose()? { // There still remains data in the stream, take and update the peeked value. - Some(new_peeked) => std::mem::replace(&mut node.peeked, new_peeked).1, + Some(new_peeked) => std::mem::replace(&mut node.peeked, new_peeked), // This stream is exhausted, remove it from the heap. - None => PeekMut::pop(node).peeked.1, + None => PeekMut::pop(node).peeked, }; } } diff --git a/src/stream/src/common/table/state_table.rs b/src/stream/src/common/table/state_table.rs index cf1b50223b6aa..62cc789aabb2f 100644 --- a/src/stream/src/common/table/state_table.rs +++ b/src/stream/src/common/table/state_table.rs @@ -51,6 +51,7 @@ use tracing::{trace, Instrument}; use super::watermark::{WatermarkBufferByEpoch, WatermarkBufferStrategy}; use crate::cache::cache_may_stale; +use crate::common::table::iter_utils::merge_sort; use crate::executor::{StreamExecutorError, StreamExecutorResult}; /// This num is arbitrary and we may want to improve this choice in the future. @@ -934,11 +935,11 @@ where .map(get_second)) } - pub async fn iter_all_vnodes_with_pk_range( + async fn iter_all_pk_and_val_with_pk_range( &self, pk_range: &(Bound, Bound), prefetch_options: PrefetchOptions, - ) -> StreamExecutorResult>> { + ) -> StreamExecutorResult> { let mut vec = Vec::with_capacity(self.vnodes.count_ones()); for vnode in self.vnodes.iter_vnodes() { vec.push( @@ -946,7 +947,20 @@ where .await?, ) } - Ok(vec) + let pinned_iter: Vec<_> = vec.into_iter().map(Box::pin).collect_vec(); + let iter = merge_sort(pinned_iter); + Ok(iter) + } + + pub async fn iter_all_with_pk_range( + &self, + pk_range: &(Bound, Bound), + prefetch_options: PrefetchOptions, + ) -> StreamExecutorResult> { + Ok(self + .iter_all_pk_and_val_with_pk_range(pk_range, prefetch_options) + .await? + .map(get_second)) } pub async fn iter_key_and_val_with_pk_range( diff --git a/src/stream/src/executor/arrangement_backfill.rs b/src/stream/src/executor/arrangement_backfill.rs index da35d6aa788c8..da82c347ed88b 100644 --- a/src/stream/src/executor/arrangement_backfill.rs +++ b/src/stream/src/executor/arrangement_backfill.rs @@ -22,7 +22,6 @@ use either::Either; use futures::stream::select_with_strategy; use futures::{pin_mut, stream, StreamExt, TryStreamExt}; use futures_async_stream::try_stream; -use itertools::Itertools; use risingwave_common::array::stream_record::Record; use risingwave_common::array::{Op, StreamChunk}; use risingwave_common::buffer::BitmapBuilder; @@ -38,7 +37,6 @@ use risingwave_storage::StateStore; use super::error::StreamExecutorError; use super::{expect_first_barrier, BoxedExecutor, Executor, ExecutorInfo, Message, PkIndicesRef}; -use crate::common::table::iter_utils::merge_sort; use crate::common::table::state_table::StateTable; use crate::executor::monitor::StreamingMetrics; use crate::executor::{Barrier, PkIndices, StreamExecutorResult, Watermark}; @@ -443,11 +441,9 @@ where } else { (Bound::Unbounded, Bound::Unbounded) }; - let iterators = upstream_table - .iter_all_vnodes_with_pk_range(&range_bounds, Default::default()) + let iter = upstream_table + .iter_all_with_pk_range(&range_bounds, Default::default()) .await?; - let pinned_iter: Vec<_> = iterators.into_iter().map(Box::pin).collect_vec(); - let iter = merge_sort(pinned_iter); pin_mut!(iter); while let Some(data_chunk) = collect_data_chunk(&mut iter, &schema, Some(CHUNK_SIZE)) .instrument_await("arrangement_backfill_snapshot_read") From 1392d4b2d6e05a37865d399a96ca69ed1d602627 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Tue, 20 Jun 2023 20:05:51 +0800 Subject: [PATCH 39/90] merge merge_sort implementations --- .../src/table/batch_table/iter_utils.rs | 129 ------------------ src/storage/src/table/batch_table/mod.rs | 1 - .../src/table/batch_table/storage_table.rs | 5 +- src/storage/src/table/mod.rs | 108 +++++++++++++++ src/stream/src/common/table/iter_utils.rs | 86 ------------ src/stream/src/common/table/mod.rs | 1 - src/stream/src/common/table/state_table.rs | 5 +- 7 files changed, 113 insertions(+), 222 deletions(-) delete mode 100644 src/storage/src/table/batch_table/iter_utils.rs delete mode 100644 src/stream/src/common/table/iter_utils.rs diff --git a/src/storage/src/table/batch_table/iter_utils.rs b/src/storage/src/table/batch_table/iter_utils.rs deleted file mode 100644 index 1378eb2d1bbd6..0000000000000 --- a/src/storage/src/table/batch_table/iter_utils.rs +++ /dev/null @@ -1,129 +0,0 @@ -// Copyright 2023 RisingWave 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::collections::binary_heap::PeekMut; -use std::collections::BinaryHeap; - -use futures::StreamExt; -use futures_async_stream::try_stream; -use risingwave_common::row::OwnedRow; - -use super::storage_table::PkAndRowStream; -use crate::error::StorageError; - -/// We use a binary heap to merge the results of the different streams in order. -/// This is the node type of the heap. -struct Node { - stream: S, - - /// The next item polled from `stream` previously. Since the `eq` and `cmp` must be synchronous - /// functions, we need to implement peeking manually. - peeked: (Vec, OwnedRow), -} - -impl PartialEq for Node { - fn eq(&self, other: &Self) -> bool { - match self.peeked.0 == other.peeked.0 { - true => unreachable!("primary key from different iters should be unique"), - false => false, - } - } -} -impl Eq for Node {} - -impl PartialOrd for Node { - fn partial_cmp(&self, other: &Self) -> Option { - Some(self.cmp(other)) - } -} -impl Ord for Node { - fn cmp(&self, other: &Self) -> std::cmp::Ordering { - // The heap is a max heap, so we need to reverse the order. - self.peeked.0.cmp(&other.peeked.0).reverse() - } -} - -/// Merge multiple streams of primary key and rows into a single stream, sorted by primary key. -/// We should ensure that the primary key from different streams are unique. -#[try_stream(ok = (Vec, OwnedRow), error = StorageError)] -pub(super) async fn merge_sort(streams: Vec) -where - S: PkAndRowStream + Unpin, -{ - let mut heap = BinaryHeap::with_capacity(streams.len()); - for mut stream in streams { - if let Some(peeked) = stream.next().await.transpose()? { - heap.push(Node { stream, peeked }); - } - } - - while let Some(mut node) = heap.peek_mut() { - // Note: If the `next` returns `Err`, we'll fail to yield the previous item. - // This is acceptable since we're not going to handle errors from cell-based table - // iteration, so where to fail does not matter. Or we need an `Option` for this. - yield match node.stream.next().await.transpose()? { - // There still remains data in the stream, take and update the peeked value. - Some(new_peeked) => std::mem::replace(&mut node.peeked, new_peeked), - // This stream is exhausted, remove it from the heap. - None => PeekMut::pop(node).peeked, - }; - } -} - -#[cfg(test)] -mod tests { - use futures_async_stream::for_await; - use risingwave_common::types::ScalarImpl; - - use super::*; - use crate::error::StorageResult; - - fn gen_pk_and_row(i: u8) -> StorageResult<(Vec, OwnedRow)> { - Ok(( - vec![i], - OwnedRow::new(vec![Some(ScalarImpl::Int64(i as _))]), - )) - } - - #[tokio::test] - async fn test_merge_sort() { - let streams = vec![ - futures::stream::iter(vec![ - gen_pk_and_row(0), - gen_pk_and_row(3), - gen_pk_and_row(6), - gen_pk_and_row(9), - ]), - futures::stream::iter(vec![ - gen_pk_and_row(1), - gen_pk_and_row(4), - gen_pk_and_row(7), - gen_pk_and_row(10), - ]), - futures::stream::iter(vec![ - gen_pk_and_row(2), - gen_pk_and_row(5), - gen_pk_and_row(8), - ]), - futures::stream::iter(vec![]), // empty stream - ]; - - let merge_sorted = merge_sort(streams); - - #[for_await] - for (i, result) in merge_sorted.enumerate() { - assert_eq!(result.unwrap(), gen_pk_and_row(i as u8).unwrap()); - } - } -} diff --git a/src/storage/src/table/batch_table/mod.rs b/src/storage/src/table/batch_table/mod.rs index b4712092dc7df..6860aaedb55b4 100644 --- a/src/storage/src/table/batch_table/mod.rs +++ b/src/storage/src/table/batch_table/mod.rs @@ -12,5 +12,4 @@ // See the License for the specific language governing permissions and // limitations under the License. -pub mod iter_utils; pub mod storage_table; diff --git a/src/storage/src/table/batch_table/storage_table.rs b/src/storage/src/table/batch_table/storage_table.rs index 1e1d2d7af60b4..dee752f5a7595 100644 --- a/src/storage/src/table/batch_table/storage_table.rs +++ b/src/storage/src/table/batch_table/storage_table.rs @@ -39,7 +39,6 @@ use risingwave_hummock_sdk::key::{end_bound_of_prefix, next_key, prefixed_range} use risingwave_hummock_sdk::HummockReadEpoch; use tracing::trace; -use super::iter_utils; use crate::error::{StorageError, StorageResult}; use crate::hummock::CachePolicy; use crate::row_serde::row_serde_util::{ @@ -47,7 +46,7 @@ use crate::row_serde::row_serde_util::{ }; use crate::row_serde::{find_columns_by_ids, ColumnMapping}; use crate::store::{PrefetchOptions, ReadOptions}; -use crate::table::{compute_vnode, Distribution, TableIter, DEFAULT_VNODE}; +use crate::table::{compute_vnode, merge_sort, Distribution, TableIter, DEFAULT_VNODE}; use crate::StateStore; /// [`StorageTableInner`] is the interface accessing relational data in KV(`StateStore`) with @@ -511,7 +510,7 @@ impl StorageTableInner { // Concat all iterators if not to preserve order. _ if !ordered => futures::stream::iter(iterators).flatten(), // Merge all iterators if to preserve order. - _ => iter_utils::merge_sort(iterators.into_iter().map(Box::pin).collect()), + _ => merge_sort(iterators.into_iter().map(Box::pin).collect()), }; Ok(iter) diff --git a/src/storage/src/table/mod.rs b/src/storage/src/table/mod.rs index 135122f390e53..dfaa44215d8ba 100644 --- a/src/storage/src/table/mod.rs +++ b/src/storage/src/table/mod.rs @@ -14,9 +14,13 @@ pub mod batch_table; +use std::collections::binary_heap::PeekMut; +use std::collections::BinaryHeap; +use std::error::Error; use std::sync::{Arc, LazyLock}; use futures::{Stream, StreamExt}; +use futures_async_stream::try_stream; use itertools::Itertools; use risingwave_common::array::DataChunk; use risingwave_common::buffer::{Bitmap, BitmapBuilder}; @@ -181,3 +185,107 @@ fn check_vnode_is_set(vnode: VirtualNode, vnodes: &Bitmap) { vnode ); } + +pub trait MergeSortKey = Eq + PartialEq + Ord + PartialOrd; + +struct Node { + stream: S, + + /// The next item polled from `stream` previously. Since the `eq` and `cmp` must be synchronous + /// functions, we need to implement peeking manually. + peeked: (K, OwnedRow), +} + +impl PartialEq for Node { + fn eq(&self, other: &Self) -> bool { + match self.peeked.0 == other.peeked.0 { + true => unreachable!("primary key from different iters should be unique"), + false => false, + } + } +} +impl Eq for Node {} + +impl PartialOrd for Node { + fn partial_cmp(&self, other: &Self) -> Option { + Some(self.cmp(other)) + } +} + +impl Ord for Node { + fn cmp(&self, other: &Self) -> std::cmp::Ordering { + // The heap is a max heap, so we need to reverse the order. + self.peeked.0.cmp(&other.peeked.0).reverse() + } +} + +#[try_stream(ok=(K, OwnedRow), error=E)] +pub async fn merge_sort<'a, K, E, R>(streams: Vec) +where + K: MergeSortKey + 'a, + E: Error + 'a, + R: Stream> + 'a + Unpin, +{ + let mut heap = BinaryHeap::new(); + for mut stream in streams { + if let Some(peeked) = stream.next().await.transpose()? { + heap.push(Node { stream, peeked }); + } + } + while let Some(mut node) = heap.peek_mut() { + // Note: If the `next` returns `Err`, we'll fail to yield the previous item. + yield match node.stream.next().await.transpose()? { + // There still remains data in the stream, take and update the peeked value. + Some(new_peeked) => std::mem::replace(&mut node.peeked, new_peeked), + // This stream is exhausted, remove it from the heap. + None => PeekMut::pop(node).peeked, + }; + } +} + +#[cfg(test)] +mod tests { + use futures_async_stream::for_await; + use risingwave_common::types::ScalarImpl; + + use super::*; + use crate::error::StorageResult; + + fn gen_pk_and_row(i: u8) -> StorageResult<(Vec, OwnedRow)> { + Ok(( + vec![i], + OwnedRow::new(vec![Some(ScalarImpl::Int64(i as _))]), + )) + } + + #[tokio::test] + async fn test_merge_sort() { + let streams = vec![ + futures::stream::iter(vec![ + gen_pk_and_row(0), + gen_pk_and_row(3), + gen_pk_and_row(6), + gen_pk_and_row(9), + ]), + futures::stream::iter(vec![ + gen_pk_and_row(1), + gen_pk_and_row(4), + gen_pk_and_row(7), + gen_pk_and_row(10), + ]), + futures::stream::iter(vec![ + gen_pk_and_row(2), + gen_pk_and_row(5), + gen_pk_and_row(8), + ]), + futures::stream::iter(vec![]), // empty stream + ]; + + let merge_sorted = merge_sort(streams); + + #[for_await] + for (i, result) in merge_sorted.enumerate() { + assert_eq!(result.unwrap(), gen_pk_and_row(i as u8).unwrap()); + } + } +} diff --git a/src/stream/src/common/table/iter_utils.rs b/src/stream/src/common/table/iter_utils.rs deleted file mode 100644 index 142674c3c32ee..0000000000000 --- a/src/stream/src/common/table/iter_utils.rs +++ /dev/null @@ -1,86 +0,0 @@ -// Copyright 2023 RisingWave 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. - -//! FIXME: This is duplicated from batch iter utils, -//! and slightly modified to change the return error + the public key type. -//! Seems difficult to unify them, since batch uses `StorageError` for its iterators, -//! whereas stream uses `StreamExecutorError` for its iterators. -//! The underlying error type should be `StorageError` as well for stream. -//! Additionally since we use `futures_async_stream`'s `try_stream` macro, -//! the return type has to be concrete, so it can't be parameterized on the PK type. -//! Since batch and stream have different PK types (one is Bytes, other is Vec), -//! this also means we have to find some other mechanism to parameterize them, perhaps a macro. - -use std::collections::binary_heap::PeekMut; -use std::collections::BinaryHeap; - -use bytes::Bytes; -use futures::{Stream, StreamExt}; -use futures_async_stream::try_stream; -use risingwave_common::row::OwnedRow; - -use crate::executor::{StreamExecutorError, StreamExecutorResult}; - -pub struct Node { - stream: S, - - /// The next item polled from `stream` previously. Since the `eq` and `cmp` must be synchronous - /// functions, we need to implement peeking manually. - peeked: (Bytes, OwnedRow), -} - -impl PartialEq for Node { - fn eq(&self, other: &Self) -> bool { - match self.peeked.0 == other.peeked.0 { - true => unreachable!("primary key from different iters should be unique"), - false => false, - } - } -} -impl Eq for Node {} - -impl PartialOrd for Node { - fn partial_cmp(&self, other: &Self) -> Option { - Some(self.cmp(other)) - } -} - -impl Ord for Node { - fn cmp(&self, other: &Self) -> std::cmp::Ordering { - // The heap is a max heap, so we need to reverse the order. - self.peeked.0.cmp(&other.peeked.0).reverse() - } -} - -#[try_stream(ok=(Bytes, OwnedRow), error=StreamExecutorError)] -pub async fn merge_sort<'a, R>(streams: Vec) -where - R: Stream> + 'a + Unpin, -{ - let mut heap = BinaryHeap::new(); - for mut stream in streams { - if let Some(peeked) = stream.next().await.transpose()? { - heap.push(Node { stream, peeked }); - } - } - while let Some(mut node) = heap.peek_mut() { - // Note: If the `next` returns `Err`, we'll fail to yield the previous item. - yield match node.stream.next().await.transpose()? { - // There still remains data in the stream, take and update the peeked value. - Some(new_peeked) => std::mem::replace(&mut node.peeked, new_peeked), - // This stream is exhausted, remove it from the heap. - None => PeekMut::pop(node).peeked, - }; - } -} diff --git a/src/stream/src/common/table/mod.rs b/src/stream/src/common/table/mod.rs index ab509d5f44e1c..613ebb3436b02 100644 --- a/src/stream/src/common/table/mod.rs +++ b/src/stream/src/common/table/mod.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -pub mod iter_utils; pub mod state_table; mod watermark; diff --git a/src/stream/src/common/table/state_table.rs b/src/stream/src/common/table/state_table.rs index 62cc789aabb2f..525030776b361 100644 --- a/src/stream/src/common/table/state_table.rs +++ b/src/stream/src/common/table/state_table.rs @@ -45,13 +45,14 @@ use risingwave_storage::row_serde::row_serde_util::{ use risingwave_storage::store::{ LocalStateStore, NewLocalOptions, PrefetchOptions, ReadOptions, StateStoreIterItemStream, }; -use risingwave_storage::table::{compute_chunk_vnode, compute_vnode, get_second, Distribution}; +use risingwave_storage::table::{ + compute_chunk_vnode, compute_vnode, get_second, merge_sort, Distribution, +}; use risingwave_storage::StateStore; use tracing::{trace, Instrument}; use super::watermark::{WatermarkBufferByEpoch, WatermarkBufferStrategy}; use crate::cache::cache_may_stale; -use crate::common::table::iter_utils::merge_sort; use crate::executor::{StreamExecutorError, StreamExecutorResult}; /// This num is arbitrary and we may want to improve this choice in the future. From d2b7ed08c6d83c927dac906607b5813c9d638e52 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Tue, 20 Jun 2023 21:58:27 +0800 Subject: [PATCH 40/90] refactor backfill -> no_shuffle_backfill --- src/stream/src/executor/backfill/mod.rs | 15 +++++++++++++++ .../no_shuffle_backfill.rs} | 11 ++++++----- src/stream/src/executor/mod.rs | 4 ++-- 3 files changed, 23 insertions(+), 7 deletions(-) create mode 100644 src/stream/src/executor/backfill/mod.rs rename src/stream/src/executor/{backfill.rs => backfill/no_shuffle_backfill.rs} (98%) diff --git a/src/stream/src/executor/backfill/mod.rs b/src/stream/src/executor/backfill/mod.rs new file mode 100644 index 0000000000000..94a9963d902b1 --- /dev/null +++ b/src/stream/src/executor/backfill/mod.rs @@ -0,0 +1,15 @@ +// Copyright 2023 RisingWave 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. + +pub mod no_shuffle_backfill; diff --git a/src/stream/src/executor/backfill.rs b/src/stream/src/executor/backfill/no_shuffle_backfill.rs similarity index 98% rename from src/stream/src/executor/backfill.rs rename to src/stream/src/executor/backfill/no_shuffle_backfill.rs index 7381409a4f662..ce5a135090bcb 100644 --- a/src/stream/src/executor/backfill.rs +++ b/src/stream/src/executor/backfill/no_shuffle_backfill.rs @@ -38,11 +38,12 @@ use risingwave_storage::table::batch_table::storage_table::StorageTable; use risingwave_storage::table::{collect_data_chunk, get_second}; use risingwave_storage::StateStore; -use super::error::StreamExecutorError; -use super::{expect_first_barrier, BoxedExecutor, Executor, ExecutorInfo, Message, PkIndicesRef}; use crate::common::table::state_table::StateTable; use crate::executor::monitor::StreamingMetrics; -use crate::executor::{PkIndices, StreamExecutorResult, Watermark}; +use crate::executor::{ + expect_first_barrier, BoxedExecutor, BoxedMessageStream, Executor, ExecutorInfo, Message, + PkIndices, PkIndicesRef, StreamExecutorError, StreamExecutorResult, Watermark, +}; use crate::task::{ActorId, CreateMviewProgress}; /// An implementation of the RFC: Use Backfill To Let Mv On Mv Stream Again.(https://github.com/risingwavelabs/rfcs/pull/13) @@ -664,7 +665,7 @@ impl Executor for BackfillExecutor where S: StateStore, { - fn execute(self: Box) -> super::BoxedMessageStream { + fn execute(self: Box) -> BoxedMessageStream { self.execute_inner().boxed() } @@ -672,7 +673,7 @@ where &self.info.schema } - fn pk_indices(&self) -> super::PkIndicesRef<'_> { + fn pk_indices(&self) -> PkIndicesRef<'_> { &self.info.pk_indices } diff --git a/src/stream/src/executor/mod.rs b/src/stream/src/executor/mod.rs index 1da82e5bf453e..c1f525ff6ce30 100644 --- a/src/stream/src/executor/mod.rs +++ b/src/stream/src/executor/mod.rs @@ -56,6 +56,7 @@ pub mod monitor; pub mod agg_common; pub mod aggregation; mod arrangement_backfill; +mod backfill; mod barrier_recv; mod batch_query; mod chain; @@ -98,14 +99,13 @@ mod watermark; mod watermark_filter; mod wrapper; -mod backfill; #[cfg(test)] mod integration_tests; pub mod test_utils; pub use actor::{Actor, ActorContext, ActorContextRef}; use anyhow::Context; -pub use backfill::*; +pub use backfill::no_shuffle_backfill::*; pub use barrier_recv::BarrierRecvExecutor; pub use batch_query::BatchQueryExecutor; pub use chain::ChainExecutor; From 0166d084e13f3f6d5d0bb00638b73735ae2e6b11 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Tue, 20 Jun 2023 23:00:52 +0800 Subject: [PATCH 41/90] use crate imports --- src/stream/src/executor/arrangement_backfill.rs | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/src/stream/src/executor/arrangement_backfill.rs b/src/stream/src/executor/arrangement_backfill.rs index da82c347ed88b..c9e93701a4292 100644 --- a/src/stream/src/executor/arrangement_backfill.rs +++ b/src/stream/src/executor/arrangement_backfill.rs @@ -35,11 +35,12 @@ use risingwave_common::util::sort_util::{cmp_datum, OrderType}; use risingwave_storage::table::collect_data_chunk; use risingwave_storage::StateStore; -use super::error::StreamExecutorError; -use super::{expect_first_barrier, BoxedExecutor, Executor, ExecutorInfo, Message, PkIndicesRef}; use crate::common::table::state_table::StateTable; use crate::executor::monitor::StreamingMetrics; -use crate::executor::{Barrier, PkIndices, StreamExecutorResult, Watermark}; +use crate::executor::{ + expect_first_barrier, Barrier, BoxedExecutor, Executor, ExecutorInfo, Message, PkIndices, + PkIndicesRef, StreamExecutorError, StreamExecutorResult, Watermark, +}; use crate::task::{ActorId, CreateMviewProgress}; pub struct ArrangementBackfillExecutor { From bbfc06046966bfac831bd7f13cc445ff4207429b Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Tue, 20 Jun 2023 23:02:23 +0800 Subject: [PATCH 42/90] interim --- src/stream/src/executor/{ => backfill}/arrangement_backfill.rs | 0 src/stream/src/executor/backfill/mod.rs | 1 + src/stream/src/executor/mod.rs | 1 - 3 files changed, 1 insertion(+), 1 deletion(-) rename src/stream/src/executor/{ => backfill}/arrangement_backfill.rs (100%) diff --git a/src/stream/src/executor/arrangement_backfill.rs b/src/stream/src/executor/backfill/arrangement_backfill.rs similarity index 100% rename from src/stream/src/executor/arrangement_backfill.rs rename to src/stream/src/executor/backfill/arrangement_backfill.rs diff --git a/src/stream/src/executor/backfill/mod.rs b/src/stream/src/executor/backfill/mod.rs index 94a9963d902b1..8d21936f09704 100644 --- a/src/stream/src/executor/backfill/mod.rs +++ b/src/stream/src/executor/backfill/mod.rs @@ -12,4 +12,5 @@ // See the License for the specific language governing permissions and // limitations under the License. +pub mod arrangement_backfill; pub mod no_shuffle_backfill; diff --git a/src/stream/src/executor/mod.rs b/src/stream/src/executor/mod.rs index c1f525ff6ce30..dacf4a8b6f642 100644 --- a/src/stream/src/executor/mod.rs +++ b/src/stream/src/executor/mod.rs @@ -55,7 +55,6 @@ pub mod monitor; pub mod agg_common; pub mod aggregation; -mod arrangement_backfill; mod backfill; mod barrier_recv; mod batch_query; From 9d170a008a4483185318ca985d831ea208ae49bd Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Tue, 20 Jun 2023 23:51:30 +0800 Subject: [PATCH 43/90] unify utils: part 1 --- .../executor/backfill/arrangement_backfill.rs | 181 ++-------------- src/stream/src/executor/backfill/mod.rs | 1 + .../executor/backfill/no_shuffle_backfill.rs | 33 +-- src/stream/src/executor/backfill/utils.rs | 198 ++++++++++++++++++ 4 files changed, 219 insertions(+), 194 deletions(-) create mode 100644 src/stream/src/executor/backfill/utils.rs diff --git a/src/stream/src/executor/backfill/arrangement_backfill.rs b/src/stream/src/executor/backfill/arrangement_backfill.rs index c9e93701a4292..937ad788eef37 100644 --- a/src/stream/src/executor/backfill/arrangement_backfill.rs +++ b/src/stream/src/executor/backfill/arrangement_backfill.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::cmp::Ordering; use std::ops::Bound; use std::pin::pin; use std::sync::Arc; @@ -24,22 +23,23 @@ use futures::{pin_mut, stream, StreamExt, TryStreamExt}; use futures_async_stream::try_stream; use risingwave_common::array::stream_record::Record; use risingwave_common::array::{Op, StreamChunk}; -use risingwave_common::buffer::BitmapBuilder; use risingwave_common::catalog::Schema; use risingwave_common::hash::VnodeBitmapExt; use risingwave_common::row::{OwnedRow, Row, RowExt}; use risingwave_common::types::Datum; use risingwave_common::util::epoch::EpochPair; -use risingwave_common::util::iter_util::ZipEqFast; -use risingwave_common::util::sort_util::{cmp_datum, OrderType}; use risingwave_storage::table::collect_data_chunk; use risingwave_storage::StateStore; use crate::common::table::state_table::StateTable; +use crate::executor::backfill::utils::{ + build_temporary_state, check_all_vnode_finished, flush_data, mapping_chunk, mapping_message, + mark_chunk_ref, update_pos, +}; use crate::executor::monitor::StreamingMetrics; use crate::executor::{ - expect_first_barrier, Barrier, BoxedExecutor, Executor, ExecutorInfo, Message, PkIndices, - PkIndicesRef, StreamExecutorError, StreamExecutorResult, Watermark, + expect_first_barrier, Barrier, BoxedExecutor, BoxedMessageStream, Executor, ExecutorInfo, + Message, PkIndices, PkIndicesRef, StreamExecutorError, StreamExecutorResult, }; use crate::task::{ActorId, CreateMviewProgress}; @@ -122,7 +122,7 @@ where let first_barrier = expect_first_barrier(&mut upstream).await?; self.state_table.init_epoch(first_barrier.epoch); - let is_finished = Self::check_all_vnode_finished(&self.state_table, state_len).await?; + let is_finished = check_all_vnode_finished(&self.state_table, state_len).await?; if is_finished { assert!(!first_barrier.is_newly_added(self.actor_id)); } @@ -221,8 +221,8 @@ where // If no current_pos, means no snapshot processed yet. // Also means we don't need propagate any updates <= current_pos. if let Some(current_pos) = ¤t_pos { - yield Message::Chunk(Self::mapping_chunk( - Self::mark_chunk( + yield Message::Chunk(mapping_chunk( + mark_chunk_ref( &chunk, current_pos, &pk_in_output_indices, @@ -332,7 +332,7 @@ where let chunk_cardinality = chunk.cardinality() as u64; cur_barrier_snapshot_processed_rows += chunk_cardinality; total_snapshot_processed_rows += chunk_cardinality; - yield Message::Chunk(Self::mapping_chunk( + yield Message::Chunk(mapping_chunk( chunk, &self.output_indices, )); @@ -344,12 +344,12 @@ where // Raise the current position. // As snapshot read streams are ordered by pk, so we can // just use the last row to update `current_pos`. - current_pos = Self::update_pos(&chunk, &pk_in_output_indices); + current_pos = update_pos(&chunk, &pk_in_output_indices); let chunk_cardinality = chunk.cardinality() as u64; cur_barrier_snapshot_processed_rows += chunk_cardinality; total_snapshot_processed_rows += chunk_cardinality; - yield Message::Chunk(Self::mapping_chunk( + yield Message::Chunk(mapping_chunk( chunk, &self.output_indices, )); @@ -369,7 +369,7 @@ where // Wait for first barrier to come after backfill is finished. // So we can update our progress + persist the status. while let Some(Ok(msg)) = upstream.next().await { - if let Some(msg) = Self::mapping_message(msg, &self.output_indices) { + if let Some(msg) = mapping_message(msg, &self.output_indices) { // If not finished then we need to update state, otherwise no need. if let Message::Barrier(barrier) = &msg && !is_finished { // If snapshot was empty, we do not need to backfill, @@ -410,7 +410,7 @@ where // as backfill is finished. #[for_await] for msg in upstream { - if let Some(msg) = Self::mapping_message(msg?, &self.output_indices) { + if let Some(msg) = mapping_message(msg?, &self.output_indices) { if let Message::Barrier(barrier) = &msg { self.state_table.commit_no_data_expected(barrier.epoch); } @@ -460,62 +460,6 @@ where yield None; } - /// Mark chunk: - /// For each row of the chunk, forward it to downstream if its pk <= `current_pos`, otherwise - /// ignore it. We implement it by changing the visibility bitmap. - fn mark_chunk( - chunk: &StreamChunk, - current_pos: &OwnedRow, - pk_in_output_indices: PkIndicesRef<'_>, - pk_order: &[OrderType], - ) -> StreamChunk { - // FIXME: Temporary workaround, we can optimize it later when benchmarking. - // We can do so with a `chunk.compact_ref()` instead of `chunk.compact()`. - let chunk = chunk.clone(); - let chunk = chunk.compact(); - let (data, ops) = chunk.into_parts(); - let mut new_visibility = BitmapBuilder::with_capacity(ops.len()); - // Use project to avoid allocation. - for v in data.rows().map(|row| { - match row - .project(pk_in_output_indices) - .iter() - .zip_eq_fast(pk_order.iter().copied()) - .cmp_by(current_pos.iter(), |(x, order), y| cmp_datum(x, y, order)) - { - Ordering::Less | Ordering::Equal => true, - Ordering::Greater => false, - } - }) { - new_visibility.append(v); - } - let (columns, _) = data.into_parts(); - StreamChunk::new(ops, columns, Some(new_visibility.finish())) - } - - /// Builds a new stream chunk with `output_indices`. - fn mapping_chunk(chunk: StreamChunk, output_indices: &[usize]) -> StreamChunk { - let (ops, columns, visibility) = chunk.into_inner(); - let mapped_columns = output_indices.iter().map(|&i| columns[i].clone()).collect(); - StreamChunk::new(ops, mapped_columns, visibility) - } - - fn mapping_watermark(watermark: Watermark, upstream_indices: &[usize]) -> Option { - watermark.transform_with_indices(upstream_indices) - } - - fn mapping_message(msg: Message, upstream_indices: &[usize]) -> Option { - match msg { - Message::Barrier(_) => Some(msg), - Message::Watermark(watermark) => { - Self::mapping_watermark(watermark, upstream_indices).map(Message::Watermark) - } - Message::Chunk(chunk) => { - Some(Message::Chunk(Self::mapping_chunk(chunk, upstream_indices))) - } - } - } - /// Schema /// | vnode | pk | `backfill_finished` | /// @@ -531,8 +475,8 @@ where ) -> StreamExecutorResult<()> { if let Some(current_pos_inner) = current_pos { // state w/o vnodes. - Self::build_temporary_state(current_state, is_finished, current_pos_inner); - Self::flush_data(table, epoch, old_state, current_state).await?; + build_temporary_state(current_state, is_finished, current_pos_inner); + flush_data(table, epoch, old_state, current_state).await?; *old_state = Some(current_state.into()); } else { table.commit_no_data_expected(epoch); @@ -540,62 +484,6 @@ where Ok(()) } - /// Flush the data - async fn flush_data( - table: &mut StateTable, - epoch: EpochPair, - old_state: &mut Option>, - current_partial_state: &mut [Datum], - ) -> StreamExecutorResult<()> { - let vnodes = table.vnodes().clone(); - if let Some(old_state) = old_state { - if old_state[1..] == current_partial_state[1..] { - table.commit_no_data_expected(epoch); - return Ok(()); - } else { - vnodes.iter_vnodes_scalar().for_each(|vnode| { - let datum = Some(vnode.into()); - current_partial_state[0] = datum.clone(); - old_state[0] = datum; - table.write_record(Record::Update { - old_row: &old_state[..], - new_row: &(*current_partial_state), - }) - }); - } - } else { - // No existing state, create a new entry. - vnodes.iter_vnodes_scalar().for_each(|vnode| { - let datum = Some(vnode.into()); - // fill the state - current_partial_state[0] = datum; - table.write_record(Record::Insert { - new_row: &(*current_partial_state), - }) - }); - } - table.commit(epoch).await - } - - // We want to avoid building a row for every vnode. - // Instead we can just modify a single row, and dispatch it to state table to write. - fn build_temporary_state(row_state: &mut [Datum], is_finished: bool, current_pos: &OwnedRow) { - row_state[1..current_pos.len() + 1].clone_from_slice(current_pos.as_inner()); - row_state[current_pos.len() + 1] = Some(is_finished.into()); - } - - fn update_pos(chunk: &StreamChunk, pk_in_output_indices: &[usize]) -> Option { - Some( - chunk - .rows() - .last() - .unwrap() - .1 - .project(pk_in_output_indices) - .into_owned_row(), - ) - } - // TODO(kwannoel): I'm not sure if ["None" ..] encoding is appropriate // for the case where upstream snapshot is empty, and we want to persist // backfill state as "finished". @@ -606,46 +494,13 @@ where fn construct_initial_finished_state(pos_len: usize) -> Option { Some(OwnedRow::new(vec![None; pos_len])) } - - /// All vnodes should be persisted with status finished. - /// TODO: In the future we will support partial backfill recovery. - /// When that is done, this logic may need to be rewritten to handle - /// partially complete states per vnode. - async fn check_all_vnode_finished( - state_table: &StateTable, - state_len: usize, - ) -> StreamExecutorResult { - debug_assert!(!state_table.vnode_bitmap().is_empty()); - let vnodes = state_table.vnodes().iter_vnodes_scalar(); - let mut is_finished = true; - for vnode in vnodes { - let key: &[Datum] = &[Some(vnode.into())]; - let row = state_table.get_row(key).await?; - - // original_backfill_datum_pos = (state_len - 1) - // value indices are set, so we can -1 for the pk (a single vnode). - let backfill_datum_pos = state_len - 2; - let vnode_is_finished = if let Some(row) = row - && let Some(vnode_is_finished) = row.datum_at(backfill_datum_pos) - { - vnode_is_finished.into_bool() - } else { - false - }; - if !vnode_is_finished { - is_finished = false; - break; - } - } - Ok(is_finished) - } } impl Executor for ArrangementBackfillExecutor where S: StateStore, { - fn execute(self: Box) -> super::BoxedMessageStream { + fn execute(self: Box) -> BoxedMessageStream { self.execute_inner().boxed() } @@ -653,7 +508,7 @@ where &self.info.schema } - fn pk_indices(&self) -> super::PkIndicesRef<'_> { + fn pk_indices(&self) -> PkIndicesRef<'_> { &self.info.pk_indices } diff --git a/src/stream/src/executor/backfill/mod.rs b/src/stream/src/executor/backfill/mod.rs index 8d21936f09704..982c98fcd61ff 100644 --- a/src/stream/src/executor/backfill/mod.rs +++ b/src/stream/src/executor/backfill/mod.rs @@ -14,3 +14,4 @@ pub mod arrangement_backfill; pub mod no_shuffle_backfill; +pub mod utils; diff --git a/src/stream/src/executor/backfill/no_shuffle_backfill.rs b/src/stream/src/executor/backfill/no_shuffle_backfill.rs index ce5a135090bcb..3ffe1bdeac1fd 100644 --- a/src/stream/src/executor/backfill/no_shuffle_backfill.rs +++ b/src/stream/src/executor/backfill/no_shuffle_backfill.rs @@ -39,6 +39,7 @@ use risingwave_storage::table::{collect_data_chunk, get_second}; use risingwave_storage::StateStore; use crate::common::table::state_table::StateTable; +use crate::executor::backfill::utils::mark_chunk; use crate::executor::monitor::StreamingMetrics; use crate::executor::{ expect_first_barrier, BoxedExecutor, BoxedMessageStream, Executor, ExecutorInfo, Message, @@ -266,7 +267,7 @@ where cur_barrier_upstream_processed_rows += chunk.cardinality() as u64; yield Message::Chunk(Self::mapping_chunk( - Self::mark_chunk( + mark_chunk( chunk, current_pos, &pk_in_output_indices, @@ -479,36 +480,6 @@ where yield None; } - /// Mark chunk: - /// For each row of the chunk, forward it to downstream if its pk <= `current_pos`, otherwise - /// ignore it. We implement it by changing the visibility bitmap. - fn mark_chunk( - chunk: StreamChunk, - current_pos: &OwnedRow, - pk_in_output_indices: PkIndicesRef<'_>, - pk_order: &[OrderType], - ) -> StreamChunk { - let chunk = chunk.compact(); - let (data, ops) = chunk.into_parts(); - let mut new_visibility = BitmapBuilder::with_capacity(ops.len()); - // Use project to avoid allocation. - for v in data.rows().map(|row| { - match row - .project(pk_in_output_indices) - .iter() - .zip_eq_fast(pk_order.iter().copied()) - .cmp_by(current_pos.iter(), |(x, order), y| cmp_datum(x, y, order)) - { - Ordering::Less | Ordering::Equal => true, - Ordering::Greater => false, - } - }) { - new_visibility.append(v); - } - let (columns, _) = data.into_parts(); - StreamChunk::new(ops, columns, Some(new_visibility.finish())) - } - /// Builds a new stream chunk with `output_indices`. fn mapping_chunk(chunk: StreamChunk, output_indices: &[usize]) -> StreamChunk { let (ops, columns, visibility) = chunk.into_inner(); diff --git a/src/stream/src/executor/backfill/utils.rs b/src/stream/src/executor/backfill/utils.rs new file mode 100644 index 0000000000000..49cdbc614f3f2 --- /dev/null +++ b/src/stream/src/executor/backfill/utils.rs @@ -0,0 +1,198 @@ +// Copyright 2023 RisingWave 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::cmp::Ordering; + +use risingwave_common::array::stream_record::Record; +use risingwave_common::array::StreamChunk; +use risingwave_common::buffer::BitmapBuilder; +use risingwave_common::hash::VnodeBitmapExt; +use risingwave_common::row::{OwnedRow, Row, RowExt}; +use risingwave_common::types::Datum; +use risingwave_common::util::epoch::EpochPair; +use risingwave_common::util::iter_util::ZipEqFast; +use risingwave_common::util::sort_util::{cmp_datum, OrderType}; +use risingwave_storage::StateStore; + +use crate::common::table::state_table::StateTable; +use crate::executor::{Message, PkIndicesRef, StreamExecutorResult, Watermark}; + +pub(crate) fn mark_chunk( + chunk: StreamChunk, + current_pos: &OwnedRow, + pk_in_output_indices: PkIndicesRef<'_>, + pk_order: &[OrderType], +) -> StreamChunk { + let chunk = chunk.compact(); + mark_chunk_inner(chunk, current_pos, pk_in_output_indices, pk_order) +} + +/// Mark chunk: +/// For each row of the chunk, forward it to downstream if its pk <= `current_pos`, otherwise +/// ignore it. We implement it by changing the visibility bitmap. +pub(crate) fn mark_chunk_ref( + chunk: &StreamChunk, + current_pos: &OwnedRow, + pk_in_output_indices: PkIndicesRef<'_>, + pk_order: &[OrderType], +) -> StreamChunk { + // FIXME: Temporary workaround, we can optimize it later when benchmarking. + // We can do so with a `chunk.compact_ref()` instead of `chunk.compact()`. + let chunk = chunk.clone(); + let chunk = chunk.compact(); + mark_chunk_inner(chunk, current_pos, pk_in_output_indices, pk_order) +} + +/// Mark chunk: +/// For each row of the chunk, forward it to downstream if its pk <= `current_pos`, otherwise +/// ignore it. We implement it by changing the visibility bitmap. +fn mark_chunk_inner( + chunk: StreamChunk, + current_pos: &OwnedRow, + pk_in_output_indices: PkIndicesRef<'_>, + pk_order: &[OrderType], +) -> StreamChunk { + let (data, ops) = chunk.into_parts(); + let mut new_visibility = BitmapBuilder::with_capacity(ops.len()); + // Use project to avoid allocation. + for v in data.rows().map(|row| { + match row + .project(pk_in_output_indices) + .iter() + .zip_eq_fast(pk_order.iter().copied()) + .cmp_by(current_pos.iter(), |(x, order), y| cmp_datum(x, y, order)) + { + Ordering::Less | Ordering::Equal => true, + Ordering::Greater => false, + } + }) { + new_visibility.append(v); + } + let (columns, _) = data.into_parts(); + StreamChunk::new(ops, columns, Some(new_visibility.finish())) +} + +/// Builds a new stream chunk with `output_indices`. +pub(crate) fn mapping_chunk(chunk: StreamChunk, output_indices: &[usize]) -> StreamChunk { + let (ops, columns, visibility) = chunk.into_inner(); + let mapped_columns = output_indices.iter().map(|&i| columns[i].clone()).collect(); + StreamChunk::new(ops, mapped_columns, visibility) +} + +fn mapping_watermark(watermark: Watermark, upstream_indices: &[usize]) -> Option { + watermark.transform_with_indices(upstream_indices) +} + +pub(crate) fn mapping_message(msg: Message, upstream_indices: &[usize]) -> Option { + match msg { + Message::Barrier(_) => Some(msg), + Message::Watermark(watermark) => { + mapping_watermark(watermark, upstream_indices).map(Message::Watermark) + } + Message::Chunk(chunk) => Some(Message::Chunk(mapping_chunk(chunk, upstream_indices))), + } +} + +/// All vnodes should be persisted with status finished. +/// TODO: In the future we will support partial backfill recovery. +/// When that is done, this logic may need to be rewritten to handle +/// partially complete states per vnode. +pub(crate) async fn check_all_vnode_finished( + state_table: &StateTable, + state_len: usize, +) -> StreamExecutorResult { + debug_assert!(!state_table.vnode_bitmap().is_empty()); + let vnodes = state_table.vnodes().iter_vnodes_scalar(); + let mut is_finished = true; + for vnode in vnodes { + let key: &[Datum] = &[Some(vnode.into())]; + let row = state_table.get_row(key).await?; + + // original_backfill_datum_pos = (state_len - 1) + // value indices are set, so we can -1 for the pk (a single vnode). + let backfill_datum_pos = state_len - 2; + let vnode_is_finished = if let Some(row) = row + && let Some(vnode_is_finished) = row.datum_at(backfill_datum_pos) + { + vnode_is_finished.into_bool() + } else { + false + }; + if !vnode_is_finished { + is_finished = false; + break; + } + } + Ok(is_finished) +} + +/// Flush the data +pub(crate) async fn flush_data( + table: &mut StateTable, + epoch: EpochPair, + old_state: &mut Option>, + current_partial_state: &mut [Datum], +) -> StreamExecutorResult<()> { + let vnodes = table.vnodes().clone(); + if let Some(old_state) = old_state { + if old_state[1..] == current_partial_state[1..] { + table.commit_no_data_expected(epoch); + return Ok(()); + } else { + vnodes.iter_vnodes_scalar().for_each(|vnode| { + let datum = Some(vnode.into()); + current_partial_state[0] = datum.clone(); + old_state[0] = datum; + table.write_record(Record::Update { + old_row: &old_state[..], + new_row: &(*current_partial_state), + }) + }); + } + } else { + // No existing state, create a new entry. + vnodes.iter_vnodes_scalar().for_each(|vnode| { + let datum = Some(vnode.into()); + // fill the state + current_partial_state[0] = datum; + table.write_record(Record::Insert { + new_row: &(*current_partial_state), + }) + }); + } + table.commit(epoch).await +} + +// We want to avoid building a row for every vnode. +// Instead we can just modify a single row, and dispatch it to state table to write. +pub(crate) fn build_temporary_state( + row_state: &mut [Datum], + is_finished: bool, + current_pos: &OwnedRow, +) { + row_state[1..current_pos.len() + 1].clone_from_slice(current_pos.as_inner()); + row_state[current_pos.len() + 1] = Some(is_finished.into()); +} + +pub(crate) fn update_pos(chunk: &StreamChunk, pk_in_output_indices: &[usize]) -> Option { + Some( + chunk + .rows() + .last() + .unwrap() + .1 + .project(pk_in_output_indices) + .into_owned_row(), + ) +} From 73fb08a77406a4759eba6392384fe04e499dd1bf Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Tue, 20 Jun 2023 23:58:48 +0800 Subject: [PATCH 44/90] unify utils: part 2 --- .../executor/backfill/arrangement_backfill.rs | 17 +- .../executor/backfill/no_shuffle_backfill.rs | 148 ++---------------- src/stream/src/executor/backfill/utils.rs | 9 ++ 3 files changed, 26 insertions(+), 148 deletions(-) diff --git a/src/stream/src/executor/backfill/arrangement_backfill.rs b/src/stream/src/executor/backfill/arrangement_backfill.rs index 937ad788eef37..b73d4a8c19be1 100644 --- a/src/stream/src/executor/backfill/arrangement_backfill.rs +++ b/src/stream/src/executor/backfill/arrangement_backfill.rs @@ -33,8 +33,8 @@ use risingwave_storage::StateStore; use crate::common::table::state_table::StateTable; use crate::executor::backfill::utils::{ - build_temporary_state, check_all_vnode_finished, flush_data, mapping_chunk, mapping_message, - mark_chunk_ref, update_pos, + build_temporary_state, check_all_vnode_finished, construct_initial_finished_state, flush_data, + mapping_chunk, mapping_message, mark_chunk_ref, update_pos, }; use crate::executor::monitor::StreamingMetrics; use crate::executor::{ @@ -380,7 +380,7 @@ where // (there's no epoch before the first epoch). if is_snapshot_empty { current_pos = - Self::construct_initial_finished_state(pk_in_output_indices.len()) + construct_initial_finished_state(pk_in_output_indices.len()) } // We will update current_pos at least once, @@ -483,17 +483,6 @@ where } Ok(()) } - - // TODO(kwannoel): I'm not sure if ["None" ..] encoding is appropriate - // for the case where upstream snapshot is empty, and we want to persist - // backfill state as "finished". - // Could it be confused with another case where pk position comprised of nulls? - // I don't think it will matter, - // because they both record that backfill is finished. - // We can revisit in future if necessary. - fn construct_initial_finished_state(pos_len: usize) -> Option { - Some(OwnedRow::new(vec![None; pos_len])) - } } impl Executor for ArrangementBackfillExecutor diff --git a/src/stream/src/executor/backfill/no_shuffle_backfill.rs b/src/stream/src/executor/backfill/no_shuffle_backfill.rs index 3ffe1bdeac1fd..51372332404e5 100644 --- a/src/stream/src/executor/backfill/no_shuffle_backfill.rs +++ b/src/stream/src/executor/backfill/no_shuffle_backfill.rs @@ -39,7 +39,10 @@ use risingwave_storage::table::{collect_data_chunk, get_second}; use risingwave_storage::StateStore; use crate::common::table::state_table::StateTable; -use crate::executor::backfill::utils::mark_chunk; +use crate::executor::backfill::utils::{ + build_temporary_state, check_all_vnode_finished, construct_initial_finished_state, flush_data, + mapping_chunk, mapping_message, mark_chunk, update_pos, +}; use crate::executor::monitor::StreamingMetrics; use crate::executor::{ expect_first_barrier, BoxedExecutor, BoxedMessageStream, Executor, ExecutorInfo, Message, @@ -142,7 +145,7 @@ where } let is_finished = if let Some(state_table) = self.state_table.as_mut() { - let is_finished = Self::check_all_vnode_finished(state_table, state_len).await?; + let is_finished = check_all_vnode_finished(state_table, state_len).await?; if is_finished { assert!(!first_barrier.is_newly_added(self.actor_id)); } @@ -266,7 +269,7 @@ where for chunk in upstream_chunk_buffer.drain(..) { cur_barrier_upstream_processed_rows += chunk.cardinality() as u64; - yield Message::Chunk(Self::mapping_chunk( + yield Message::Chunk(mapping_chunk( mark_chunk( chunk, current_pos, @@ -340,7 +343,7 @@ where let chunk_cardinality = chunk.cardinality() as u64; cur_barrier_snapshot_processed_rows += chunk_cardinality; total_snapshot_processed_rows += chunk_cardinality; - yield Message::Chunk(Self::mapping_chunk( + yield Message::Chunk(mapping_chunk( chunk, &self.output_indices, )); @@ -352,12 +355,12 @@ where // Raise the current position. // As snapshot read streams are ordered by pk, so we can // just use the last row to update `current_pos`. - current_pos = Self::update_pos(&chunk, &pk_in_output_indices); + current_pos = update_pos(&chunk, &pk_in_output_indices); let chunk_cardinality = chunk.cardinality() as u64; cur_barrier_snapshot_processed_rows += chunk_cardinality; total_snapshot_processed_rows += chunk_cardinality; - yield Message::Chunk(Self::mapping_chunk( + yield Message::Chunk(mapping_chunk( chunk, &self.output_indices, )); @@ -377,7 +380,7 @@ where // Wait for first barrier to come after backfill is finished. // So we can update our progress + persist the status. while let Some(Ok(msg)) = upstream.next().await { - if let Some(msg) = Self::mapping_message(msg, &self.output_indices) { + if let Some(msg) = mapping_message(msg, &self.output_indices) { // If not finished then we need to update state, otherwise no need. if let Message::Barrier(barrier) = &msg && !is_finished { // If snapshot was empty, we do not need to backfill, @@ -388,7 +391,7 @@ where // (there's no epoch before the first epoch). if is_snapshot_empty { current_pos = - Self::construct_initial_finished_state(pk_in_output_indices.len()) + construct_initial_finished_state(pk_in_output_indices.len()) } // We will update current_pos at least once, @@ -418,7 +421,7 @@ where // as backfill is finished. #[for_await] for msg in upstream { - if let Some(msg) = Self::mapping_message(msg?, &self.output_indices) { + if let Some(msg) = mapping_message(msg?, &self.output_indices) { if let Some(state_table) = self.state_table.as_mut() && let Message::Barrier(barrier) = &msg { state_table.commit_no_data_expected(barrier.epoch); } @@ -480,29 +483,6 @@ where yield None; } - /// Builds a new stream chunk with `output_indices`. - fn mapping_chunk(chunk: StreamChunk, output_indices: &[usize]) -> StreamChunk { - let (ops, columns, visibility) = chunk.into_inner(); - let mapped_columns = output_indices.iter().map(|&i| columns[i].clone()).collect(); - StreamChunk::new(ops, mapped_columns, visibility) - } - - fn mapping_watermark(watermark: Watermark, upstream_indices: &[usize]) -> Option { - watermark.transform_with_indices(upstream_indices) - } - - fn mapping_message(msg: Message, upstream_indices: &[usize]) -> Option { - match msg { - Message::Barrier(_) => Some(msg), - Message::Watermark(watermark) => { - Self::mapping_watermark(watermark, upstream_indices).map(Message::Watermark) - } - Message::Chunk(chunk) => { - Some(Message::Chunk(Self::mapping_chunk(chunk, upstream_indices))) - } - } - } - /// Schema /// | vnode | pk | `backfill_finished` | /// @@ -522,114 +502,14 @@ where }; if let Some(current_pos_inner) = current_pos { // state w/o vnodes. - Self::build_temporary_state(current_state, is_finished, current_pos_inner); - Self::flush_data(table, epoch, old_state, current_state).await?; + build_temporary_state(current_state, is_finished, current_pos_inner); + flush_data(table, epoch, old_state, current_state).await?; *old_state = Some(current_state.into()); } else { table.commit_no_data_expected(epoch); } Ok(()) } - - /// Flush the data - async fn flush_data( - table: &mut StateTable, - epoch: EpochPair, - old_state: &mut Option>, - current_partial_state: &mut [Datum], - ) -> StreamExecutorResult<()> { - let vnodes = table.vnodes().clone(); - if let Some(old_state) = old_state { - if old_state[1..] == current_partial_state[1..] { - table.commit_no_data_expected(epoch); - return Ok(()); - } else { - vnodes.iter_vnodes_scalar().for_each(|vnode| { - let datum = Some(vnode.into()); - current_partial_state[0] = datum.clone(); - old_state[0] = datum; - table.write_record(Record::Update { - old_row: &old_state[..], - new_row: &(*current_partial_state), - }) - }); - } - } else { - // No existing state, create a new entry. - vnodes.iter_vnodes_scalar().for_each(|vnode| { - let datum = Some(vnode.into()); - // fill the state - current_partial_state[0] = datum; - table.write_record(Record::Insert { - new_row: &(*current_partial_state), - }) - }); - } - table.commit(epoch).await - } - - // We want to avoid building a row for every vnode. - // Instead we can just modify a single row, and dispatch it to state table to write. - fn build_temporary_state(row_state: &mut [Datum], is_finished: bool, current_pos: &OwnedRow) { - row_state[1..current_pos.len() + 1].clone_from_slice(current_pos.as_inner()); - row_state[current_pos.len() + 1] = Some(is_finished.into()); - } - - fn update_pos(chunk: &StreamChunk, pk_in_output_indices: &[usize]) -> Option { - Some( - chunk - .rows() - .last() - .unwrap() - .1 - .project(pk_in_output_indices) - .into_owned_row(), - ) - } - - // TODO(kwannoel): I'm not sure if ["None" ..] encoding is appropriate - // for the case where upstream snapshot is empty, and we want to persist - // backfill state as "finished". - // Could it be confused with another case where pk position comprised of nulls? - // I don't think it will matter, - // because they both record that backfill is finished. - // We can revisit in future if necessary. - fn construct_initial_finished_state(pos_len: usize) -> Option { - Some(OwnedRow::new(vec![None; pos_len])) - } - - /// All vnodes should be persisted with status finished. - /// TODO: In the future we will support partial backfill recovery. - /// When that is done, this logic may need to be rewritten to handle - /// partially complete states per vnode. - async fn check_all_vnode_finished( - state_table: &StateTable, - state_len: usize, - ) -> StreamExecutorResult { - debug_assert!(!state_table.vnode_bitmap().is_empty()); - let vnodes = state_table.vnodes().iter_vnodes_scalar(); - let mut is_finished = true; - for vnode in vnodes { - let key: &[Datum] = &[Some(vnode.into())]; - let row = state_table.get_row(key).await?; - - // original_backfill_datum_pos = (state_len - 1) - // value indices are set, so we can -1 for the pk (a single vnode). - let backfill_datum_pos = state_len - 2; - let vnode_is_finished = if let Some(row) = row - && let Some(vnode_is_finished) = row.datum_at(backfill_datum_pos) - { - vnode_is_finished.into_bool() - } else { - false - }; - if !vnode_is_finished { - is_finished = false; - break; - } - } - Ok(is_finished) - } } impl Executor for BackfillExecutor diff --git a/src/stream/src/executor/backfill/utils.rs b/src/stream/src/executor/backfill/utils.rs index 49cdbc614f3f2..a6a61daed811c 100644 --- a/src/stream/src/executor/backfill/utils.rs +++ b/src/stream/src/executor/backfill/utils.rs @@ -196,3 +196,12 @@ pub(crate) fn update_pos(chunk: &StreamChunk, pk_in_output_indices: &[usize]) -> .into_owned_row(), ) } + +// NOTE(kwannoel): ["None" ..] encoding should be appropriate to mark +// the case where upstream snapshot is empty. +// This is so we can persist backfill state as "finished". +// It won't be confused with another case where pk position comprised of nulls, +// because they both record that backfill is finished. +pub(crate) fn construct_initial_finished_state(pos_len: usize) -> Option { + Some(OwnedRow::new(vec![None; pos_len])) +} From ab1c71e488614080f45e6bb11cfda78cc4aac43f Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Wed, 21 Jun 2023 10:07:40 +0800 Subject: [PATCH 45/90] refactor snapshot read internals --- .../executor/backfill/no_shuffle_backfill.rs | 44 +++++--------- src/stream/src/executor/backfill/utils.rs | 57 ++++++++++++++++++- 2 files changed, 69 insertions(+), 32 deletions(-) diff --git a/src/stream/src/executor/backfill/no_shuffle_backfill.rs b/src/stream/src/executor/backfill/no_shuffle_backfill.rs index 51372332404e5..5b447bf2623b3 100644 --- a/src/stream/src/executor/backfill/no_shuffle_backfill.rs +++ b/src/stream/src/executor/backfill/no_shuffle_backfill.rs @@ -21,7 +21,7 @@ use await_tree::InstrumentAwait; use either::Either; use futures::stream::select_with_strategy; use futures::{pin_mut, stream, StreamExt, TryStreamExt}; -use futures_async_stream::try_stream; +use futures_async_stream::{for_await, try_stream}; use risingwave_common::array::stream_record::Record; use risingwave_common::array::{Op, StreamChunk}; use risingwave_common::buffer::BitmapBuilder; @@ -40,8 +40,9 @@ use risingwave_storage::StateStore; use crate::common::table::state_table::StateTable; use crate::executor::backfill::utils::{ - build_temporary_state, check_all_vnode_finished, construct_initial_finished_state, flush_data, - mapping_chunk, mapping_message, mark_chunk, update_pos, + build_temporary_state, check_all_vnode_finished, compute_bounds, + construct_initial_finished_state, flush_data, iter_chunks, mapping_chunk, mapping_message, + mark_chunk, update_pos, }; use crate::executor::monitor::StreamingMetrics; use crate::executor::{ @@ -437,22 +438,14 @@ where current_pos: Option, ordered: bool, ) { - // `current_pos` is None means it needs to scan from the beginning, so we use Unbounded to - // scan. Otherwise, use Excluded. - let range_bounds = if let Some(current_pos) = current_pos { - // If `current_pos` is an empty row which means upstream mv contains only one row and it - // has been consumed. The iter interface doesn't support - // `Excluded(empty_row)` range bound, so we can simply return `None`. - if current_pos.is_empty() { - assert!(upstream_table.pk_indices().is_empty()); - yield None; - return Ok(()); - } + // FIXME(kwannoel): `let-else` pattern does not work with `yield` for some reason. + let range_bounds = compute_bounds(upstream_table.pk_indices(), current_pos); + if range_bounds.is_none() { + yield None; + return Ok(()); + } + let range_bounds = range_bounds.unwrap(); - (Bound::Excluded(current_pos), Bound::Unbounded) - } else { - (Bound::Unbounded, Bound::Unbounded) - }; // We use uncommitted read here, because we have already scheduled the `BackfillExecutor` // together with the upstream mv. let iter = upstream_table @@ -468,19 +461,10 @@ where pin_mut!(iter); - while let Some(data_chunk) = - collect_data_chunk(&mut iter, upstream_table.schema(), Some(CHUNK_SIZE)) - .instrument_await("backfill_snapshot_read") - .await? - { - if data_chunk.cardinality() != 0 { - let ops = vec![Op::Insert; data_chunk.capacity()]; - let stream_chunk = StreamChunk::from_parts(ops, data_chunk); - yield Some(stream_chunk); - } + #[for_await] + for chunk in iter_chunks(iter, upstream_table.schema(), CHUNK_SIZE) { + yield chunk?; } - - yield None; } /// Schema diff --git a/src/stream/src/executor/backfill/utils.rs b/src/stream/src/executor/backfill/utils.rs index a6a61daed811c..942e2fed11223 100644 --- a/src/stream/src/executor/backfill/utils.rs +++ b/src/stream/src/executor/backfill/utils.rs @@ -13,20 +13,28 @@ // limitations under the License. use std::cmp::Ordering; +use std::ops::Bound; +use await_tree::InstrumentAwait; +use futures::Stream; +use futures_async_stream::try_stream; use risingwave_common::array::stream_record::Record; -use risingwave_common::array::StreamChunk; +use risingwave_common::array::{Op, StreamChunk}; use risingwave_common::buffer::BitmapBuilder; +use risingwave_common::catalog::Schema; use risingwave_common::hash::VnodeBitmapExt; use risingwave_common::row::{OwnedRow, Row, RowExt}; use risingwave_common::types::Datum; use risingwave_common::util::epoch::EpochPair; use risingwave_common::util::iter_util::ZipEqFast; use risingwave_common::util::sort_util::{cmp_datum, OrderType}; +use risingwave_storage::table::collect_data_chunk; use risingwave_storage::StateStore; use crate::common::table::state_table::StateTable; -use crate::executor::{Message, PkIndicesRef, StreamExecutorResult, Watermark}; +use crate::executor::{ + Message, PkIndicesRef, StreamExecutorError, StreamExecutorResult, Watermark, +}; pub(crate) fn mark_chunk( chunk: StreamChunk, @@ -205,3 +213,48 @@ pub(crate) fn update_pos(chunk: &StreamChunk, pk_in_output_indices: &[usize]) -> pub(crate) fn construct_initial_finished_state(pos_len: usize) -> Option { Some(OwnedRow::new(vec![None; pos_len])) } + +pub(crate) fn compute_bounds( + pk_indices: &[usize], + current_pos: Option, +) -> Option<(Bound, Bound)> { + // `current_pos` is None means it needs to scan from the beginning, so we use Unbounded to + // scan. Otherwise, use Excluded. + if let Some(current_pos) = current_pos { + // If `current_pos` is an empty row which means upstream mv contains only one row and it + // has been consumed. The iter interface doesn't support + // `Excluded(empty_row)` range bound, so we can simply return `None`. + if current_pos.is_empty() { + assert!(pk_indices.is_empty()); + return None; + } + + Some((Bound::Excluded(current_pos), Bound::Unbounded)) + } else { + Some((Bound::Unbounded, Bound::Unbounded)) + } +} + +#[try_stream(ok = Option, error = StreamExecutorError)] +pub(crate) async fn iter_chunks<'a, S, E>( + mut iter: S, + upstream_table_schema: &'a Schema, + chunk_size: usize, +) where + StreamExecutorError: From, + S: Stream> + Unpin + 'a, +{ + while let Some(data_chunk) = + collect_data_chunk(&mut iter, upstream_table_schema, Some(chunk_size)) + .instrument_await("backfill_snapshot_read") + .await? + { + if data_chunk.cardinality() != 0 { + let ops = vec![Op::Insert; data_chunk.capacity()]; + let stream_chunk = StreamChunk::from_parts(ops, data_chunk); + yield Some(stream_chunk); + } + } + + yield None; +} From de00edb9c794cff4cf5b0bfb284af487a9f729fc Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Wed, 21 Jun 2023 10:20:50 +0800 Subject: [PATCH 46/90] finish persist state --- .../executor/backfill/arrangement_backfill.rs | 30 ++----------------- .../executor/backfill/no_shuffle_backfill.rs | 26 +++++++--------- src/stream/src/executor/backfill/utils.rs | 24 +++++++++++++++ 3 files changed, 38 insertions(+), 42 deletions(-) diff --git a/src/stream/src/executor/backfill/arrangement_backfill.rs b/src/stream/src/executor/backfill/arrangement_backfill.rs index b73d4a8c19be1..baf6beaeb73a4 100644 --- a/src/stream/src/executor/backfill/arrangement_backfill.rs +++ b/src/stream/src/executor/backfill/arrangement_backfill.rs @@ -34,7 +34,7 @@ use risingwave_storage::StateStore; use crate::common::table::state_table::StateTable; use crate::executor::backfill::utils::{ build_temporary_state, check_all_vnode_finished, construct_initial_finished_state, flush_data, - mapping_chunk, mapping_message, mark_chunk_ref, update_pos, + mapping_chunk, mapping_message, mark_chunk_ref, persist_state, update_pos, }; use crate::executor::monitor::StreamingMetrics; use crate::executor::{ @@ -266,7 +266,7 @@ where ); // Persist state on barrier - Self::persist_state( + persist_state( barrier.epoch, &mut self.state_table, false, @@ -388,7 +388,7 @@ where // Or snapshot was empty and we construct a placeholder state. debug_assert_ne!(current_pos, None); - Self::persist_state( + persist_state( barrier.epoch, &mut self.state_table, true, @@ -459,30 +459,6 @@ where yield None; } - - /// Schema - /// | vnode | pk | `backfill_finished` | - /// - /// For `current_pos` and `old_pos` are just pk of upstream. - /// They should be strictly increasing. - async fn persist_state( - epoch: EpochPair, - table: &mut StateTable, - is_finished: bool, - current_pos: &Option, - old_state: &mut Option>, - current_state: &mut [Datum], - ) -> StreamExecutorResult<()> { - if let Some(current_pos_inner) = current_pos { - // state w/o vnodes. - build_temporary_state(current_state, is_finished, current_pos_inner); - flush_data(table, epoch, old_state, current_state).await?; - *old_state = Some(current_state.into()); - } else { - table.commit_no_data_expected(epoch); - } - Ok(()) - } } impl Executor for ArrangementBackfillExecutor diff --git a/src/stream/src/executor/backfill/no_shuffle_backfill.rs b/src/stream/src/executor/backfill/no_shuffle_backfill.rs index 5b447bf2623b3..cb6a45b4928a2 100644 --- a/src/stream/src/executor/backfill/no_shuffle_backfill.rs +++ b/src/stream/src/executor/backfill/no_shuffle_backfill.rs @@ -39,6 +39,7 @@ use risingwave_storage::table::{collect_data_chunk, get_second}; use risingwave_storage::StateStore; use crate::common::table::state_table::StateTable; +use crate::executor::backfill::utils; use crate::executor::backfill::utils::{ build_temporary_state, check_all_vnode_finished, compute_bounds, construct_initial_finished_state, flush_data, iter_chunks, mapping_chunk, mapping_message, @@ -438,7 +439,7 @@ where current_pos: Option, ordered: bool, ) { - // FIXME(kwannoel): `let-else` pattern does not work with `yield` for some reason. + // FIXME(kwannoel): `let-else` pattern does not work in generator. let range_bounds = compute_bounds(upstream_table.pk_indices(), current_pos); if range_bounds.is_none() { yield None; @@ -467,11 +468,6 @@ where } } - /// Schema - /// | vnode | pk | `backfill_finished` | - /// - /// For `current_pos` and `old_pos` are just pk of upstream. - /// They should be strictly increasing. async fn persist_state( epoch: EpochPair, table: &mut Option>, @@ -484,15 +480,15 @@ where let Some(table) = table else { return Ok(()) }; - if let Some(current_pos_inner) = current_pos { - // state w/o vnodes. - build_temporary_state(current_state, is_finished, current_pos_inner); - flush_data(table, epoch, old_state, current_state).await?; - *old_state = Some(current_state.into()); - } else { - table.commit_no_data_expected(epoch); - } - Ok(()) + utils::persist_state( + epoch, + table, + is_finished, + current_pos, + old_state, + current_state, + ) + .await } } diff --git a/src/stream/src/executor/backfill/utils.rs b/src/stream/src/executor/backfill/utils.rs index 942e2fed11223..1876508dc8f06 100644 --- a/src/stream/src/executor/backfill/utils.rs +++ b/src/stream/src/executor/backfill/utils.rs @@ -258,3 +258,27 @@ pub(crate) async fn iter_chunks<'a, S, E>( yield None; } + +/// Schema +/// | vnode | pk | `backfill_finished` | +/// +/// For `current_pos` and `old_pos` are just pk of upstream. +/// They should be strictly increasing. +pub(crate) async fn persist_state( + epoch: EpochPair, + table: &mut StateTable, + is_finished: bool, + current_pos: &Option, + old_state: &mut Option>, + current_state: &mut [Datum], +) -> StreamExecutorResult<()> { + if let Some(current_pos_inner) = current_pos { + // state w/o vnodes. + build_temporary_state(current_state, is_finished, current_pos_inner); + flush_data(table, epoch, old_state, current_state).await?; + *old_state = Some(current_state.into()); + } else { + table.commit_no_data_expected(epoch); + } + Ok(()) +} From 402aa3817b412ddaaccd0c340732e457556af395 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Wed, 21 Jun 2023 10:27:30 +0800 Subject: [PATCH 47/90] refactor snapshot read --- .../executor/backfill/arrangement_backfill.rs | 53 ++++++------------- .../executor/backfill/no_shuffle_backfill.rs | 24 +++------ 2 files changed, 23 insertions(+), 54 deletions(-) diff --git a/src/stream/src/executor/backfill/arrangement_backfill.rs b/src/stream/src/executor/backfill/arrangement_backfill.rs index baf6beaeb73a4..51281e7816ed5 100644 --- a/src/stream/src/executor/backfill/arrangement_backfill.rs +++ b/src/stream/src/executor/backfill/arrangement_backfill.rs @@ -12,34 +12,28 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::ops::Bound; use std::pin::pin; use std::sync::Arc; -use await_tree::InstrumentAwait; use either::Either; use futures::stream::select_with_strategy; use futures::{pin_mut, stream, StreamExt, TryStreamExt}; use futures_async_stream::try_stream; -use risingwave_common::array::stream_record::Record; -use risingwave_common::array::{Op, StreamChunk}; +use risingwave_common::array::StreamChunk; use risingwave_common::catalog::Schema; -use risingwave_common::hash::VnodeBitmapExt; -use risingwave_common::row::{OwnedRow, Row, RowExt}; +use risingwave_common::row::OwnedRow; use risingwave_common::types::Datum; -use risingwave_common::util::epoch::EpochPair; -use risingwave_storage::table::collect_data_chunk; use risingwave_storage::StateStore; use crate::common::table::state_table::StateTable; use crate::executor::backfill::utils::{ - build_temporary_state, check_all_vnode_finished, construct_initial_finished_state, flush_data, + check_all_vnode_finished, compute_bounds, construct_initial_finished_state, iter_chunks, mapping_chunk, mapping_message, mark_chunk_ref, persist_state, update_pos, }; use crate::executor::monitor::StreamingMetrics; use crate::executor::{ expect_first_barrier, Barrier, BoxedExecutor, BoxedMessageStream, Executor, ExecutorInfo, - Message, PkIndices, PkIndicesRef, StreamExecutorError, StreamExecutorResult, + Message, PkIndices, PkIndicesRef, StreamExecutorError, }; use crate::task::{ActorId, CreateMviewProgress}; @@ -425,39 +419,22 @@ where upstream_table: &StateTable, current_pos: Option, ) { - // `current_pos` is None means it needs to scan from the beginning, so we use Unbounded to - // scan. Otherwise, use Excluded. - let range_bounds: (Bound, Bound) = - if let Some(current_pos) = current_pos { - // If `current_pos` is an empty row which means upstream mv contains only one row - // and it has been consumed. The iter interface doesn't support - // `Excluded(empty_row)` range bound, so we can simply return `None`. - if current_pos.is_empty() { - assert!(upstream_table.pk_indices().is_empty()); - yield None; - return Ok(()); - } - - (Bound::Excluded(current_pos), Bound::Unbounded) - } else { - (Bound::Unbounded, Bound::Unbounded) - }; + // FIXME(kwannoel): `let-else` pattern does not work in generator. + let range_bounds = compute_bounds(upstream_table.pk_indices(), current_pos); + if range_bounds.is_none() { + yield None; + return Ok(()); + } + let range_bounds = range_bounds.unwrap(); let iter = upstream_table .iter_all_with_pk_range(&range_bounds, Default::default()) .await?; pin_mut!(iter); - while let Some(data_chunk) = collect_data_chunk(&mut iter, &schema, Some(CHUNK_SIZE)) - .instrument_await("arrangement_backfill_snapshot_read") - .await? - { - if data_chunk.cardinality() != 0 { - let ops = vec![Op::Insert; data_chunk.capacity()]; - let stream_chunk = StreamChunk::from_parts(ops, data_chunk); - yield Some(stream_chunk); - } - } - yield None; + #[for_await] + for chunk in iter_chunks(iter, &schema, CHUNK_SIZE) { + yield chunk?; + } } } diff --git a/src/stream/src/executor/backfill/no_shuffle_backfill.rs b/src/stream/src/executor/backfill/no_shuffle_backfill.rs index cb6a45b4928a2..5d30acb27d6b3 100644 --- a/src/stream/src/executor/backfill/no_shuffle_backfill.rs +++ b/src/stream/src/executor/backfill/no_shuffle_backfill.rs @@ -12,43 +12,35 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::cmp::Ordering; -use std::ops::Bound; use std::pin::pin; use std::sync::Arc; -use await_tree::InstrumentAwait; use either::Either; use futures::stream::select_with_strategy; use futures::{pin_mut, stream, StreamExt, TryStreamExt}; -use futures_async_stream::{for_await, try_stream}; -use risingwave_common::array::stream_record::Record; -use risingwave_common::array::{Op, StreamChunk}; -use risingwave_common::buffer::BitmapBuilder; +use futures_async_stream::try_stream; +use risingwave_common::array::StreamChunk; use risingwave_common::catalog::Schema; -use risingwave_common::hash::VnodeBitmapExt; -use risingwave_common::row::{self, OwnedRow, Row, RowExt}; +use risingwave_common::row; +use risingwave_common::row::OwnedRow; use risingwave_common::types::Datum; use risingwave_common::util::epoch::EpochPair; -use risingwave_common::util::iter_util::ZipEqFast; -use risingwave_common::util::sort_util::{cmp_datum, OrderType}; use risingwave_hummock_sdk::HummockReadEpoch; use risingwave_storage::store::PrefetchOptions; use risingwave_storage::table::batch_table::storage_table::StorageTable; -use risingwave_storage::table::{collect_data_chunk, get_second}; +use risingwave_storage::table::get_second; use risingwave_storage::StateStore; use crate::common::table::state_table::StateTable; use crate::executor::backfill::utils; use crate::executor::backfill::utils::{ - build_temporary_state, check_all_vnode_finished, compute_bounds, - construct_initial_finished_state, flush_data, iter_chunks, mapping_chunk, mapping_message, - mark_chunk, update_pos, + check_all_vnode_finished, compute_bounds, construct_initial_finished_state, iter_chunks, + mapping_chunk, mapping_message, mark_chunk, update_pos, }; use crate::executor::monitor::StreamingMetrics; use crate::executor::{ expect_first_barrier, BoxedExecutor, BoxedMessageStream, Executor, ExecutorInfo, Message, - PkIndices, PkIndicesRef, StreamExecutorError, StreamExecutorResult, Watermark, + PkIndices, PkIndicesRef, StreamExecutorError, StreamExecutorResult, }; use crate::task::{ActorId, CreateMviewProgress}; From 6eb6e21a3d32744fd0594ca247fd323efae85c5e Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Wed, 21 Jun 2023 10:45:31 +0800 Subject: [PATCH 48/90] fix replicated state store init --- src/stream/src/common/table/state_table.rs | 23 ++++++++++--------- .../executor/backfill/arrangement_backfill.rs | 5 ++++ 2 files changed, 17 insertions(+), 11 deletions(-) diff --git a/src/stream/src/common/table/state_table.rs b/src/stream/src/common/table/state_table.rs index 525030776b361..e8efeffb6f412 100644 --- a/src/stream/src/common/table/state_table.rs +++ b/src/stream/src/common/table/state_table.rs @@ -64,6 +64,7 @@ const STATE_CLEANING_PERIOD_EPOCH: usize = 5; pub struct StateTableInner< S, SD = BasicSerde, + const IS_REPLICATED: bool = false, W = WatermarkBufferByEpoch, > where S: StateStore, @@ -122,9 +123,10 @@ pub struct StateTableInner< /// `StateTable` will use `BasicSerde` as default pub type StateTable = StateTableInner; +pub type ReplicatedStateTable = StateTableInner; // initialize -impl StateTableInner +impl StateTableInner where S: StateStore, SD: ValueRowSerde, @@ -190,13 +192,12 @@ where }; let table_option = TableOption::build_table_option(table_catalog.get_properties()); - let local_state_store = store - .new_local(NewLocalOptions::new( - table_id, - is_consistent_op, - table_option, - )) - .await; + let new_local_options = if IS_REPLICATED { + NewLocalOptions::new(table_id, is_consistent_op, table_option) + } else { + NewLocalOptions::new_replicated(table_id, is_consistent_op, table_option) + }; + let local_state_store = store.new_local(new_local_options).await; let pk_data_types = pk_indices .iter() @@ -513,7 +514,7 @@ where } // point get -impl StateTableInner +impl StateTableInner where S: StateStore, SD: ValueRowSerde, @@ -611,7 +612,7 @@ where } // write -impl StateTableInner +impl StateTableInner where S: StateStore, SD: ValueRowSerde, @@ -873,7 +874,7 @@ where } // Iterator functions -impl StateTableInner +impl StateTableInner where S: StateStore, SD: ValueRowSerde, diff --git a/src/stream/src/executor/backfill/arrangement_backfill.rs b/src/stream/src/executor/backfill/arrangement_backfill.rs index 51281e7816ed5..bd227cf35fd16 100644 --- a/src/stream/src/executor/backfill/arrangement_backfill.rs +++ b/src/stream/src/executor/backfill/arrangement_backfill.rs @@ -37,6 +37,11 @@ use crate::executor::{ }; use crate::task::{ActorId, CreateMviewProgress}; +/// Similar to [`BackfillExecutor`]. +/// Main differences: +/// - [`ArrangementBackfillExecutor`] can reside on a different CN, so it can be scaled +/// independently. +/// - To synchronize upstream shared buffer, it is initialized with a [`ReplicatedStateTable`]. pub struct ArrangementBackfillExecutor { /// Upstream table upstream_table: StateTable, From 3d32ee015225a73c852eca6d70cacff4ff1820fc Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Wed, 21 Jun 2023 10:51:23 +0800 Subject: [PATCH 49/90] change instances of state table --- .../src/executor/backfill/arrangement_backfill.rs | 12 ++++++------ src/stream/src/executor/backfill/utils.rs | 15 ++++++++------- 2 files changed, 14 insertions(+), 13 deletions(-) diff --git a/src/stream/src/executor/backfill/arrangement_backfill.rs b/src/stream/src/executor/backfill/arrangement_backfill.rs index bd227cf35fd16..fa54a4b93c0be 100644 --- a/src/stream/src/executor/backfill/arrangement_backfill.rs +++ b/src/stream/src/executor/backfill/arrangement_backfill.rs @@ -25,7 +25,7 @@ use risingwave_common::row::OwnedRow; use risingwave_common::types::Datum; use risingwave_storage::StateStore; -use crate::common::table::state_table::StateTable; +use crate::common::table::state_table::ReplicatedStateTable; use crate::executor::backfill::utils::{ check_all_vnode_finished, compute_bounds, construct_initial_finished_state, iter_chunks, mapping_chunk, mapping_message, mark_chunk_ref, persist_state, update_pos, @@ -44,13 +44,13 @@ use crate::task::{ActorId, CreateMviewProgress}; /// - To synchronize upstream shared buffer, it is initialized with a [`ReplicatedStateTable`]. pub struct ArrangementBackfillExecutor { /// Upstream table - upstream_table: StateTable, + upstream_table: ReplicatedStateTable, /// Upstream with the same schema with the upstream table. upstream: BoxedExecutor, /// Internal state table for persisting state of backfill state. - state_table: StateTable, + state_table: ReplicatedStateTable, /// The column indices need to be forwarded to the downstream from the upstream and table scan. output_indices: Vec, @@ -73,9 +73,9 @@ where #[allow(clippy::too_many_arguments)] #[allow(dead_code)] pub fn new( - upstream_table: StateTable, + upstream_table: ReplicatedStateTable, upstream: BoxedExecutor, - state_table: StateTable, + state_table: ReplicatedStateTable, output_indices: Vec, progress: CreateMviewProgress, schema: Schema, @@ -421,7 +421,7 @@ where #[try_stream(ok = Option, error = StreamExecutorError)] async fn snapshot_read( schema: Arc, - upstream_table: &StateTable, + upstream_table: &ReplicatedStateTable, current_pos: Option, ) { // FIXME(kwannoel): `let-else` pattern does not work in generator. diff --git a/src/stream/src/executor/backfill/utils.rs b/src/stream/src/executor/backfill/utils.rs index 1876508dc8f06..45b5633ba59bb 100644 --- a/src/stream/src/executor/backfill/utils.rs +++ b/src/stream/src/executor/backfill/utils.rs @@ -28,10 +28,11 @@ use risingwave_common::types::Datum; use risingwave_common::util::epoch::EpochPair; use risingwave_common::util::iter_util::ZipEqFast; use risingwave_common::util::sort_util::{cmp_datum, OrderType}; +use risingwave_common::util::value_encoding::BasicSerde; use risingwave_storage::table::collect_data_chunk; use risingwave_storage::StateStore; -use crate::common::table::state_table::StateTable; +use crate::common::table::state_table::StateTableInner; use crate::executor::{ Message, PkIndicesRef, StreamExecutorError, StreamExecutorResult, Watermark, }; @@ -116,8 +117,8 @@ pub(crate) fn mapping_message(msg: Message, upstream_indices: &[usize]) -> Optio /// TODO: In the future we will support partial backfill recovery. /// When that is done, this logic may need to be rewritten to handle /// partially complete states per vnode. -pub(crate) async fn check_all_vnode_finished( - state_table: &StateTable, +pub(crate) async fn check_all_vnode_finished( + state_table: &StateTableInner, state_len: usize, ) -> StreamExecutorResult { debug_assert!(!state_table.vnode_bitmap().is_empty()); @@ -146,8 +147,8 @@ pub(crate) async fn check_all_vnode_finished( } /// Flush the data -pub(crate) async fn flush_data( - table: &mut StateTable, +pub(crate) async fn flush_data( + table: &mut StateTableInner, epoch: EpochPair, old_state: &mut Option>, current_partial_state: &mut [Datum], @@ -264,9 +265,9 @@ pub(crate) async fn iter_chunks<'a, S, E>( /// /// For `current_pos` and `old_pos` are just pk of upstream. /// They should be strictly increasing. -pub(crate) async fn persist_state( +pub(crate) async fn persist_state( epoch: EpochPair, - table: &mut StateTable, + table: &mut StateTableInner, is_finished: bool, current_pos: &Option, old_state: &mut Option>, From 600dae9655fff6d932fb92105775ee368dd71477 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Wed, 21 Jun 2023 11:00:12 +0800 Subject: [PATCH 50/90] add fixme --- src/stream/src/executor/backfill/arrangement_backfill.rs | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/stream/src/executor/backfill/arrangement_backfill.rs b/src/stream/src/executor/backfill/arrangement_backfill.rs index fa54a4b93c0be..cc49a476aac27 100644 --- a/src/stream/src/executor/backfill/arrangement_backfill.rs +++ b/src/stream/src/executor/backfill/arrangement_backfill.rs @@ -239,6 +239,8 @@ where upstream_table.commit(barrier.epoch).await?; } + // TODO(kwannoel): use different counters, otherwise both + // backfill + arrangement backfill executors can't co-exist in same cluster. self.metrics .backfill_snapshot_read_row_count .with_label_values(&[ From b7b065b65a7205f9ad3e9b09032d02002a201cdc Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Wed, 21 Jun 2023 11:02:00 +0800 Subject: [PATCH 51/90] fix --- src/stream/src/common/table/state_table.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/stream/src/common/table/state_table.rs b/src/stream/src/common/table/state_table.rs index e8efeffb6f412..a6f2dfa2b37d0 100644 --- a/src/stream/src/common/table/state_table.rs +++ b/src/stream/src/common/table/state_table.rs @@ -193,9 +193,9 @@ where let table_option = TableOption::build_table_option(table_catalog.get_properties()); let new_local_options = if IS_REPLICATED { - NewLocalOptions::new(table_id, is_consistent_op, table_option) - } else { NewLocalOptions::new_replicated(table_id, is_consistent_op, table_option) + } else { + NewLocalOptions::new(table_id, is_consistent_op, table_option) }; let local_state_store = store.new_local(new_local_options).await; From 30e9eb196d75e3aa19eb936f97fa6a20c12af54f Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Mon, 26 Jun 2023 23:37:02 +0800 Subject: [PATCH 52/90] use match for range_bounds --- .../src/executor/backfill/no_shuffle_backfill.rs | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/src/stream/src/executor/backfill/no_shuffle_backfill.rs b/src/stream/src/executor/backfill/no_shuffle_backfill.rs index 5d30acb27d6b3..9d73445a83904 100644 --- a/src/stream/src/executor/backfill/no_shuffle_backfill.rs +++ b/src/stream/src/executor/backfill/no_shuffle_backfill.rs @@ -431,13 +431,14 @@ where current_pos: Option, ordered: bool, ) { - // FIXME(kwannoel): `let-else` pattern does not work in generator. let range_bounds = compute_bounds(upstream_table.pk_indices(), current_pos); - if range_bounds.is_none() { - yield None; - return Ok(()); - } - let range_bounds = range_bounds.unwrap(); + let range_bounds = match range_bounds { + None => { + yield None; + return Ok(()); + } + Some(range_bounds) => range_bounds, + }; // We use uncommitted read here, because we have already scheduled the `BackfillExecutor` // together with the upstream mv. From 19c806900337690f10264ac3b0ee0bf42cbd5f13 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Tue, 27 Jun 2023 14:51:17 +0800 Subject: [PATCH 53/90] separate MergeSort to separate module --- .../src/table/batch_table/storage_table.rs | 3 +- src/storage/src/table/merge_sort.rs | 125 ++++++++++++++++++ src/storage/src/table/mod.rs | 106 +-------------- src/stream/src/common/table/state_table.rs | 5 +- src/stream/src/executor/backfill/utils.rs | 4 +- 5 files changed, 133 insertions(+), 110 deletions(-) create mode 100644 src/storage/src/table/merge_sort.rs diff --git a/src/storage/src/table/batch_table/storage_table.rs b/src/storage/src/table/batch_table/storage_table.rs index dee752f5a7595..a5cdd76120419 100644 --- a/src/storage/src/table/batch_table/storage_table.rs +++ b/src/storage/src/table/batch_table/storage_table.rs @@ -46,7 +46,8 @@ use crate::row_serde::row_serde_util::{ }; use crate::row_serde::{find_columns_by_ids, ColumnMapping}; use crate::store::{PrefetchOptions, ReadOptions}; -use crate::table::{compute_vnode, merge_sort, Distribution, TableIter, DEFAULT_VNODE}; +use crate::table::merge_sort::merge_sort; +use crate::table::{compute_vnode, Distribution, TableIter, DEFAULT_VNODE}; use crate::StateStore; /// [`StorageTableInner`] is the interface accessing relational data in KV(`StateStore`) with diff --git a/src/storage/src/table/merge_sort.rs b/src/storage/src/table/merge_sort.rs new file mode 100644 index 0000000000000..b349bed10b1da --- /dev/null +++ b/src/storage/src/table/merge_sort.rs @@ -0,0 +1,125 @@ +// Copyright 2023 RisingWave 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::collections::binary_heap::PeekMut; +use std::collections::BinaryHeap; +use std::error::Error; + +use futures::{Stream, StreamExt}; +use futures_async_stream::try_stream; +use risingwave_common::row::OwnedRow; + +pub trait MergeSortKey = Eq + PartialEq + Ord + PartialOrd; + +struct Node { + stream: S, + + /// The next item polled from `stream` previously. Since the `eq` and `cmp` must be synchronous + /// functions, we need to implement peeking manually. + peeked: (K, OwnedRow), +} + +impl PartialEq for Node { + fn eq(&self, other: &Self) -> bool { + match self.peeked.0 == other.peeked.0 { + true => unreachable!("primary key from different iters should be unique"), + false => false, + } + } +} +impl Eq for Node {} + +impl PartialOrd for Node { + fn partial_cmp(&self, other: &Self) -> Option { + Some(self.cmp(other)) + } +} + +impl Ord for Node { + fn cmp(&self, other: &Self) -> std::cmp::Ordering { + // The heap is a max heap, so we need to reverse the order. + self.peeked.0.cmp(&other.peeked.0).reverse() + } +} + +#[try_stream(ok=(K, OwnedRow), error=E)] +pub async fn merge_sort<'a, K, E, R>(streams: Vec) +where + K: MergeSortKey + 'a, + E: Error + 'a, + R: Stream> + 'a + Unpin, +{ + let mut heap = BinaryHeap::new(); + for mut stream in streams { + if let Some(peeked) = stream.next().await.transpose()? { + heap.push(Node { stream, peeked }); + } + } + while let Some(mut node) = heap.peek_mut() { + // Note: If the `next` returns `Err`, we'll fail to yield the previous item. + yield match node.stream.next().await.transpose()? { + // There still remains data in the stream, take and update the peeked value. + Some(new_peeked) => std::mem::replace(&mut node.peeked, new_peeked), + // This stream is exhausted, remove it from the heap. + None => PeekMut::pop(node).peeked, + }; + } +} + +#[cfg(test)] +mod tests { + use futures_async_stream::for_await; + use risingwave_common::types::ScalarImpl; + + use super::*; + use crate::error::StorageResult; + + fn gen_pk_and_row(i: u8) -> StorageResult<(Vec, OwnedRow)> { + Ok(( + vec![i], + OwnedRow::new(vec![Some(ScalarImpl::Int64(i as _))]), + )) + } + + #[tokio::test] + async fn test_merge_sort() { + let streams = vec![ + futures::stream::iter(vec![ + gen_pk_and_row(0), + gen_pk_and_row(3), + gen_pk_and_row(6), + gen_pk_and_row(9), + ]), + futures::stream::iter(vec![ + gen_pk_and_row(1), + gen_pk_and_row(4), + gen_pk_and_row(7), + gen_pk_and_row(10), + ]), + futures::stream::iter(vec![ + gen_pk_and_row(2), + gen_pk_and_row(5), + gen_pk_and_row(8), + ]), + futures::stream::iter(vec![]), // empty stream + ]; + + let merge_sorted = merge_sort(streams); + + #[for_await] + for (i, result) in merge_sorted.enumerate() { + assert_eq!(result.unwrap(), gen_pk_and_row(i as u8).unwrap()); + } + } +} diff --git a/src/storage/src/table/mod.rs b/src/storage/src/table/mod.rs index dfaa44215d8ba..88ef053e0de88 100644 --- a/src/storage/src/table/mod.rs +++ b/src/storage/src/table/mod.rs @@ -13,7 +13,7 @@ // limitations under the License. pub mod batch_table; - +pub mod merge_sort; use std::collections::binary_heap::PeekMut; use std::collections::BinaryHeap; use std::error::Error; @@ -185,107 +185,3 @@ fn check_vnode_is_set(vnode: VirtualNode, vnodes: &Bitmap) { vnode ); } - -pub trait MergeSortKey = Eq + PartialEq + Ord + PartialOrd; - -struct Node { - stream: S, - - /// The next item polled from `stream` previously. Since the `eq` and `cmp` must be synchronous - /// functions, we need to implement peeking manually. - peeked: (K, OwnedRow), -} - -impl PartialEq for Node { - fn eq(&self, other: &Self) -> bool { - match self.peeked.0 == other.peeked.0 { - true => unreachable!("primary key from different iters should be unique"), - false => false, - } - } -} -impl Eq for Node {} - -impl PartialOrd for Node { - fn partial_cmp(&self, other: &Self) -> Option { - Some(self.cmp(other)) - } -} - -impl Ord for Node { - fn cmp(&self, other: &Self) -> std::cmp::Ordering { - // The heap is a max heap, so we need to reverse the order. - self.peeked.0.cmp(&other.peeked.0).reverse() - } -} - -#[try_stream(ok=(K, OwnedRow), error=E)] -pub async fn merge_sort<'a, K, E, R>(streams: Vec) -where - K: MergeSortKey + 'a, - E: Error + 'a, - R: Stream> + 'a + Unpin, -{ - let mut heap = BinaryHeap::new(); - for mut stream in streams { - if let Some(peeked) = stream.next().await.transpose()? { - heap.push(Node { stream, peeked }); - } - } - while let Some(mut node) = heap.peek_mut() { - // Note: If the `next` returns `Err`, we'll fail to yield the previous item. - yield match node.stream.next().await.transpose()? { - // There still remains data in the stream, take and update the peeked value. - Some(new_peeked) => std::mem::replace(&mut node.peeked, new_peeked), - // This stream is exhausted, remove it from the heap. - None => PeekMut::pop(node).peeked, - }; - } -} - -#[cfg(test)] -mod tests { - use futures_async_stream::for_await; - use risingwave_common::types::ScalarImpl; - - use super::*; - use crate::error::StorageResult; - - fn gen_pk_and_row(i: u8) -> StorageResult<(Vec, OwnedRow)> { - Ok(( - vec![i], - OwnedRow::new(vec![Some(ScalarImpl::Int64(i as _))]), - )) - } - - #[tokio::test] - async fn test_merge_sort() { - let streams = vec![ - futures::stream::iter(vec![ - gen_pk_and_row(0), - gen_pk_and_row(3), - gen_pk_and_row(6), - gen_pk_and_row(9), - ]), - futures::stream::iter(vec![ - gen_pk_and_row(1), - gen_pk_and_row(4), - gen_pk_and_row(7), - gen_pk_and_row(10), - ]), - futures::stream::iter(vec![ - gen_pk_and_row(2), - gen_pk_and_row(5), - gen_pk_and_row(8), - ]), - futures::stream::iter(vec![]), // empty stream - ]; - - let merge_sorted = merge_sort(streams); - - #[for_await] - for (i, result) in merge_sorted.enumerate() { - assert_eq!(result.unwrap(), gen_pk_and_row(i as u8).unwrap()); - } - } -} diff --git a/src/stream/src/common/table/state_table.rs b/src/stream/src/common/table/state_table.rs index a6f2dfa2b37d0..d2d4a57c6425d 100644 --- a/src/stream/src/common/table/state_table.rs +++ b/src/stream/src/common/table/state_table.rs @@ -45,9 +45,8 @@ use risingwave_storage::row_serde::row_serde_util::{ use risingwave_storage::store::{ LocalStateStore, NewLocalOptions, PrefetchOptions, ReadOptions, StateStoreIterItemStream, }; -use risingwave_storage::table::{ - compute_chunk_vnode, compute_vnode, get_second, merge_sort, Distribution, -}; +use risingwave_storage::table::merge_sort::merge_sort; +use risingwave_storage::table::{compute_chunk_vnode, compute_vnode, get_second, Distribution}; use risingwave_storage::StateStore; use tracing::{trace, Instrument}; diff --git a/src/stream/src/executor/backfill/utils.rs b/src/stream/src/executor/backfill/utils.rs index 45b5633ba59bb..c3df7d6711855 100644 --- a/src/stream/src/executor/backfill/utils.rs +++ b/src/stream/src/executor/backfill/utils.rs @@ -56,9 +56,11 @@ pub(crate) fn mark_chunk_ref( pk_in_output_indices: PkIndicesRef<'_>, pk_order: &[OrderType], ) -> StreamChunk { - // FIXME: Temporary workaround, we can optimize it later when benchmarking. + // FIXME(kwannoel): Temporary workaround, we can optimize it later when benchmarking. // We can do so with a `chunk.compact_ref()` instead of `chunk.compact()`. let chunk = chunk.clone(); + // TODO(kwannoel): Do we even need `compact` here? + // Probably need some benchmark to know for sure. let chunk = chunk.compact(); mark_chunk_inner(chunk, current_pos, pk_in_output_indices, pk_order) } From f40f109f8c5c8f15f14fa560127ef94a5be60d34 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Tue, 27 Jun 2023 15:45:48 +0800 Subject: [PATCH 54/90] cmp datum iter --- src/stream/src/executor/backfill/utils.rs | 12 +++++------- 1 file changed, 5 insertions(+), 7 deletions(-) diff --git a/src/stream/src/executor/backfill/utils.rs b/src/stream/src/executor/backfill/utils.rs index c3df7d6711855..b0c340034cda4 100644 --- a/src/stream/src/executor/backfill/utils.rs +++ b/src/stream/src/executor/backfill/utils.rs @@ -27,7 +27,7 @@ use risingwave_common::row::{OwnedRow, Row, RowExt}; use risingwave_common::types::Datum; use risingwave_common::util::epoch::EpochPair; use risingwave_common::util::iter_util::ZipEqFast; -use risingwave_common::util::sort_util::{cmp_datum, OrderType}; +use risingwave_common::util::sort_util::{cmp_datum, cmp_datum_iter, OrderType}; use risingwave_common::util::value_encoding::BasicSerde; use risingwave_storage::table::collect_data_chunk; use risingwave_storage::StateStore; @@ -78,12 +78,10 @@ fn mark_chunk_inner( let mut new_visibility = BitmapBuilder::with_capacity(ops.len()); // Use project to avoid allocation. for v in data.rows().map(|row| { - match row - .project(pk_in_output_indices) - .iter() - .zip_eq_fast(pk_order.iter().copied()) - .cmp_by(current_pos.iter(), |(x, order), y| cmp_datum(x, y, order)) - { + let lhs = row.project(pk_in_output_indices); + let rhs = current_pos.project(pk_in_output_indices); + let order = cmp_datum_iter(lhs.iter(), rhs.iter(), pk_order.iter().copied()); + match order { Ordering::Less | Ordering::Equal => true, Ordering::Greater => false, } From 4c7e6c02fc50672d7aba1a9091efa08e60e2d1fe Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Tue, 27 Jun 2023 15:48:26 +0800 Subject: [PATCH 55/90] debug_assert data_chunk cardinality --- src/stream/src/executor/backfill/utils.rs | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/src/stream/src/executor/backfill/utils.rs b/src/stream/src/executor/backfill/utils.rs index b0c340034cda4..a8f59700df52e 100644 --- a/src/stream/src/executor/backfill/utils.rs +++ b/src/stream/src/executor/backfill/utils.rs @@ -250,11 +250,10 @@ pub(crate) async fn iter_chunks<'a, S, E>( .instrument_await("backfill_snapshot_read") .await? { - if data_chunk.cardinality() != 0 { - let ops = vec![Op::Insert; data_chunk.capacity()]; - let stream_chunk = StreamChunk::from_parts(ops, data_chunk); - yield Some(stream_chunk); - } + debug_assert!(data_chunk.cardinality() > 0); + let ops = vec![Op::Insert; data_chunk.capacity()]; + let stream_chunk = StreamChunk::from_parts(ops, data_chunk); + yield Some(stream_chunk); } yield None; From c15244d12480ef6aaf6e9a74720ad56c5b14b074 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Tue, 27 Jun 2023 16:10:05 +0800 Subject: [PATCH 56/90] rm compact + risedev cf --- src/storage/src/table/mod.rs | 5 +---- src/stream/src/executor/backfill/utils.rs | 8 +------- 2 files changed, 2 insertions(+), 11 deletions(-) diff --git a/src/storage/src/table/mod.rs b/src/storage/src/table/mod.rs index 88ef053e0de88..85c40d6722fee 100644 --- a/src/storage/src/table/mod.rs +++ b/src/storage/src/table/mod.rs @@ -14,13 +14,10 @@ pub mod batch_table; pub mod merge_sort; -use std::collections::binary_heap::PeekMut; -use std::collections::BinaryHeap; -use std::error::Error; + use std::sync::{Arc, LazyLock}; use futures::{Stream, StreamExt}; -use futures_async_stream::try_stream; use itertools::Itertools; use risingwave_common::array::DataChunk; use risingwave_common::buffer::{Bitmap, BitmapBuilder}; diff --git a/src/stream/src/executor/backfill/utils.rs b/src/stream/src/executor/backfill/utils.rs index a8f59700df52e..54bb2e24b6574 100644 --- a/src/stream/src/executor/backfill/utils.rs +++ b/src/stream/src/executor/backfill/utils.rs @@ -26,8 +26,7 @@ use risingwave_common::hash::VnodeBitmapExt; use risingwave_common::row::{OwnedRow, Row, RowExt}; use risingwave_common::types::Datum; use risingwave_common::util::epoch::EpochPair; -use risingwave_common::util::iter_util::ZipEqFast; -use risingwave_common::util::sort_util::{cmp_datum, cmp_datum_iter, OrderType}; +use risingwave_common::util::sort_util::{cmp_datum_iter, OrderType}; use risingwave_common::util::value_encoding::BasicSerde; use risingwave_storage::table::collect_data_chunk; use risingwave_storage::StateStore; @@ -56,12 +55,7 @@ pub(crate) fn mark_chunk_ref( pk_in_output_indices: PkIndicesRef<'_>, pk_order: &[OrderType], ) -> StreamChunk { - // FIXME(kwannoel): Temporary workaround, we can optimize it later when benchmarking. - // We can do so with a `chunk.compact_ref()` instead of `chunk.compact()`. let chunk = chunk.clone(); - // TODO(kwannoel): Do we even need `compact` here? - // Probably need some benchmark to know for sure. - let chunk = chunk.compact(); mark_chunk_inner(chunk, current_pos, pk_in_output_indices, pk_order) } From 4321e7ed99fa8206766c2b32465afc7fa320d639 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Tue, 27 Jun 2023 16:37:08 +0800 Subject: [PATCH 57/90] docs + add current_pos map --- .../executor/backfill/arrangement_backfill.rs | 32 ++++++++++++------- 1 file changed, 20 insertions(+), 12 deletions(-) diff --git a/src/stream/src/executor/backfill/arrangement_backfill.rs b/src/stream/src/executor/backfill/arrangement_backfill.rs index cc49a476aac27..6c43a91d41854 100644 --- a/src/stream/src/executor/backfill/arrangement_backfill.rs +++ b/src/stream/src/executor/backfill/arrangement_backfill.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::collections::HashMap; use std::pin::pin; use std::sync::Arc; @@ -141,20 +142,24 @@ where } }; - // | backfill_is_finished | snapshot_empty | need_to_backfill | - // | t | t/f | f | - // | f | t | f | - // | f | f | t | + // | backfill_is_finished | snapshot_empty | -> | need_to_backfill | + // | -------------------- | -------------- | -- | ---------------- | + // | t | t/f | -> | f | + // | f | t | -> | f | + // | f | f | -> | t | let to_backfill = !is_finished && !is_snapshot_empty; - // Current position of the upstream_table storage primary key. + // Current position of upstream_table primary key. + // Current position is computed **per vnode**. + let mut current_pos_map: HashMap, OwnedRow> = HashMap::new(); + + // Current position of the upstream_table primary key. // `None` means it starts from the beginning. let mut current_pos: Option = None; // Use these to persist state. - // They contain the backfill position, - // as well as the progress. - // However, they do not contain the vnode key at index 0. + // They contain the backfill position, and the progress. + // However, they do not contain the vnode key (index 0). // That is filled in when we flush the state table. let mut current_state: Vec = vec![None; state_len]; let mut old_state: Option> = None; @@ -178,7 +183,7 @@ where // Keep track of rows from the snapshot. let mut total_snapshot_processed_rows: u64 = 0; - // Backfill Algorithm: + // Arrangement Backfill Algorithm: // // backfill_stream // / \ @@ -188,10 +193,13 @@ where // stream as its right input. When a chunk comes from upstream, we will buffer it. // // When a barrier comes from upstream: - // - Update the `snapshot_read_epoch`. - // - For each row of the upstream chunk buffer, forward it to downstream if its pk <= + // Immediately break out of backfill loop. + // - For each row of the upstream chunk buffer, compute vnode. + // - Get the current_pos corresponding to the vnode. forward it to downstream if its pk <= // `current_pos`, otherwise ignore it. - // - reconstruct the whole backfill stream with upstream and new mv snapshot read stream + // - Flush all buffered upstream_chunks to replicated state table. + // - Update the `snapshot_read_epoch`. + // - Reconstruct the whole backfill stream with upstream and new mv snapshot read stream // with the `snapshot_read_epoch`. // // When a chunk comes from snapshot, we forward it to the downstream and raise From f8de0c077a0b2b15f3c4e3bd53e5f81a8d0f066a Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Tue, 27 Jun 2023 16:38:43 +0800 Subject: [PATCH 58/90] just use a vec --- src/stream/src/executor/backfill/arrangement_backfill.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/stream/src/executor/backfill/arrangement_backfill.rs b/src/stream/src/executor/backfill/arrangement_backfill.rs index 6c43a91d41854..6cd43674bfca5 100644 --- a/src/stream/src/executor/backfill/arrangement_backfill.rs +++ b/src/stream/src/executor/backfill/arrangement_backfill.rs @@ -151,7 +151,7 @@ where // Current position of upstream_table primary key. // Current position is computed **per vnode**. - let mut current_pos_map: HashMap, OwnedRow> = HashMap::new(); + let mut current_pos_map = vec![None; self.upstream_table.vnodes().count_ones()]; // Current position of the upstream_table primary key. // `None` means it starts from the beginning. From 424a7ca98a720cc718c296839bba224dd27cf3b9 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Tue, 27 Jun 2023 17:03:29 +0800 Subject: [PATCH 59/90] add mark chunk per vnode --- .../executor/backfill/arrangement_backfill.rs | 7 ++-- src/stream/src/executor/backfill/utils.rs | 34 ++++++++++++++++++- 2 files changed, 37 insertions(+), 4 deletions(-) diff --git a/src/stream/src/executor/backfill/arrangement_backfill.rs b/src/stream/src/executor/backfill/arrangement_backfill.rs index 6cd43674bfca5..5aee76e65ae3f 100644 --- a/src/stream/src/executor/backfill/arrangement_backfill.rs +++ b/src/stream/src/executor/backfill/arrangement_backfill.rs @@ -22,6 +22,7 @@ use futures::{pin_mut, stream, StreamExt, TryStreamExt}; use futures_async_stream::try_stream; use risingwave_common::array::StreamChunk; use risingwave_common::catalog::Schema; +use risingwave_common::hash::VirtualNode; use risingwave_common::row::OwnedRow; use risingwave_common::types::Datum; use risingwave_storage::StateStore; @@ -151,7 +152,7 @@ where // Current position of upstream_table primary key. // Current position is computed **per vnode**. - let mut current_pos_map = vec![None; self.upstream_table.vnodes().count_ones()]; + let mut current_pos_map: HashMap = HashMap::new(); // Current position of the upstream_table primary key. // `None` means it starts from the beginning. @@ -195,8 +196,8 @@ where // When a barrier comes from upstream: // Immediately break out of backfill loop. // - For each row of the upstream chunk buffer, compute vnode. - // - Get the current_pos corresponding to the vnode. forward it to downstream if its pk <= - // `current_pos`, otherwise ignore it. + // - Get the `current_pos` corresponding to the vnode. Forward it to downstream if its pk + // <= `current_pos`, otherwise ignore it. // - Flush all buffered upstream_chunks to replicated state table. // - Update the `snapshot_read_epoch`. // - Reconstruct the whole backfill stream with upstream and new mv snapshot read stream diff --git a/src/stream/src/executor/backfill/utils.rs b/src/stream/src/executor/backfill/utils.rs index 54bb2e24b6574..473e68e2314da 100644 --- a/src/stream/src/executor/backfill/utils.rs +++ b/src/stream/src/executor/backfill/utils.rs @@ -13,6 +13,8 @@ // limitations under the License. use std::cmp::Ordering; +use std::collections::HashMap; +use std::hash::Hash; use std::ops::Bound; use await_tree::InstrumentAwait; @@ -22,7 +24,7 @@ use risingwave_common::array::stream_record::Record; use risingwave_common::array::{Op, StreamChunk}; use risingwave_common::buffer::BitmapBuilder; use risingwave_common::catalog::Schema; -use risingwave_common::hash::VnodeBitmapExt; +use risingwave_common::hash::{VirtualNode, VnodeBitmapExt}; use risingwave_common::row::{OwnedRow, Row, RowExt}; use risingwave_common::types::Datum; use risingwave_common::util::epoch::EpochPair; @@ -46,6 +48,36 @@ pub(crate) fn mark_chunk( mark_chunk_inner(chunk, current_pos, pk_in_output_indices, pk_order) } +/// Mark chunk: +/// For each row of the chunk, forward it to downstream if its pk <= `current_pos` for the +/// corresponding vnode, otherwise ignore it. We implement it by changing the visibility bitmap. +pub(crate) fn mark_chunk_ref_by_vnode( + chunk: &StreamChunk, + current_pos_map: &HashMap, + pk_in_output_indices: PkIndicesRef<'_>, + pk_order: &[OrderType], +) -> StreamChunk { + let chunk = chunk.clone(); + let (data, ops) = chunk.into_parts(); + let mut new_visibility = BitmapBuilder::with_capacity(ops.len()); + // Use project to avoid allocation. + for v in data.rows().map(|row| { + let vnode = VirtualNode::ZERO; // FIXME: compute vnode from row + state table. + let current_pos = current_pos_map.get(&vnode).unwrap(); + let lhs = row.project(pk_in_output_indices); + let rhs = current_pos.project(pk_in_output_indices); + let order = cmp_datum_iter(lhs.iter(), rhs.iter(), pk_order.iter().copied()); + match order { + Ordering::Less | Ordering::Equal => true, + Ordering::Greater => false, + } + }) { + new_visibility.append(v); + } + let (columns, _) = data.into_parts(); + StreamChunk::new(ops, columns, Some(new_visibility.finish())) +} + /// Mark chunk: /// For each row of the chunk, forward it to downstream if its pk <= `current_pos`, otherwise /// ignore it. We implement it by changing the visibility bitmap. From 565ff49601e3f51bf59d510874418439c8504d0b Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Tue, 27 Jun 2023 17:21:51 +0800 Subject: [PATCH 60/90] add docs + FIXME --- src/stream/src/executor/backfill/arrangement_backfill.rs | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/stream/src/executor/backfill/arrangement_backfill.rs b/src/stream/src/executor/backfill/arrangement_backfill.rs index 5aee76e65ae3f..5567b988d0fce 100644 --- a/src/stream/src/executor/backfill/arrangement_backfill.rs +++ b/src/stream/src/executor/backfill/arrangement_backfill.rs @@ -103,6 +103,7 @@ where #[try_stream(ok = Message, error = StreamExecutorError)] async fn execute_inner(mut self) { // The primary key columns, in the output columns of the upstream_table scan. + // Table scan scans a subset of the columns of the upstream table. let pk_in_output_indices = self .upstream_table .pk_indices() @@ -110,6 +111,7 @@ where .map(|&i| self.output_indices.iter().position(|&j| i == j)) .collect::>>() .unwrap(); + // FIXME: This is wrong, it should use pk. let state_len = pk_in_output_indices.len() + 2; // +1 for backfill_finished, +1 for vnode key. let pk_order = self.upstream_table.pk_serde().get_order_types().to_vec(); let upstream_table_id = self.upstream_table.table_id(); From 16cf7c6a6c95ea0ef3a595e69362ded777a3d571 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Mon, 3 Jul 2023 13:33:16 +0800 Subject: [PATCH 61/90] add snapshot_read_per_vnode_skeleton --- .../executor/backfill/arrangement_backfill.rs | 60 +++++++++++++++++-- 1 file changed, 55 insertions(+), 5 deletions(-) diff --git a/src/stream/src/executor/backfill/arrangement_backfill.rs b/src/stream/src/executor/backfill/arrangement_backfill.rs index 5567b988d0fce..eb3a1d1092773 100644 --- a/src/stream/src/executor/backfill/arrangement_backfill.rs +++ b/src/stream/src/executor/backfill/arrangement_backfill.rs @@ -22,7 +22,7 @@ use futures::{pin_mut, stream, StreamExt, TryStreamExt}; use futures_async_stream::try_stream; use risingwave_common::array::StreamChunk; use risingwave_common::catalog::Schema; -use risingwave_common::hash::VirtualNode; +use risingwave_common::hash::{VirtualNode, VnodeBitmapExt}; use risingwave_common::row::OwnedRow; use risingwave_common::types::Datum; use risingwave_storage::StateStore; @@ -30,7 +30,8 @@ use risingwave_storage::StateStore; use crate::common::table::state_table::ReplicatedStateTable; use crate::executor::backfill::utils::{ check_all_vnode_finished, compute_bounds, construct_initial_finished_state, iter_chunks, - mapping_chunk, mapping_message, mark_chunk_ref, persist_state, update_pos, + mapping_chunk, mapping_message, mark_chunk_ref, mark_chunk_ref_by_vnode, persist_state, + update_pos, }; use crate::executor::monitor::StreamingMetrics; use crate::executor::{ @@ -39,6 +40,8 @@ use crate::executor::{ }; use crate::task::{ActorId, CreateMviewProgress}; +type CurrentPosMap = HashMap; + /// Similar to [`BackfillExecutor`]. /// Main differences: /// - [`ArrangementBackfillExecutor`] can reside on a different CN, so it can be scaled @@ -154,7 +157,7 @@ where // Current position of upstream_table primary key. // Current position is computed **per vnode**. - let mut current_pos_map: HashMap = HashMap::new(); + let mut current_pos_map: CurrentPosMap = HashMap::new(); // Current position of the upstream_table primary key. // `None` means it starts from the beginning. @@ -232,9 +235,9 @@ where // Also means we don't need propagate any updates <= current_pos. if let Some(current_pos) = ¤t_pos { yield Message::Chunk(mapping_chunk( - mark_chunk_ref( + mark_chunk_ref_by_vnode( &chunk, - current_pos, + ¤t_pos_map, &pk_in_output_indices, &pk_order, ), @@ -431,6 +434,53 @@ where } } + /// Read snapshot per vnode. + /// These streams should be sorted in storage layer. + /// 1. Get row iterator / vnode. + /// 2. Merge it with futures unordered. + /// 3. Change it into a chunk iterator with `iter_chunks`. + #[try_stream(ok = Option, error = StreamExecutorError)] + async fn snapshot_read_per_vnode<'a>( + schema: Arc, + upstream_table: &'a ReplicatedStateTable, + current_pos_map: &'a CurrentPosMap, + ) { + for vnode in upstream_table.vnodes().iter_vnodes() { + let current_pos = current_pos_map.get(&vnode); + let range_bounds = compute_bounds(upstream_table.pk_indices(), current_pos.cloned()); + if range_bounds.is_none() { + yield None; + return Ok(()); + } + let range_bounds = range_bounds.unwrap(); + let iter = upstream_table + .iter_all_with_pk_range(&range_bounds, Default::default()) + .await?; + pin_mut!(iter); + + #[for_await] + for chunk in iter_chunks(iter, &schema, CHUNK_SIZE) { + yield chunk?; + } + } + // let range_bounds = compute_bounds(upstream_table.pk_indices(), current_pos); + // if range_bounds.is_none() { + // yield None; + // return Ok(()); + // } + // let range_bounds = range_bounds.unwrap(); + // let iter = upstream_table + // .iter_all_with_pk_range(&range_bounds, Default::default()) + // .await?; + // pin_mut!(iter); + // + // #[for_await] + // for chunk in iter_chunks(iter, &schema, CHUNK_SIZE) { + // yield chunk?; + // } + yield None; + } + #[try_stream(ok = Option, error = StreamExecutorError)] async fn snapshot_read( schema: Arc, From 056a1ed4417f54a24c28e6e723a1a8f97ad19bdc Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Mon, 3 Jul 2023 14:32:06 +0800 Subject: [PATCH 62/90] finish implement snapshot read per vnode --- src/stream/src/common/table/state_table.rs | 2 + .../executor/backfill/arrangement_backfill.rs | 46 +++++++++---------- 2 files changed, 23 insertions(+), 25 deletions(-) diff --git a/src/stream/src/common/table/state_table.rs b/src/stream/src/common/table/state_table.rs index d2d4a57c6425d..1328535e54bbd 100644 --- a/src/stream/src/common/table/state_table.rs +++ b/src/stream/src/common/table/state_table.rs @@ -936,6 +936,7 @@ where .map(get_second)) } + // FIXME: Remove this. async fn iter_all_pk_and_val_with_pk_range( &self, pk_range: &(Bound, Bound), @@ -953,6 +954,7 @@ where Ok(iter) } + // FIXME: Remove this. pub async fn iter_all_with_pk_range( &self, pk_range: &(Bound, Bound), diff --git a/src/stream/src/executor/backfill/arrangement_backfill.rs b/src/stream/src/executor/backfill/arrangement_backfill.rs index eb3a1d1092773..e48fe784f3582 100644 --- a/src/stream/src/executor/backfill/arrangement_backfill.rs +++ b/src/stream/src/executor/backfill/arrangement_backfill.rs @@ -17,7 +17,7 @@ use std::pin::pin; use std::sync::Arc; use either::Either; -use futures::stream::select_with_strategy; +use futures::stream::{select_with_strategy, FuturesUnordered}; use futures::{pin_mut, stream, StreamExt, TryStreamExt}; use futures_async_stream::try_stream; use risingwave_common::array::StreamChunk; @@ -25,6 +25,7 @@ use risingwave_common::catalog::Schema; use risingwave_common::hash::{VirtualNode, VnodeBitmapExt}; use risingwave_common::row::OwnedRow; use risingwave_common::types::Datum; +use risingwave_common::util::select_all; use risingwave_storage::StateStore; use crate::common::table::state_table::ReplicatedStateTable; @@ -439,48 +440,43 @@ where /// 1. Get row iterator / vnode. /// 2. Merge it with futures unordered. /// 3. Change it into a chunk iterator with `iter_chunks`. + /// This means it should fetch a row from each iterator to form a chunk. + /// Within each vnode, rows will be ordered by pk. + /// TODO: What if one of the range bounds yields none?? #[try_stream(ok = Option, error = StreamExecutorError)] async fn snapshot_read_per_vnode<'a>( schema: Arc, upstream_table: &'a ReplicatedStateTable, current_pos_map: &'a CurrentPosMap, ) { + let mut streams = Vec::with_capacity(upstream_table.vnodes().len()); for vnode in upstream_table.vnodes().iter_vnodes() { let current_pos = current_pos_map.get(&vnode); let range_bounds = compute_bounds(upstream_table.pk_indices(), current_pos.cloned()); if range_bounds.is_none() { - yield None; - return Ok(()); + continue; + // TODO: Highlight this change to reviewers. + // yield None; + // return Ok(()); } let range_bounds = range_bounds.unwrap(); - let iter = upstream_table + let vnode_iter = upstream_table .iter_all_with_pk_range(&range_bounds, Default::default()) .await?; - pin_mut!(iter); - - #[for_await] - for chunk in iter_chunks(iter, &schema, CHUNK_SIZE) { - yield chunk?; - } + streams.push(Box::pin(vnode_iter)); } - // let range_bounds = compute_bounds(upstream_table.pk_indices(), current_pos); - // if range_bounds.is_none() { - // yield None; - // return Ok(()); - // } - // let range_bounds = range_bounds.unwrap(); - // let iter = upstream_table - // .iter_all_with_pk_range(&range_bounds, Default::default()) - // .await?; - // pin_mut!(iter); - // - // #[for_await] - // for chunk in iter_chunks(iter, &schema, CHUNK_SIZE) { - // yield chunk?; - // } + let iter = select_all(streams); + #[for_await] + for chunk in iter_chunks(iter, &schema, CHUNK_SIZE) { + yield chunk?; + } + // TODO: Highlight this change to reviewers. + // yield None; yield None; + return Ok(()); } + // FIXME: Remove #[try_stream(ok = Option, error = StreamExecutorError)] async fn snapshot_read( schema: Arc, From 4b95e43295e7f6d9c166a6f89993938ac4efb785 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Mon, 3 Jul 2023 14:37:31 +0800 Subject: [PATCH 63/90] docs --- src/stream/src/executor/backfill/arrangement_backfill.rs | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/stream/src/executor/backfill/arrangement_backfill.rs b/src/stream/src/executor/backfill/arrangement_backfill.rs index e48fe784f3582..726abd89e27a2 100644 --- a/src/stream/src/executor/backfill/arrangement_backfill.rs +++ b/src/stream/src/executor/backfill/arrangement_backfill.rs @@ -465,6 +465,8 @@ where .await?; streams.push(Box::pin(vnode_iter)); } + // TODO: Highlight this change to patrick vs `FuturesUnordered`, since `FuturesUnordered` + // merges `Futures` -> `Stream`, whereas `select_all` merges `Streams` -> `Stream`. let iter = select_all(streams); #[for_await] for chunk in iter_chunks(iter, &schema, CHUNK_SIZE) { From c8936ccfb27eda241e4b09d5807e46abe52b3a7a Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Mon, 3 Jul 2023 14:43:41 +0800 Subject: [PATCH 64/90] replace all snapshot_read with snapshot_read_per_vnode --- .../executor/backfill/arrangement_backfill.rs | 28 ++++++++++++------- 1 file changed, 18 insertions(+), 10 deletions(-) diff --git a/src/stream/src/executor/backfill/arrangement_backfill.rs b/src/stream/src/executor/backfill/arrangement_backfill.rs index 726abd89e27a2..af9adeaa16527 100644 --- a/src/stream/src/executor/backfill/arrangement_backfill.rs +++ b/src/stream/src/executor/backfill/arrangement_backfill.rs @@ -134,6 +134,10 @@ where assert!(!first_barrier.is_newly_added(self.actor_id)); } + // Current position of upstream_table primary key. + // Current position is computed **per vnode**. + let mut current_pos_map: CurrentPosMap = HashMap::new(); + // If the snapshot is empty, we don't need to backfill. // We cannot complete progress now, as we want to persist // finished state to state store first. @@ -143,7 +147,14 @@ where // It is finished, so just assign a value to avoid accessing storage table again. false } else { - let snapshot = Self::snapshot_read(schema.clone(), &upstream_table, None); + // TODO: This should work. + // current_pos (None) -> current_pos_map::get (Should still be none per vnode, + // as we have not persisted the state yet). + let snapshot = Self::snapshot_read_per_vnode( + schema.clone(), + &upstream_table, + &mut current_pos_map, + ); pin_mut!(snapshot); snapshot.try_next().await?.unwrap().is_none() } @@ -156,10 +167,6 @@ where // | f | f | -> | t | let to_backfill = !is_finished && !is_snapshot_empty; - // Current position of upstream_table primary key. - // Current position is computed **per vnode**. - let mut current_pos_map: CurrentPosMap = HashMap::new(); - // Current position of the upstream_table primary key. // `None` means it starts from the beginning. let mut current_pos: Option = None; @@ -297,11 +304,12 @@ where let left_upstream = upstream.by_ref().map(Either::Left); - let right_snapshot = - pin!( - Self::snapshot_read(schema.clone(), &upstream_table, current_pos.clone(),) - .map(Either::Right), - ); + let right_snapshot = pin!(Self::snapshot_read_per_vnode( + schema.clone(), + &upstream_table, + &mut current_pos_map + ) + .map(Either::Right),); // Prefer to select upstream, so we can stop snapshot stream as soon as the barrier // comes. From 042ce2d80c8947edb040c7b8315456b79949e94b Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Mon, 3 Jul 2023 16:59:56 +0800 Subject: [PATCH 65/90] refactor barrier processing --- .../executor/backfill/arrangement_backfill.rs | 168 +++++++++--------- 1 file changed, 87 insertions(+), 81 deletions(-) diff --git a/src/stream/src/executor/backfill/arrangement_backfill.rs b/src/stream/src/executor/backfill/arrangement_backfill.rs index af9adeaa16527..a319c9927d718 100644 --- a/src/stream/src/executor/backfill/arrangement_backfill.rs +++ b/src/stream/src/executor/backfill/arrangement_backfill.rs @@ -230,6 +230,93 @@ where let mut cur_barrier_snapshot_processed_rows: u64 = 0; let mut cur_barrier_upstream_processed_rows: u64 = 0; + // NOTE(kwannoel): Scope it so that immutable reference to `upstream_table` can be dropped. + // Then we can write to `upstream_table` on barrier in the next block. + { + let left_upstream = upstream.by_ref().map(Either::Left); + + let right_snapshot = pin!(Self::snapshot_read_per_vnode( + schema.clone(), + &upstream_table, + &mut current_pos_map + ) + .map(Either::Right),); + + // Prefer to select upstream, so we can stop snapshot stream as soon as the barrier + // comes. + let backfill_stream = + select_with_strategy(left_upstream, right_snapshot, |_: &mut ()| { + stream::PollNext::Left + }); + + #[for_await] + for either in backfill_stream { + match either { + // Upstream + Either::Left(msg) => { + match msg? { + Message::Barrier(barrier) => { + // We have to process the barrier outside of the loop. + // This is because our state_table reference is still live + // here, we have to break the loop to drop it, + // so we can do replication of upstream state_table. + pending_barrier = Some(barrier); + + // Break the for loop and start a new snapshot read stream. + break; + } + Message::Chunk(chunk) => { + // Buffer the upstream chunk. + upstream_chunk_buffer.push(chunk.compact()); + } + Message::Watermark(_) => { + // Ignore watermark during backfill. + } + } + } + // Snapshot read + Either::Right(msg) => { + match msg? { + None => { + // End of the snapshot read stream. + // We should not mark the chunk anymore, + // otherwise, we will ignore some rows + // in the buffer. Here we choose to never mark the chunk. + // Consume with the renaming stream buffer chunk without + // mark. + for chunk in upstream_chunk_buffer.drain(..) { + let chunk_cardinality = chunk.cardinality() as u64; + cur_barrier_snapshot_processed_rows += + chunk_cardinality; + total_snapshot_processed_rows += chunk_cardinality; + yield Message::Chunk(mapping_chunk( + chunk, + &self.output_indices, + )); + } + + break 'backfill_loop; + } + Some(chunk) => { + // Raise the current position. + // As snapshot read streams are ordered by pk, so we can + // just use the last row to update `current_pos`. + current_pos = update_pos(&chunk, &pk_in_output_indices); + + let chunk_cardinality = chunk.cardinality() as u64; + cur_barrier_snapshot_processed_rows += chunk_cardinality; + total_snapshot_processed_rows += chunk_cardinality; + yield Message::Chunk(mapping_chunk( + chunk, + &self.output_indices, + )); + } + } + } + } + } + } + // Process barrier // Each time we break out of the backfill_stream loop, // it is typically due to barrier (except the first time). @@ -301,87 +388,6 @@ where yield Message::Barrier(barrier); } - - let left_upstream = upstream.by_ref().map(Either::Left); - - let right_snapshot = pin!(Self::snapshot_read_per_vnode( - schema.clone(), - &upstream_table, - &mut current_pos_map - ) - .map(Either::Right),); - - // Prefer to select upstream, so we can stop snapshot stream as soon as the barrier - // comes. - let backfill_stream = - select_with_strategy(left_upstream, right_snapshot, |_: &mut ()| { - stream::PollNext::Left - }); - - #[for_await] - for either in backfill_stream { - match either { - // Upstream - Either::Left(msg) => { - match msg? { - Message::Barrier(barrier) => { - // We have to process the barrier outside of the loop. - // This is because our state_table reference is still live - // here, we have to break the loop to drop it, - // so we can do replication of upstream state_table. - pending_barrier = Some(barrier); - - // Break the for loop and start a new snapshot read stream. - break; - } - Message::Chunk(chunk) => { - // Buffer the upstream chunk. - upstream_chunk_buffer.push(chunk.compact()); - } - Message::Watermark(_) => { - // Ignore watermark during backfill. - } - } - } - // Snapshot read - Either::Right(msg) => { - match msg? { - None => { - // End of the snapshot read stream. - // We should not mark the chunk anymore, - // otherwise, we will ignore some rows - // in the buffer. Here we choose to never mark the chunk. - // Consume with the renaming stream buffer chunk without mark. - for chunk in upstream_chunk_buffer.drain(..) { - let chunk_cardinality = chunk.cardinality() as u64; - cur_barrier_snapshot_processed_rows += chunk_cardinality; - total_snapshot_processed_rows += chunk_cardinality; - yield Message::Chunk(mapping_chunk( - chunk, - &self.output_indices, - )); - } - - break 'backfill_loop; - } - Some(chunk) => { - // Raise the current position. - // As snapshot read streams are ordered by pk, so we can - // just use the last row to update `current_pos`. - current_pos = update_pos(&chunk, &pk_in_output_indices); - - let chunk_cardinality = chunk.cardinality() as u64; - cur_barrier_snapshot_processed_rows += chunk_cardinality; - total_snapshot_processed_rows += chunk_cardinality; - yield Message::Chunk(mapping_chunk( - chunk, - &self.output_indices, - )); - } - } - } - } - } } } From dbd44919b5ad370035d9981ccfecf487ddf55ef6 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Mon, 3 Jul 2023 18:03:19 +0800 Subject: [PATCH 66/90] fix some warnings --- .../src/executor/backfill/arrangement_backfill.rs | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/src/stream/src/executor/backfill/arrangement_backfill.rs b/src/stream/src/executor/backfill/arrangement_backfill.rs index a319c9927d718..e00b306a1aa31 100644 --- a/src/stream/src/executor/backfill/arrangement_backfill.rs +++ b/src/stream/src/executor/backfill/arrangement_backfill.rs @@ -153,7 +153,7 @@ where let snapshot = Self::snapshot_read_per_vnode( schema.clone(), &upstream_table, - &mut current_pos_map, + ¤t_pos_map, ); pin_mut!(snapshot); snapshot.try_next().await?.unwrap().is_none() @@ -230,20 +230,21 @@ where let mut cur_barrier_snapshot_processed_rows: u64 = 0; let mut cur_barrier_upstream_processed_rows: u64 = 0; - // NOTE(kwannoel): Scope it so that immutable reference to `upstream_table` can be dropped. - // Then we can write to `upstream_table` on barrier in the next block. + // NOTE(kwannoel): Scope it so that immutable reference to `upstream_table` can be + // dropped. Then we can write to `upstream_table` on barrier in the + // next block. { let left_upstream = upstream.by_ref().map(Either::Left); let right_snapshot = pin!(Self::snapshot_read_per_vnode( schema.clone(), &upstream_table, - &mut current_pos_map + ¤t_pos_map ) .map(Either::Right),); - // Prefer to select upstream, so we can stop snapshot stream as soon as the barrier - // comes. + // Prefer to select upstream, so we can stop snapshot stream as soon as the + // barrier comes. let backfill_stream = select_with_strategy(left_upstream, right_snapshot, |_: &mut ()| { stream::PollNext::Left From 39b7cbb81349b718a957473a7a9a8bec51fe2c3a Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Mon, 3 Jul 2023 18:38:23 +0800 Subject: [PATCH 67/90] implement compute vnode --- src/stream/src/executor/backfill/arrangement_backfill.rs | 2 -- src/stream/src/executor/backfill/utils.rs | 3 ++- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/src/stream/src/executor/backfill/arrangement_backfill.rs b/src/stream/src/executor/backfill/arrangement_backfill.rs index e00b306a1aa31..b5eb0eff1791f 100644 --- a/src/stream/src/executor/backfill/arrangement_backfill.rs +++ b/src/stream/src/executor/backfill/arrangement_backfill.rs @@ -115,7 +115,6 @@ where .map(|&i| self.output_indices.iter().position(|&j| i == j)) .collect::>>() .unwrap(); - // FIXME: This is wrong, it should use pk. let state_len = pk_in_output_indices.len() + 2; // +1 for backfill_finished, +1 for vnode key. let pk_order = self.upstream_table.pk_serde().get_order_types().to_vec(); let upstream_table_id = self.upstream_table.table_id(); @@ -457,7 +456,6 @@ where /// 3. Change it into a chunk iterator with `iter_chunks`. /// This means it should fetch a row from each iterator to form a chunk. /// Within each vnode, rows will be ordered by pk. - /// TODO: What if one of the range bounds yields none?? #[try_stream(ok = Option, error = StreamExecutorError)] async fn snapshot_read_per_vnode<'a>( schema: Arc, diff --git a/src/stream/src/executor/backfill/utils.rs b/src/stream/src/executor/backfill/utils.rs index 473e68e2314da..1a3b1abb9eb5a 100644 --- a/src/stream/src/executor/backfill/utils.rs +++ b/src/stream/src/executor/backfill/utils.rs @@ -62,7 +62,8 @@ pub(crate) fn mark_chunk_ref_by_vnode( let mut new_visibility = BitmapBuilder::with_capacity(ops.len()); // Use project to avoid allocation. for v in data.rows().map(|row| { - let vnode = VirtualNode::ZERO; // FIXME: compute vnode from row + state table. + // TODO: Ask for review of this. + let vnode = VirtualNode::compute_row(row, pk_in_output_indices); let current_pos = current_pos_map.get(&vnode).unwrap(); let lhs = row.project(pk_in_output_indices); let rhs = current_pos.project(pk_in_output_indices); From 352d8dd0abb2a5a9f8cbad3e1f782a9472484257 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Tue, 4 Jul 2023 10:01:50 +0800 Subject: [PATCH 68/90] risedev cf --- src/stream/src/executor/backfill/arrangement_backfill.rs | 8 ++++---- src/stream/src/executor/backfill/utils.rs | 2 +- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/src/stream/src/executor/backfill/arrangement_backfill.rs b/src/stream/src/executor/backfill/arrangement_backfill.rs index b5eb0eff1791f..b619e6bf0cc2a 100644 --- a/src/stream/src/executor/backfill/arrangement_backfill.rs +++ b/src/stream/src/executor/backfill/arrangement_backfill.rs @@ -17,7 +17,7 @@ use std::pin::pin; use std::sync::Arc; use either::Either; -use futures::stream::{select_with_strategy, FuturesUnordered}; +use futures::stream::{select_with_strategy}; use futures::{pin_mut, stream, StreamExt, TryStreamExt}; use futures_async_stream::try_stream; use risingwave_common::array::StreamChunk; @@ -31,7 +31,7 @@ use risingwave_storage::StateStore; use crate::common::table::state_table::ReplicatedStateTable; use crate::executor::backfill::utils::{ check_all_vnode_finished, compute_bounds, construct_initial_finished_state, iter_chunks, - mapping_chunk, mapping_message, mark_chunk_ref, mark_chunk_ref_by_vnode, persist_state, + mapping_chunk, mapping_message, mark_chunk_ref_by_vnode, persist_state, update_pos, }; use crate::executor::monitor::StreamingMetrics; @@ -135,7 +135,7 @@ where // Current position of upstream_table primary key. // Current position is computed **per vnode**. - let mut current_pos_map: CurrentPosMap = HashMap::new(); + let current_pos_map: CurrentPosMap = HashMap::new(); // If the snapshot is empty, we don't need to backfill. // We cannot complete progress now, as we want to persist @@ -328,7 +328,7 @@ where // Flush downstream. // If no current_pos, means no snapshot processed yet. // Also means we don't need propagate any updates <= current_pos. - if let Some(current_pos) = ¤t_pos { + if let Some(_current_pos) = ¤t_pos { yield Message::Chunk(mapping_chunk( mark_chunk_ref_by_vnode( &chunk, diff --git a/src/stream/src/executor/backfill/utils.rs b/src/stream/src/executor/backfill/utils.rs index 1a3b1abb9eb5a..bd76eafaf5e91 100644 --- a/src/stream/src/executor/backfill/utils.rs +++ b/src/stream/src/executor/backfill/utils.rs @@ -14,7 +14,7 @@ use std::cmp::Ordering; use std::collections::HashMap; -use std::hash::Hash; + use std::ops::Bound; use await_tree::InstrumentAwait; From 687b719a891bde815349e636b67398eca8015c6f Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Tue, 4 Jul 2023 12:32:59 +0800 Subject: [PATCH 69/90] clean --- .../executor/backfill/arrangement_backfill.rs | 30 ++----------------- src/stream/src/executor/backfill/utils.rs | 14 --------- 2 files changed, 2 insertions(+), 42 deletions(-) diff --git a/src/stream/src/executor/backfill/arrangement_backfill.rs b/src/stream/src/executor/backfill/arrangement_backfill.rs index b619e6bf0cc2a..fa4abd160fffa 100644 --- a/src/stream/src/executor/backfill/arrangement_backfill.rs +++ b/src/stream/src/executor/backfill/arrangement_backfill.rs @@ -17,7 +17,7 @@ use std::pin::pin; use std::sync::Arc; use either::Either; -use futures::stream::{select_with_strategy}; +use futures::stream::select_with_strategy; use futures::{pin_mut, stream, StreamExt, TryStreamExt}; use futures_async_stream::try_stream; use risingwave_common::array::StreamChunk; @@ -31,8 +31,7 @@ use risingwave_storage::StateStore; use crate::common::table::state_table::ReplicatedStateTable; use crate::executor::backfill::utils::{ check_all_vnode_finished, compute_bounds, construct_initial_finished_state, iter_chunks, - mapping_chunk, mapping_message, mark_chunk_ref_by_vnode, persist_state, - update_pos, + mapping_chunk, mapping_message, mark_chunk_ref_by_vnode, persist_state, update_pos, }; use crate::executor::monitor::StreamingMetrics; use crate::executor::{ @@ -490,31 +489,6 @@ where yield None; return Ok(()); } - - // FIXME: Remove - #[try_stream(ok = Option, error = StreamExecutorError)] - async fn snapshot_read( - schema: Arc, - upstream_table: &ReplicatedStateTable, - current_pos: Option, - ) { - // FIXME(kwannoel): `let-else` pattern does not work in generator. - let range_bounds = compute_bounds(upstream_table.pk_indices(), current_pos); - if range_bounds.is_none() { - yield None; - return Ok(()); - } - let range_bounds = range_bounds.unwrap(); - let iter = upstream_table - .iter_all_with_pk_range(&range_bounds, Default::default()) - .await?; - pin_mut!(iter); - - #[for_await] - for chunk in iter_chunks(iter, &schema, CHUNK_SIZE) { - yield chunk?; - } - } } impl Executor for ArrangementBackfillExecutor diff --git a/src/stream/src/executor/backfill/utils.rs b/src/stream/src/executor/backfill/utils.rs index bd76eafaf5e91..111507390d96e 100644 --- a/src/stream/src/executor/backfill/utils.rs +++ b/src/stream/src/executor/backfill/utils.rs @@ -14,7 +14,6 @@ use std::cmp::Ordering; use std::collections::HashMap; - use std::ops::Bound; use await_tree::InstrumentAwait; @@ -79,19 +78,6 @@ pub(crate) fn mark_chunk_ref_by_vnode( StreamChunk::new(ops, columns, Some(new_visibility.finish())) } -/// Mark chunk: -/// For each row of the chunk, forward it to downstream if its pk <= `current_pos`, otherwise -/// ignore it. We implement it by changing the visibility bitmap. -pub(crate) fn mark_chunk_ref( - chunk: &StreamChunk, - current_pos: &OwnedRow, - pk_in_output_indices: PkIndicesRef<'_>, - pk_order: &[OrderType], -) -> StreamChunk { - let chunk = chunk.clone(); - mark_chunk_inner(chunk, current_pos, pk_in_output_indices, pk_order) -} - /// Mark chunk: /// For each row of the chunk, forward it to downstream if its pk <= `current_pos`, otherwise /// ignore it. We implement it by changing the visibility bitmap. From 75cd4fa013c01f1cdfab97bce68564a7f053b8d0 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Tue, 4 Jul 2023 15:55:24 +0800 Subject: [PATCH 70/90] clean --- src/stream/src/common/table/state_table.rs | 31 ------------------- .../executor/backfill/arrangement_backfill.rs | 2 +- 2 files changed, 1 insertion(+), 32 deletions(-) diff --git a/src/stream/src/common/table/state_table.rs b/src/stream/src/common/table/state_table.rs index 1328535e54bbd..12caa05c18d16 100644 --- a/src/stream/src/common/table/state_table.rs +++ b/src/stream/src/common/table/state_table.rs @@ -45,7 +45,6 @@ use risingwave_storage::row_serde::row_serde_util::{ use risingwave_storage::store::{ LocalStateStore, NewLocalOptions, PrefetchOptions, ReadOptions, StateStoreIterItemStream, }; -use risingwave_storage::table::merge_sort::merge_sort; use risingwave_storage::table::{compute_chunk_vnode, compute_vnode, get_second, Distribution}; use risingwave_storage::StateStore; use tracing::{trace, Instrument}; @@ -936,36 +935,6 @@ where .map(get_second)) } - // FIXME: Remove this. - async fn iter_all_pk_and_val_with_pk_range( - &self, - pk_range: &(Bound, Bound), - prefetch_options: PrefetchOptions, - ) -> StreamExecutorResult> { - let mut vec = Vec::with_capacity(self.vnodes.count_ones()); - for vnode in self.vnodes.iter_vnodes() { - vec.push( - self.iter_key_and_val_with_pk_range(pk_range, vnode, prefetch_options) - .await?, - ) - } - let pinned_iter: Vec<_> = vec.into_iter().map(Box::pin).collect_vec(); - let iter = merge_sort(pinned_iter); - Ok(iter) - } - - // FIXME: Remove this. - pub async fn iter_all_with_pk_range( - &self, - pk_range: &(Bound, Bound), - prefetch_options: PrefetchOptions, - ) -> StreamExecutorResult> { - Ok(self - .iter_all_pk_and_val_with_pk_range(pk_range, prefetch_options) - .await? - .map(get_second)) - } - pub async fn iter_key_and_val_with_pk_range( &self, pk_range: &(Bound, Bound), diff --git a/src/stream/src/executor/backfill/arrangement_backfill.rs b/src/stream/src/executor/backfill/arrangement_backfill.rs index fa4abd160fffa..9327b2745ea6f 100644 --- a/src/stream/src/executor/backfill/arrangement_backfill.rs +++ b/src/stream/src/executor/backfill/arrangement_backfill.rs @@ -473,7 +473,7 @@ where } let range_bounds = range_bounds.unwrap(); let vnode_iter = upstream_table - .iter_all_with_pk_range(&range_bounds, Default::default()) + .iter_with_pk_range(&range_bounds, vnode, Default::default()) .await?; streams.push(Box::pin(vnode_iter)); } From 317c7c3afed6b5413b6848f3cb5abb43372f09b3 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Tue, 4 Jul 2023 16:08:41 +0800 Subject: [PATCH 71/90] enforce barrier should be present --- .../executor/backfill/arrangement_backfill.rs | 137 +++++++++--------- 1 file changed, 70 insertions(+), 67 deletions(-) diff --git a/src/stream/src/executor/backfill/arrangement_backfill.rs b/src/stream/src/executor/backfill/arrangement_backfill.rs index 9327b2745ea6f..50b238497e98a 100644 --- a/src/stream/src/executor/backfill/arrangement_backfill.rs +++ b/src/stream/src/executor/backfill/arrangement_backfill.rs @@ -21,6 +21,7 @@ use futures::stream::select_with_strategy; use futures::{pin_mut, stream, StreamExt, TryStreamExt}; use futures_async_stream::try_stream; use risingwave_common::array::StreamChunk; +use risingwave_common::bail; use risingwave_common::catalog::Schema; use risingwave_common::hash::{VirtualNode, VnodeBitmapExt}; use risingwave_common::row::OwnedRow; @@ -317,76 +318,78 @@ where } // Process barrier - // Each time we break out of the backfill_stream loop, - // it is typically due to barrier (except the first time). - // So we should immediately process barrier if there's one pending. - if let Some(barrier) = pending_barrier.clone() { - let upstream_chunk_buffer_is_empty = upstream_chunk_buffer.is_empty(); - for chunk in upstream_chunk_buffer.drain(..) { - cur_barrier_upstream_processed_rows += chunk.cardinality() as u64; - // Flush downstream. - // If no current_pos, means no snapshot processed yet. - // Also means we don't need propagate any updates <= current_pos. - if let Some(_current_pos) = ¤t_pos { - yield Message::Chunk(mapping_chunk( - mark_chunk_ref_by_vnode( - &chunk, - ¤t_pos_map, - &pk_in_output_indices, - &pk_order, - ), - &self.output_indices, - )); - } - // Replicate - upstream_table.write_chunk(chunk); - } - if upstream_chunk_buffer_is_empty { - upstream_table.commit_no_data_expected(barrier.epoch) - } else { - upstream_table.commit(barrier.epoch).await?; + // When we break out of inner backfill_stream loop, it means we have a barrier. + // If there are no updates and there are no snapshots left, + // we already finished backfill and should have exited the outer backfill loop. + let barrier = match pending_barrier.take() { + Some(barrier) => barrier, + None => bail!("BUG: current_backfill loop exited without a barrier"), + }; + let upstream_chunk_buffer_is_empty = upstream_chunk_buffer.is_empty(); + for chunk in upstream_chunk_buffer.drain(..) { + cur_barrier_upstream_processed_rows += chunk.cardinality() as u64; + // Flush downstream. + // If no current_pos, means no snapshot processed yet. + // Also means we don't need propagate any updates <= current_pos. + if let Some(_current_pos) = ¤t_pos { + yield Message::Chunk(mapping_chunk( + mark_chunk_ref_by_vnode( + &chunk, + ¤t_pos_map, + &pk_in_output_indices, + &pk_order, + ), + &self.output_indices, + )); } + // Replicate + upstream_table.write_chunk(chunk); + } + if upstream_chunk_buffer_is_empty { + upstream_table.commit_no_data_expected(barrier.epoch) + } else { + upstream_table.commit(barrier.epoch).await?; + } - // TODO(kwannoel): use different counters, otherwise both - // backfill + arrangement backfill executors can't co-exist in same cluster. - self.metrics - .backfill_snapshot_read_row_count - .with_label_values(&[ - upstream_table_id.to_string().as_str(), - self.actor_id.to_string().as_str(), - ]) - .inc_by(cur_barrier_snapshot_processed_rows); - - self.metrics - .backfill_upstream_output_row_count - .with_label_values(&[ - upstream_table_id.to_string().as_str(), - self.actor_id.to_string().as_str(), - ]) - .inc_by(cur_barrier_upstream_processed_rows); - - // Update snapshot read epoch. - snapshot_read_epoch = barrier.epoch.prev; - - self.progress.update( - barrier.epoch.curr, - snapshot_read_epoch, - total_snapshot_processed_rows, - ); - - // Persist state on barrier - persist_state( - barrier.epoch, - &mut self.state_table, - false, - ¤t_pos, - &mut old_state, - &mut current_state, - ) - .await?; + // TODO(kwannoel): use different counters, otherwise both + // backfill + arrangement backfill executors can't co-exist in same cluster. + self.metrics + .backfill_snapshot_read_row_count + .with_label_values(&[ + upstream_table_id.to_string().as_str(), + self.actor_id.to_string().as_str(), + ]) + .inc_by(cur_barrier_snapshot_processed_rows); + + self.metrics + .backfill_upstream_output_row_count + .with_label_values(&[ + upstream_table_id.to_string().as_str(), + self.actor_id.to_string().as_str(), + ]) + .inc_by(cur_barrier_upstream_processed_rows); + + // Update snapshot read epoch. + snapshot_read_epoch = barrier.epoch.prev; + + self.progress.update( + barrier.epoch.curr, + snapshot_read_epoch, + total_snapshot_processed_rows, + ); - yield Message::Barrier(barrier); - } + // Persist state on barrier + persist_state( + barrier.epoch, + &mut self.state_table, + false, + ¤t_pos, + &mut old_state, + &mut current_state, + ) + .await?; + + yield Message::Barrier(barrier); } } From 6d6fbfdabcc0f7ae9d784dcecd7a871adbc49b60 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Tue, 4 Jul 2023 16:29:02 +0800 Subject: [PATCH 72/90] handle stats + TODOs --- .../executor/backfill/arrangement_backfill.rs | 13 ++--------- .../src/executor/monitor/streaming_stats.rs | 23 +++++++++++++++++++ 2 files changed, 25 insertions(+), 11 deletions(-) diff --git a/src/stream/src/executor/backfill/arrangement_backfill.rs b/src/stream/src/executor/backfill/arrangement_backfill.rs index 50b238497e98a..3d6faaef3dbc6 100644 --- a/src/stream/src/executor/backfill/arrangement_backfill.rs +++ b/src/stream/src/executor/backfill/arrangement_backfill.rs @@ -351,10 +351,8 @@ where upstream_table.commit(barrier.epoch).await?; } - // TODO(kwannoel): use different counters, otherwise both - // backfill + arrangement backfill executors can't co-exist in same cluster. self.metrics - .backfill_snapshot_read_row_count + .arrangement_backfill_snapshot_read_row_count .with_label_values(&[ upstream_table_id.to_string().as_str(), self.actor_id.to_string().as_str(), @@ -362,7 +360,7 @@ where .inc_by(cur_barrier_snapshot_processed_rows); self.metrics - .backfill_upstream_output_row_count + .arrangement_backfill_upstream_output_row_count .with_label_values(&[ upstream_table_id.to_string().as_str(), self.actor_id.to_string().as_str(), @@ -470,9 +468,6 @@ where let range_bounds = compute_bounds(upstream_table.pk_indices(), current_pos.cloned()); if range_bounds.is_none() { continue; - // TODO: Highlight this change to reviewers. - // yield None; - // return Ok(()); } let range_bounds = range_bounds.unwrap(); let vnode_iter = upstream_table @@ -480,15 +475,11 @@ where .await?; streams.push(Box::pin(vnode_iter)); } - // TODO: Highlight this change to patrick vs `FuturesUnordered`, since `FuturesUnordered` - // merges `Futures` -> `Stream`, whereas `select_all` merges `Streams` -> `Stream`. let iter = select_all(streams); #[for_await] for chunk in iter_chunks(iter, &schema, CHUNK_SIZE) { yield chunk?; } - // TODO: Highlight this change to reviewers. - // yield None; yield None; return Ok(()); } diff --git a/src/stream/src/executor/monitor/streaming_stats.rs b/src/stream/src/executor/monitor/streaming_stats.rs index 50a1744914e75..9f009988f20ab 100644 --- a/src/stream/src/executor/monitor/streaming_stats.rs +++ b/src/stream/src/executor/monitor/streaming_stats.rs @@ -90,6 +90,10 @@ pub struct StreamingMetrics { pub backfill_snapshot_read_row_count: GenericCounterVec, pub backfill_upstream_output_row_count: GenericCounterVec, + // Arrangement Backfill + pub arrangement_backfill_snapshot_read_row_count: GenericCounterVec, + pub arrangement_backfill_upstream_output_row_count: GenericCounterVec, + /// The duration from receipt of barrier to all actors collection. /// And the max of all node `barrier_inflight_latency` is the latency for a barrier /// to flow through the graph. @@ -540,6 +544,23 @@ impl StreamingMetrics { ) .unwrap(); + let arrangement_backfill_snapshot_read_row_count = register_int_counter_vec_with_registry!( + "stream_arrangement_backfill_snapshot_read_row_count", + "Total number of rows that have been read from the arrangement_backfill snapshot", + &["table_id", "actor_id"], + registry + ) + .unwrap(); + + let arrangement_backfill_upstream_output_row_count = + register_int_counter_vec_with_registry!( + "stream_arrangement_backfill_upstream_output_row_count", + "Total number of rows that have been output from the arrangement_backfill upstream", + &["table_id", "actor_id"], + registry + ) + .unwrap(); + let opts = histogram_opts!( "stream_barrier_inflight_duration_seconds", "barrier_inflight_latency", @@ -697,6 +718,8 @@ impl StreamingMetrics { temporal_join_cached_entry_count, backfill_snapshot_read_row_count, backfill_upstream_output_row_count, + arrangement_backfill_snapshot_read_row_count, + arrangement_backfill_upstream_output_row_count, barrier_inflight_latency, barrier_sync_latency, sink_commit_duration, From 60b6a07f6ba2bda0762692e3d5b66192ef406b6d Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Tue, 4 Jul 2023 16:41:12 +0800 Subject: [PATCH 73/90] docs --- src/stream/src/executor/backfill/arrangement_backfill.rs | 2 +- src/stream/src/executor/backfill/utils.rs | 3 ++- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/src/stream/src/executor/backfill/arrangement_backfill.rs b/src/stream/src/executor/backfill/arrangement_backfill.rs index 3d6faaef3dbc6..ae6bcdcab03ea 100644 --- a/src/stream/src/executor/backfill/arrangement_backfill.rs +++ b/src/stream/src/executor/backfill/arrangement_backfill.rs @@ -452,7 +452,7 @@ where /// Read snapshot per vnode. /// These streams should be sorted in storage layer. /// 1. Get row iterator / vnode. - /// 2. Merge it with futures unordered. + /// 2. Merge it with `select_all`. /// 3. Change it into a chunk iterator with `iter_chunks`. /// This means it should fetch a row from each iterator to form a chunk. /// Within each vnode, rows will be ordered by pk. diff --git a/src/stream/src/executor/backfill/utils.rs b/src/stream/src/executor/backfill/utils.rs index 111507390d96e..a83c137ad09e0 100644 --- a/src/stream/src/executor/backfill/utils.rs +++ b/src/stream/src/executor/backfill/utils.rs @@ -61,7 +61,8 @@ pub(crate) fn mark_chunk_ref_by_vnode( let mut new_visibility = BitmapBuilder::with_capacity(ops.len()); // Use project to avoid allocation. for v in data.rows().map(|row| { - // TODO: Ask for review of this. + // TODO(kwannoel): Is this logic correct for compute vnode? + // I will revisit it again when arrangement_backfill is implemented e2e. let vnode = VirtualNode::compute_row(row, pk_in_output_indices); let current_pos = current_pos_map.get(&vnode).unwrap(); let lhs = row.project(pk_in_output_indices); From ba3a7e6a5d8cd1581dd59acf86a4b8f685b0d843 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Tue, 4 Jul 2023 16:57:24 +0800 Subject: [PATCH 74/90] remove TODO --- src/stream/src/executor/backfill/arrangement_backfill.rs | 3 --- 1 file changed, 3 deletions(-) diff --git a/src/stream/src/executor/backfill/arrangement_backfill.rs b/src/stream/src/executor/backfill/arrangement_backfill.rs index ae6bcdcab03ea..3db6228133570 100644 --- a/src/stream/src/executor/backfill/arrangement_backfill.rs +++ b/src/stream/src/executor/backfill/arrangement_backfill.rs @@ -146,9 +146,6 @@ where // It is finished, so just assign a value to avoid accessing storage table again. false } else { - // TODO: This should work. - // current_pos (None) -> current_pos_map::get (Should still be none per vnode, - // as we have not persisted the state yet). let snapshot = Self::snapshot_read_per_vnode( schema.clone(), &upstream_table, From 694b11e460a75686125d209ad74d5b97e4af51b6 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Tue, 4 Jul 2023 17:37:56 +0800 Subject: [PATCH 75/90] add function to fetch progress per vnode --- src/stream/src/executor/backfill/utils.rs | 42 +++++++++++++++++++++-- 1 file changed, 39 insertions(+), 3 deletions(-) diff --git a/src/stream/src/executor/backfill/utils.rs b/src/stream/src/executor/backfill/utils.rs index a83c137ad09e0..88846e11989db 100644 --- a/src/stream/src/executor/backfill/utils.rs +++ b/src/stream/src/executor/backfill/utils.rs @@ -127,10 +127,46 @@ pub(crate) fn mapping_message(msg: Message, upstream_indices: &[usize]) -> Optio } } +/// Used for tracking backfill state per vnode +#[derive(Eq, PartialEq, Debug)] +pub enum BackfillProgress { + NotStarted, + InProgress(OwnedRow), + Completed, +} + +/// Gets progress per vnode, so we know which to backfill. +pub(crate) async fn get_progress_per_vnode( + state_table: &StateTableInner, + state_len: usize, +) -> StreamExecutorResult> { + debug_assert!(!state_table.vnode_bitmap().is_empty()); + let vnodes = state_table.vnodes().iter_vnodes_scalar(); + let mut result = Vec::with_capacity(state_table.vnodes().len()); + for vnode in vnodes { + let vnode_key: &[Datum] = &[Some(vnode.into())]; + let state_for_vnode_key = state_table.get_row(vnode_key).await?; + + // original_backfill_datum_pos = (state_len - 1) + // value indices are set, so we can -1 for the pk (a single vnode). + let backfill_datum_pos = state_len - 2; + let backfill_progress = match state_for_vnode_key { + Some(row) => { + let vnode_is_finished = row.datum_at(backfill_datum_pos).unwrap(); + if vnode_is_finished.into_bool() { + BackfillProgress::Completed + } else { + BackfillProgress::InProgress(row) + } + } + None => BackfillProgress::NotStarted, + }; + result.push(backfill_progress); + } + Ok(result) +} + /// All vnodes should be persisted with status finished. -/// TODO: In the future we will support partial backfill recovery. -/// When that is done, this logic may need to be rewritten to handle -/// partially complete states per vnode. pub(crate) async fn check_all_vnode_finished( state_table: &StateTableInner, state_len: usize, From d5a4dd3e6fa2fc2d1932a161cf7d2e5a68fa898a Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Tue, 4 Jul 2023 19:05:30 +0800 Subject: [PATCH 76/90] start refactoring state persistence to be per vnode * Add backfill state struct * Add methods for persisting state per vnode, fyi some of these are incomplete --- .../executor/backfill/arrangement_backfill.rs | 4 +- src/stream/src/executor/backfill/utils.rs | 61 +++++++++++++++---- 2 files changed, 50 insertions(+), 15 deletions(-) diff --git a/src/stream/src/executor/backfill/arrangement_backfill.rs b/src/stream/src/executor/backfill/arrangement_backfill.rs index 3db6228133570..c2db81ee0e93f 100644 --- a/src/stream/src/executor/backfill/arrangement_backfill.rs +++ b/src/stream/src/executor/backfill/arrangement_backfill.rs @@ -33,6 +33,7 @@ use crate::common::table::state_table::ReplicatedStateTable; use crate::executor::backfill::utils::{ check_all_vnode_finished, compute_bounds, construct_initial_finished_state, iter_chunks, mapping_chunk, mapping_message, mark_chunk_ref_by_vnode, persist_state, update_pos, + CurrentPosMap, }; use crate::executor::monitor::StreamingMetrics; use crate::executor::{ @@ -41,8 +42,6 @@ use crate::executor::{ }; use crate::task::{ActorId, CreateMviewProgress}; -type CurrentPosMap = HashMap; - /// Similar to [`BackfillExecutor`]. /// Main differences: /// - [`ArrangementBackfillExecutor`] can reside on a different CN, so it can be scaled @@ -165,6 +164,7 @@ where // Current position of the upstream_table primary key. // `None` means it starts from the beginning. + // FIXME: Remove it let mut current_pos: Option = None; // Use these to persist state. diff --git a/src/stream/src/executor/backfill/utils.rs b/src/stream/src/executor/backfill/utils.rs index 88846e11989db..e4588a6bf9884 100644 --- a/src/stream/src/executor/backfill/utils.rs +++ b/src/stream/src/executor/backfill/utils.rs @@ -37,6 +37,24 @@ use crate::executor::{ Message, PkIndicesRef, StreamExecutorError, StreamExecutorResult, Watermark, }; +pub type CurrentPosMap = HashMap; + +pub struct BackfillState { + /// Used to track backfill progress. + backfill_progress: HashMap, + + /// We need this to process state updates. + committed_progress: HashMap>, +} + +/// Used for tracking backfill state per vnode +#[derive(Eq, PartialEq, Debug)] +pub enum BackfillProgressPerVnode { + NotStarted, + InProgress(OwnedRow), + Completed, +} + pub(crate) fn mark_chunk( chunk: StreamChunk, current_pos: &OwnedRow, @@ -49,7 +67,8 @@ pub(crate) fn mark_chunk( /// Mark chunk: /// For each row of the chunk, forward it to downstream if its pk <= `current_pos` for the -/// corresponding vnode, otherwise ignore it. We implement it by changing the visibility bitmap. +/// corresponding `vnode`, otherwise ignore it. +/// We implement it by changing the visibility bitmap. pub(crate) fn mark_chunk_ref_by_vnode( chunk: &StreamChunk, current_pos_map: &HashMap, @@ -127,19 +146,11 @@ pub(crate) fn mapping_message(msg: Message, upstream_indices: &[usize]) -> Optio } } -/// Used for tracking backfill state per vnode -#[derive(Eq, PartialEq, Debug)] -pub enum BackfillProgress { - NotStarted, - InProgress(OwnedRow), - Completed, -} - /// Gets progress per vnode, so we know which to backfill. pub(crate) async fn get_progress_per_vnode( state_table: &StateTableInner, state_len: usize, -) -> StreamExecutorResult> { +) -> StreamExecutorResult> { debug_assert!(!state_table.vnode_bitmap().is_empty()); let vnodes = state_table.vnodes().iter_vnodes_scalar(); let mut result = Vec::with_capacity(state_table.vnodes().len()); @@ -154,12 +165,12 @@ pub(crate) async fn get_progress_per_vnode { let vnode_is_finished = row.datum_at(backfill_datum_pos).unwrap(); if vnode_is_finished.into_bool() { - BackfillProgress::Completed + BackfillProgressPerVnode::Completed } else { - BackfillProgress::InProgress(row) + BackfillProgressPerVnode::InProgress(row) } } - None => BackfillProgress::NotStarted, + None => BackfillProgressPerVnode::NotStarted, }; result.push(backfill_progress); } @@ -309,6 +320,30 @@ pub(crate) async fn iter_chunks<'a, S, E>( yield None; } +/// Schema +/// | vnode | pk | `backfill_finished` | +/// +/// `current_pos_map` is the map from vnode to the current backfilled pk position. +pub(crate) async fn persist_state_per_vnode( + epoch: EpochPair, + table: &mut StateTableInner, + is_finished: bool, + current_pos_map: &CurrentPosMap, + old_state: &mut Option>, + current_state: &mut [Datum], +) -> StreamExecutorResult<()> { + if current_pos_map.is_empty() { + table.commit_no_data_expected(epoch); + } + for current_pos in current_pos_map.values() { + // state w/o vnodes. + build_temporary_state(current_state, is_finished, current_pos); + flush_data(table, epoch, old_state, current_state).await?; + *old_state = Some(current_state.into()); + } + Ok(()) +} + /// Schema /// | vnode | pk | `backfill_finished` | /// From bc397537bc670e9924f31e50d73cb337751b51b8 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Tue, 4 Jul 2023 19:18:31 +0800 Subject: [PATCH 77/90] add update_pos_per_vnode interface + interleave by chunk in snapshot read --- .../executor/backfill/arrangement_backfill.rs | 16 +++++++++++----- src/stream/src/executor/backfill/utils.rs | 16 ++++++++++++++++ 2 files changed, 27 insertions(+), 5 deletions(-) diff --git a/src/stream/src/executor/backfill/arrangement_backfill.rs b/src/stream/src/executor/backfill/arrangement_backfill.rs index c2db81ee0e93f..2d65b0906ca4e 100644 --- a/src/stream/src/executor/backfill/arrangement_backfill.rs +++ b/src/stream/src/executor/backfill/arrangement_backfill.rs @@ -452,7 +452,10 @@ where /// 2. Merge it with `select_all`. /// 3. Change it into a chunk iterator with `iter_chunks`. /// This means it should fetch a row from each iterator to form a chunk. - /// Within each vnode, rows will be ordered by pk. + /// + /// NOTE(kwannoel): We interleave at chunk per vnode level rather than rows. + /// This is so that we can compute `current_pos` once per chunk, since they correspond to 1 + /// vnode. #[try_stream(ok = Option, error = StreamExecutorError)] async fn snapshot_read_per_vnode<'a>( schema: Arc, @@ -467,14 +470,17 @@ where continue; } let range_bounds = range_bounds.unwrap(); - let vnode_iter = upstream_table + let vnode_row_iter = upstream_table .iter_with_pk_range(&range_bounds, vnode, Default::default()) .await?; - streams.push(Box::pin(vnode_iter)); + // TODO: Is there some way to avoid double-pin here? + let vnode_row_iter = Box::pin(vnode_row_iter); + let vnode_chunk_iter = iter_chunks(vnode_row_iter, &schema, CHUNK_SIZE); + // TODO: Is there some way to avoid double-pin + streams.push(Box::pin(vnode_chunk_iter)); } - let iter = select_all(streams); #[for_await] - for chunk in iter_chunks(iter, &schema, CHUNK_SIZE) { + for chunk in select_all(streams) { yield chunk?; } yield None; diff --git a/src/stream/src/executor/backfill/utils.rs b/src/stream/src/executor/backfill/utils.rs index e4588a6bf9884..b2b37aea26b0a 100644 --- a/src/stream/src/executor/backfill/utils.rs +++ b/src/stream/src/executor/backfill/utils.rs @@ -255,6 +255,21 @@ pub(crate) fn build_temporary_state( row_state[current_pos.len() + 1] = Some(is_finished.into()); } +pub(crate) fn update_pos_per_vnode( + chunk: &StreamChunk, + pk_in_output_indices: &[usize], +) -> Option { + Some( + chunk + .rows() + .last() + .unwrap() + .1 + .project(pk_in_output_indices) + .into_owned_row(), + ) +} + pub(crate) fn update_pos(chunk: &StreamChunk, pk_in_output_indices: &[usize]) -> Option { Some( chunk @@ -339,6 +354,7 @@ pub(crate) async fn persist_state_per_vnode Date: Tue, 4 Jul 2023 19:32:59 +0800 Subject: [PATCH 78/90] finish refactor to produce chunk + vnode --- src/stream/src/executor/backfill/arrangement_backfill.rs | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/src/stream/src/executor/backfill/arrangement_backfill.rs b/src/stream/src/executor/backfill/arrangement_backfill.rs index 2d65b0906ca4e..fc3015d4da80d 100644 --- a/src/stream/src/executor/backfill/arrangement_backfill.rs +++ b/src/stream/src/executor/backfill/arrangement_backfill.rs @@ -294,7 +294,7 @@ where break 'backfill_loop; } - Some(chunk) => { + Some((_, chunk)) => { // Raise the current position. // As snapshot read streams are ordered by pk, so we can // just use the last row to update `current_pos`. @@ -456,7 +456,7 @@ where /// NOTE(kwannoel): We interleave at chunk per vnode level rather than rows. /// This is so that we can compute `current_pos` once per chunk, since they correspond to 1 /// vnode. - #[try_stream(ok = Option, error = StreamExecutorError)] + #[try_stream(ok = Option<(VirtualNode, StreamChunk)>, error = StreamExecutorError)] async fn snapshot_read_per_vnode<'a>( schema: Arc, upstream_table: &'a ReplicatedStateTable, @@ -475,7 +475,8 @@ where .await?; // TODO: Is there some way to avoid double-pin here? let vnode_row_iter = Box::pin(vnode_row_iter); - let vnode_chunk_iter = iter_chunks(vnode_row_iter, &schema, CHUNK_SIZE); + let vnode_chunk_iter = iter_chunks(vnode_row_iter, &schema, CHUNK_SIZE) + .map_ok(move |chunk_opt| chunk_opt.map(|chunk| (vnode, chunk))); // TODO: Is there some way to avoid double-pin streams.push(Box::pin(vnode_chunk_iter)); } From d522787f37c8f9d558c9c55d8b6b3cf6ce2fbf56 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Tue, 4 Jul 2023 19:48:51 +0800 Subject: [PATCH 79/90] workaround current_pos_map reference, by cloning it --- .../executor/backfill/arrangement_backfill.rs | 22 ++++++++----- .../executor/backfill/no_shuffle_backfill.rs | 6 ++-- src/stream/src/executor/backfill/utils.rs | 31 +++++++++++++------ 3 files changed, 39 insertions(+), 20 deletions(-) diff --git a/src/stream/src/executor/backfill/arrangement_backfill.rs b/src/stream/src/executor/backfill/arrangement_backfill.rs index fc3015d4da80d..fc2791f4c8f60 100644 --- a/src/stream/src/executor/backfill/arrangement_backfill.rs +++ b/src/stream/src/executor/backfill/arrangement_backfill.rs @@ -32,7 +32,7 @@ use risingwave_storage::StateStore; use crate::common::table::state_table::ReplicatedStateTable; use crate::executor::backfill::utils::{ check_all_vnode_finished, compute_bounds, construct_initial_finished_state, iter_chunks, - mapping_chunk, mapping_message, mark_chunk_ref_by_vnode, persist_state, update_pos, + mapping_chunk, mapping_message, mark_chunk_ref_by_vnode, persist_state, update_pos_by_vnode, CurrentPosMap, }; use crate::executor::monitor::StreamingMetrics; @@ -134,7 +134,7 @@ where // Current position of upstream_table primary key. // Current position is computed **per vnode**. - let current_pos_map: CurrentPosMap = HashMap::new(); + let mut current_pos_map: CurrentPosMap = HashMap::new(); // If the snapshot is empty, we don't need to backfill. // We cannot complete progress now, as we want to persist @@ -148,7 +148,7 @@ where let snapshot = Self::snapshot_read_per_vnode( schema.clone(), &upstream_table, - ¤t_pos_map, + current_pos_map.clone(), // FIXME: temporary workaround... How to avoid it? ); pin_mut!(snapshot); snapshot.try_next().await?.unwrap().is_none() @@ -235,7 +235,7 @@ where let right_snapshot = pin!(Self::snapshot_read_per_vnode( schema.clone(), &upstream_table, - ¤t_pos_map + current_pos_map.clone(), // FIXME: temporary workaround, how to avoid it? ) .map(Either::Right),); @@ -294,11 +294,17 @@ where break 'backfill_loop; } - Some((_, chunk)) => { + Some((vnode, chunk)) => { // Raise the current position. // As snapshot read streams are ordered by pk, so we can // just use the last row to update `current_pos`. - current_pos = update_pos(&chunk, &pk_in_output_indices); + // current_pos = up(&chunk, &pk_in_output_indices); + update_pos_by_vnode( + vnode, + &chunk, + &pk_in_output_indices, + &mut current_pos_map, + ); let chunk_cardinality = chunk.cardinality() as u64; cur_barrier_snapshot_processed_rows += chunk_cardinality; @@ -325,6 +331,7 @@ where let upstream_chunk_buffer_is_empty = upstream_chunk_buffer.is_empty(); for chunk in upstream_chunk_buffer.drain(..) { cur_barrier_upstream_processed_rows += chunk.cardinality() as u64; + // FIXME: Replace with `snapshot_is_processed` // Flush downstream. // If no current_pos, means no snapshot processed yet. // Also means we don't need propagate any updates <= current_pos. @@ -342,6 +349,7 @@ where // Replicate upstream_table.write_chunk(chunk); } + if upstream_chunk_buffer_is_empty { upstream_table.commit_no_data_expected(barrier.epoch) } else { @@ -460,7 +468,7 @@ where async fn snapshot_read_per_vnode<'a>( schema: Arc, upstream_table: &'a ReplicatedStateTable, - current_pos_map: &'a CurrentPosMap, + current_pos_map: CurrentPosMap, ) { let mut streams = Vec::with_capacity(upstream_table.vnodes().len()); for vnode in upstream_table.vnodes().iter_vnodes() { diff --git a/src/stream/src/executor/backfill/no_shuffle_backfill.rs b/src/stream/src/executor/backfill/no_shuffle_backfill.rs index 9d73445a83904..36b79fa57c62a 100644 --- a/src/stream/src/executor/backfill/no_shuffle_backfill.rs +++ b/src/stream/src/executor/backfill/no_shuffle_backfill.rs @@ -34,8 +34,8 @@ use risingwave_storage::StateStore; use crate::common::table::state_table::StateTable; use crate::executor::backfill::utils; use crate::executor::backfill::utils::{ - check_all_vnode_finished, compute_bounds, construct_initial_finished_state, iter_chunks, - mapping_chunk, mapping_message, mark_chunk, update_pos, + check_all_vnode_finished, compute_bounds, construct_initial_finished_state, get_new_pos, + iter_chunks, mapping_chunk, mapping_message, mark_chunk, }; use crate::executor::monitor::StreamingMetrics; use crate::executor::{ @@ -349,7 +349,7 @@ where // Raise the current position. // As snapshot read streams are ordered by pk, so we can // just use the last row to update `current_pos`. - current_pos = update_pos(&chunk, &pk_in_output_indices); + current_pos = Some(get_new_pos(&chunk, &pk_in_output_indices)); let chunk_cardinality = chunk.cardinality() as u64; cur_barrier_snapshot_processed_rows += chunk_cardinality; diff --git a/src/stream/src/executor/backfill/utils.rs b/src/stream/src/executor/backfill/utils.rs index b2b37aea26b0a..7e384bb989a1f 100644 --- a/src/stream/src/executor/backfill/utils.rs +++ b/src/stream/src/executor/backfill/utils.rs @@ -270,16 +270,27 @@ pub(crate) fn update_pos_per_vnode( ) } -pub(crate) fn update_pos(chunk: &StreamChunk, pk_in_output_indices: &[usize]) -> Option { - Some( - chunk - .rows() - .last() - .unwrap() - .1 - .project(pk_in_output_indices) - .into_owned_row(), - ) +/// Update backfill pos by vnode. +pub(crate) fn update_pos_by_vnode( + vnode: VirtualNode, + chunk: &StreamChunk, + pk_in_output_indices: &[usize], + current_pos_map: &mut CurrentPosMap, +) { + let new_pos = get_new_pos(chunk, pk_in_output_indices); + current_pos_map.insert(vnode, new_pos); +} + +/// Get new backfill pos from the chunk. Since chunk should have ordered rows, we can just take the +/// last row. +pub(crate) fn get_new_pos(chunk: &StreamChunk, pk_in_output_indices: &[usize]) -> OwnedRow { + chunk + .rows() + .last() + .unwrap() + .1 + .project(pk_in_output_indices) + .into_owned_row() } // NOTE(kwannoel): ["None" ..] encoding should be appropriate to mark From da8fffb3d6d6a61f3d4ecf4b86d060aae1a16989 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Tue, 4 Jul 2023 19:49:53 +0800 Subject: [PATCH 80/90] add FIXME --- src/stream/src/executor/backfill/arrangement_backfill.rs | 1 + 1 file changed, 1 insertion(+) diff --git a/src/stream/src/executor/backfill/arrangement_backfill.rs b/src/stream/src/executor/backfill/arrangement_backfill.rs index fc2791f4c8f60..bcc84653f2501 100644 --- a/src/stream/src/executor/backfill/arrangement_backfill.rs +++ b/src/stream/src/executor/backfill/arrangement_backfill.rs @@ -382,6 +382,7 @@ where ); // Persist state on barrier + // FIXME: This should persist state per vnode. persist_state( barrier.epoch, &mut self.state_table, From 5e4a5c2dac5b7ecee59ef7580ae5108a2dece033 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Tue, 4 Jul 2023 22:33:10 +0800 Subject: [PATCH 81/90] update persist_data per vnode --- .../executor/backfill/arrangement_backfill.rs | 9 ++ src/stream/src/executor/backfill/utils.rs | 94 +++++++++++++++---- 2 files changed, 86 insertions(+), 17 deletions(-) diff --git a/src/stream/src/executor/backfill/arrangement_backfill.rs b/src/stream/src/executor/backfill/arrangement_backfill.rs index bcc84653f2501..dbfb93c7dc6f4 100644 --- a/src/stream/src/executor/backfill/arrangement_backfill.rs +++ b/src/stream/src/executor/backfill/arrangement_backfill.rs @@ -134,6 +134,7 @@ where // Current position of upstream_table primary key. // Current position is computed **per vnode**. + // FIXME: This should be backfill state instead. We need to know which vnode is finished. let mut current_pos_map: CurrentPosMap = HashMap::new(); // If the snapshot is empty, we don't need to backfill. @@ -465,6 +466,14 @@ where /// NOTE(kwannoel): We interleave at chunk per vnode level rather than rows. /// This is so that we can compute `current_pos` once per chunk, since they correspond to 1 /// vnode. + /// + /// TODO(kwannoel): Support partially complete snapshot reads. + /// That will require the following changes: + /// 1. Instead of returning stream chunk and vnode, we need to dispatch 3 diff messages: + /// a. COMPLETE_VNODE(vnode): Current iterator is complete, in that case we need to handle it + /// in arrangement backfill. We should not buffer updates for this vnode, and forward + /// all messages. b. MESSAGE(CHUNK): Current iterator is not complete, in that case we + /// need to buffer updates for this vnode. c. FINISHED: All iterators finished. #[try_stream(ok = Option<(VirtualNode, StreamChunk)>, error = StreamExecutorError)] async fn snapshot_read_per_vnode<'a>( schema: Arc, diff --git a/src/stream/src/executor/backfill/utils.rs b/src/stream/src/executor/backfill/utils.rs index 7e384bb989a1f..96f3a8d4d4a7b 100644 --- a/src/stream/src/executor/backfill/utils.rs +++ b/src/stream/src/executor/backfill/utils.rs @@ -42,9 +42,24 @@ pub type CurrentPosMap = HashMap; pub struct BackfillState { /// Used to track backfill progress. backfill_progress: HashMap, +} - /// We need this to process state updates. - committed_progress: HashMap>, +impl BackfillState { + fn new() -> Self { + Self { + backfill_progress: HashMap::new(), + } + } + + fn has_no_progress(&self) -> bool { + self.backfill_progress.is_empty() + } + + fn iter_backfill_progress( + &self, + ) -> impl Iterator { + self.backfill_progress.iter() + } } /// Used for tracking backfill state per vnode @@ -69,6 +84,8 @@ pub(crate) fn mark_chunk( /// For each row of the chunk, forward it to downstream if its pk <= `current_pos` for the /// corresponding `vnode`, otherwise ignore it. /// We implement it by changing the visibility bitmap. +/// +/// TODO(kwannoel): We should always forward rows with status `FINISHED`. pub(crate) fn mark_chunk_ref_by_vnode( chunk: &StreamChunk, current_pos_map: &HashMap, @@ -80,7 +97,7 @@ pub(crate) fn mark_chunk_ref_by_vnode( let mut new_visibility = BitmapBuilder::with_capacity(ops.len()); // Use project to avoid allocation. for v in data.rows().map(|row| { - // TODO(kwannoel): Is this logic correct for compute vnode? + // TODO(kwannoel): Is this logic correct for computing vnode? // I will revisit it again when arrangement_backfill is implemented e2e. let vnode = VirtualNode::compute_row(row, pk_in_output_indices); let current_pos = current_pos_map.get(&vnode).unwrap(); @@ -244,8 +261,21 @@ pub(crate) async fn flush_data( table.commit(epoch).await } -// We want to avoid building a row for every vnode. -// Instead we can just modify a single row, and dispatch it to state table to write. +/// We want to avoid allocating a row for every vnode. +pub(crate) fn build_temporary_state_with_vnode( + row_state: &mut [Datum], + vnode: VirtualNode, + is_finished: bool, + current_pos: &OwnedRow, +) { + build_temporary_state(row_state, is_finished, current_pos); + row_state[0] = Some(vnode.to_scalar().into()); +} + +/// We want to avoid allocating a row for every vnode. +/// Instead we can just modify a single row, and dispatch it to state table to write. +/// This builds the `current_pos` segment of the row. +/// Vnode needs to be filled in as well. pub(crate) fn build_temporary_state( row_state: &mut [Datum], is_finished: bool, @@ -348,25 +378,55 @@ pub(crate) async fn iter_chunks<'a, S, E>( /// Schema /// | vnode | pk | `backfill_finished` | -/// -/// `current_pos_map` is the map from vnode to the current backfilled pk position. +/// Persists the state per vnode. +/// 1. For each (vnode, current_pos), +/// either insert OR update the state, +/// depending if there was an old state. pub(crate) async fn persist_state_per_vnode( epoch: EpochPair, table: &mut StateTableInner, is_finished: bool, - current_pos_map: &CurrentPosMap, - old_state: &mut Option>, - current_state: &mut [Datum], + backfill_state: &mut BackfillState, + committed_progress: &mut HashMap>, + temporary_state: &mut [Datum], ) -> StreamExecutorResult<()> { - if current_pos_map.is_empty() { + // No progress -> No need to commit anything. + if backfill_state.has_no_progress() { table.commit_no_data_expected(epoch); } - for current_pos in current_pos_map.values() { - // state w/o vnodes. - build_temporary_state(current_state, is_finished, current_pos); - flush_data(table, epoch, old_state, current_state).await?; - // FIXME - *old_state = Some(current_state.into()); + + for (vnode, backfill_progress) in backfill_state.iter_backfill_progress() { + let current_pos = match backfill_progress { + BackfillProgressPerVnode::Completed | BackfillProgressPerVnode::NotStarted => { + continue; + } + BackfillProgressPerVnode::InProgress(current_pos) => current_pos, + }; + build_temporary_state_with_vnode(temporary_state, *vnode, is_finished, current_pos); + + let old_state = committed_progress.get(&vnode); + + if let Some(old_state) = old_state { + // No progress for vnode, means no data + if old_state == current_pos.as_inner() { + table.commit_no_data_expected(epoch); + return Ok(()); + } else { + // There's some progress, update the state. + table.write_record(Record::Update { + old_row: &old_state[..], + new_row: &(*temporary_state), + }); + table.commit(epoch).await?; + } + } else { + // No existing state, create a new entry. + table.write_record(Record::Insert { + new_row: &(*temporary_state), + }); + table.commit(epoch).await?; + } + committed_progress.insert(*vnode, current_pos.as_inner().to_vec()); } Ok(()) } From 3558a0d3daa79bd029b6da86b0ba509f235828fd Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Tue, 4 Jul 2023 22:44:25 +0800 Subject: [PATCH 82/90] naming + init backfill state --- .../executor/backfill/arrangement_backfill.rs | 22 ++++++++++------- src/stream/src/executor/backfill/utils.rs | 24 ++++++++++++------- 2 files changed, 30 insertions(+), 16 deletions(-) diff --git a/src/stream/src/executor/backfill/arrangement_backfill.rs b/src/stream/src/executor/backfill/arrangement_backfill.rs index dbfb93c7dc6f4..4af88a60dccb7 100644 --- a/src/stream/src/executor/backfill/arrangement_backfill.rs +++ b/src/stream/src/executor/backfill/arrangement_backfill.rs @@ -31,9 +31,9 @@ use risingwave_storage::StateStore; use crate::common::table::state_table::ReplicatedStateTable; use crate::executor::backfill::utils::{ - check_all_vnode_finished, compute_bounds, construct_initial_finished_state, iter_chunks, - mapping_chunk, mapping_message, mark_chunk_ref_by_vnode, persist_state, update_pos_by_vnode, - CurrentPosMap, + check_all_vnode_finished, compute_bounds, construct_initial_finished_state, + get_progress_per_vnode, iter_chunks, mapping_chunk, mapping_message, mark_chunk_ref_by_vnode, + persist_state, update_pos_by_vnode, BackfillProgressPerVnode, BackfillState, CurrentPosMap, }; use crate::executor::monitor::StreamingMetrics; use crate::executor::{ @@ -127,11 +127,17 @@ where let first_barrier = expect_first_barrier(&mut upstream).await?; self.state_table.init_epoch(first_barrier.epoch); - let is_finished = check_all_vnode_finished(&self.state_table, state_len).await?; - if is_finished { + let progress_per_vnode = get_progress_per_vnode(&self.state_table, state_len).await?; + + let is_completely_finished = progress_per_vnode + .iter() + .all(|(_, p)| *p == BackfillProgressPerVnode::Completed); + if is_completely_finished { assert!(!first_barrier.is_newly_added(self.actor_id)); } + let mut backfill_state: BackfillState = progress_per_vnode.into(); + // Current position of upstream_table primary key. // Current position is computed **per vnode**. // FIXME: This should be backfill state instead. We need to know which vnode is finished. @@ -142,7 +148,7 @@ where // finished state to state store first. // As such we will wait for next barrier. let is_snapshot_empty: bool = { - if is_finished { + if is_completely_finished { // It is finished, so just assign a value to avoid accessing storage table again. false } else { @@ -161,7 +167,7 @@ where // | t | t/f | -> | f | // | f | t | -> | f | // | f | f | -> | t | - let to_backfill = !is_finished && !is_snapshot_empty; + let to_backfill = !is_completely_finished && !is_snapshot_empty; // Current position of the upstream_table primary key. // `None` means it starts from the beginning. @@ -408,7 +414,7 @@ where while let Some(Ok(msg)) = upstream.next().await { if let Some(msg) = mapping_message(msg, &self.output_indices) { // If not finished then we need to update state, otherwise no need. - if let Message::Barrier(barrier) = &msg && !is_finished { + if let Message::Barrier(barrier) = &msg && !is_completely_finished { // If snapshot was empty, we do not need to backfill, // but we still need to persist the finished state. // We currently persist it on the second barrier here rather than first. diff --git a/src/stream/src/executor/backfill/utils.rs b/src/stream/src/executor/backfill/utils.rs index 96f3a8d4d4a7b..8d6cc6d449a8c 100644 --- a/src/stream/src/executor/backfill/utils.rs +++ b/src/stream/src/executor/backfill/utils.rs @@ -41,24 +41,32 @@ pub type CurrentPosMap = HashMap; pub struct BackfillState { /// Used to track backfill progress. - backfill_progress: HashMap, + inner: HashMap, } impl BackfillState { fn new() -> Self { Self { - backfill_progress: HashMap::new(), + inner: HashMap::new(), } } fn has_no_progress(&self) -> bool { - self.backfill_progress.is_empty() + self.inner.is_empty() } fn iter_backfill_progress( &self, ) -> impl Iterator { - self.backfill_progress.iter() + self.inner.iter() + } +} + +impl From> for BackfillState { + fn from(v: Vec<(VirtualNode, BackfillProgressPerVnode)>) -> Self { + Self { + inner: v.into_iter().collect(), + } } } @@ -167,12 +175,12 @@ pub(crate) fn mapping_message(msg: Message, upstream_indices: &[usize]) -> Optio pub(crate) async fn get_progress_per_vnode( state_table: &StateTableInner, state_len: usize, -) -> StreamExecutorResult> { +) -> StreamExecutorResult> { debug_assert!(!state_table.vnode_bitmap().is_empty()); - let vnodes = state_table.vnodes().iter_vnodes_scalar(); + let vnodes = state_table.vnodes().iter_vnodes(); let mut result = Vec::with_capacity(state_table.vnodes().len()); for vnode in vnodes { - let vnode_key: &[Datum] = &[Some(vnode.into())]; + let vnode_key: &[Datum] = &[Some(vnode.to_scalar().into())]; let state_for_vnode_key = state_table.get_row(vnode_key).await?; // original_backfill_datum_pos = (state_len - 1) @@ -189,7 +197,7 @@ pub(crate) async fn get_progress_per_vnode BackfillProgressPerVnode::NotStarted, }; - result.push(backfill_progress); + result.push((vnode, backfill_progress)); } Ok(result) } From 12d7d93c98ff6d7a7ac704ac0ea1aae92b0b3a74 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Tue, 4 Jul 2023 22:59:13 +0800 Subject: [PATCH 83/90] refactor to snapshot_read with backfill_state --- .../executor/backfill/arrangement_backfill.rs | 37 ++++++++++++++----- src/stream/src/executor/backfill/utils.rs | 7 +++- 2 files changed, 33 insertions(+), 11 deletions(-) diff --git a/src/stream/src/executor/backfill/arrangement_backfill.rs b/src/stream/src/executor/backfill/arrangement_backfill.rs index 4af88a60dccb7..54d04f0568e8c 100644 --- a/src/stream/src/executor/backfill/arrangement_backfill.rs +++ b/src/stream/src/executor/backfill/arrangement_backfill.rs @@ -155,7 +155,7 @@ where let snapshot = Self::snapshot_read_per_vnode( schema.clone(), &upstream_table, - current_pos_map.clone(), // FIXME: temporary workaround... How to avoid it? + backfill_state.clone(), // FIXME: temporary workaround... How to avoid it? ); pin_mut!(snapshot); snapshot.try_next().await?.unwrap().is_none() @@ -242,7 +242,7 @@ where let right_snapshot = pin!(Self::snapshot_read_per_vnode( schema.clone(), &upstream_table, - current_pos_map.clone(), // FIXME: temporary workaround, how to avoid it? + backfill_state.clone(), // FIXME: temporary workaround, how to avoid it? ) .map(Either::Right),); @@ -475,21 +475,38 @@ where /// /// TODO(kwannoel): Support partially complete snapshot reads. /// That will require the following changes: - /// 1. Instead of returning stream chunk and vnode, we need to dispatch 3 diff messages: - /// a. COMPLETE_VNODE(vnode): Current iterator is complete, in that case we need to handle it - /// in arrangement backfill. We should not buffer updates for this vnode, and forward - /// all messages. b. MESSAGE(CHUNK): Current iterator is not complete, in that case we - /// need to buffer updates for this vnode. c. FINISHED: All iterators finished. + /// Instead of returning stream chunk and vnode, we need to dispatch 3 diff messages: + /// 1. COMPLETE_VNODE(vnode): Current iterator is complete, in that case we need to handle it + /// in arrangement backfill. We should not buffer updates for this vnode, and forward + /// all messages. + /// 2. MESSAGE(CHUNK): Current iterator is not complete, in that case we + /// need to buffer updates for this vnode. + /// 3. FINISHED: All iterators finished. + /// + /// For now we only support the case where all iterators are complete. #[try_stream(ok = Option<(VirtualNode, StreamChunk)>, error = StreamExecutorError)] async fn snapshot_read_per_vnode<'a>( schema: Arc, upstream_table: &'a ReplicatedStateTable, - current_pos_map: CurrentPosMap, + backfill_state: BackfillState, ) { let mut streams = Vec::with_capacity(upstream_table.vnodes().len()); for vnode in upstream_table.vnodes().iter_vnodes() { - let current_pos = current_pos_map.get(&vnode); - let range_bounds = compute_bounds(upstream_table.pk_indices(), current_pos.cloned()); + let backfill_progress = match backfill_state.get_progress(&vnode) { + None => bail!( + "Backfill progress for vnode {:#?} not found, backfill_state not initialized properly", + vnode, + ), + Some(p) => p, + }; + let current_pos = match backfill_progress { + BackfillProgressPerVnode::Completed => { + continue; + } + BackfillProgressPerVnode::NotStarted => None, + BackfillProgressPerVnode::InProgress(current_pos) => Some(current_pos.clone()), + }; + let range_bounds = compute_bounds(upstream_table.pk_indices(), current_pos.clone()); if range_bounds.is_none() { continue; } diff --git a/src/stream/src/executor/backfill/utils.rs b/src/stream/src/executor/backfill/utils.rs index 8d6cc6d449a8c..2bd18502261ad 100644 --- a/src/stream/src/executor/backfill/utils.rs +++ b/src/stream/src/executor/backfill/utils.rs @@ -39,6 +39,7 @@ use crate::executor::{ pub type CurrentPosMap = HashMap; +#[derive(Clone, Debug)] pub struct BackfillState { /// Used to track backfill progress. inner: HashMap, @@ -55,6 +56,10 @@ impl BackfillState { self.inner.is_empty() } + pub fn get_progress(&self, vnode: &VirtualNode) -> Option<&BackfillProgressPerVnode> { + self.inner.get(vnode) + } + fn iter_backfill_progress( &self, ) -> impl Iterator { @@ -71,7 +76,7 @@ impl From> for BackfillState { } /// Used for tracking backfill state per vnode -#[derive(Eq, PartialEq, Debug)] +#[derive(Clone, Eq, PartialEq, Debug)] pub enum BackfillProgressPerVnode { NotStarted, InProgress(OwnedRow), From df4942a78245eea32cb6afefc5fa9a1966fc0abc Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Tue, 4 Jul 2023 23:01:11 +0800 Subject: [PATCH 84/90] update progress to backfill_state --- .../src/executor/backfill/arrangement_backfill.rs | 2 +- src/stream/src/executor/backfill/utils.rs | 12 ++++++++++-- 2 files changed, 11 insertions(+), 3 deletions(-) diff --git a/src/stream/src/executor/backfill/arrangement_backfill.rs b/src/stream/src/executor/backfill/arrangement_backfill.rs index 54d04f0568e8c..1a902f15fd38c 100644 --- a/src/stream/src/executor/backfill/arrangement_backfill.rs +++ b/src/stream/src/executor/backfill/arrangement_backfill.rs @@ -310,7 +310,7 @@ where vnode, &chunk, &pk_in_output_indices, - &mut current_pos_map, + &mut backfill_state, ); let chunk_cardinality = chunk.cardinality() as u64; diff --git a/src/stream/src/executor/backfill/utils.rs b/src/stream/src/executor/backfill/utils.rs index 2bd18502261ad..5764bcd249baf 100644 --- a/src/stream/src/executor/backfill/utils.rs +++ b/src/stream/src/executor/backfill/utils.rs @@ -60,6 +60,14 @@ impl BackfillState { self.inner.get(vnode) } + pub fn update_progress( + &mut self, + vnode: VirtualNode, + progress: BackfillProgressPerVnode, + ) -> Option { + self.inner.insert(vnode, progress) + } + fn iter_backfill_progress( &self, ) -> impl Iterator { @@ -318,10 +326,10 @@ pub(crate) fn update_pos_by_vnode( vnode: VirtualNode, chunk: &StreamChunk, pk_in_output_indices: &[usize], - current_pos_map: &mut CurrentPosMap, + backfill_state: &mut BackfillState, ) { let new_pos = get_new_pos(chunk, pk_in_output_indices); - current_pos_map.insert(vnode, new_pos); + backfill_state.update_progress(vnode, BackfillProgressPerVnode::InProgress(new_pos)); } /// Get new backfill pos from the chunk. Since chunk should have ordered rows, we can just take the From 55d26dea21eca2f93ac33395e5e979d718873902 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Tue, 4 Jul 2023 23:27:13 +0800 Subject: [PATCH 85/90] update mark_chunk_ref_by_vnode to use backfill_state --- .../executor/backfill/arrangement_backfill.rs | 13 ++--- .../executor/backfill/no_shuffle_backfill.rs | 2 + src/stream/src/executor/backfill/utils.rs | 49 +++++++++++++------ 3 files changed, 40 insertions(+), 24 deletions(-) diff --git a/src/stream/src/executor/backfill/arrangement_backfill.rs b/src/stream/src/executor/backfill/arrangement_backfill.rs index 1a902f15fd38c..f02f9ba8e092b 100644 --- a/src/stream/src/executor/backfill/arrangement_backfill.rs +++ b/src/stream/src/executor/backfill/arrangement_backfill.rs @@ -346,10 +346,10 @@ where yield Message::Chunk(mapping_chunk( mark_chunk_ref_by_vnode( &chunk, - ¤t_pos_map, + &backfill_state, &pk_in_output_indices, &pk_order, - ), + )?, &self.output_indices, )); } @@ -431,6 +431,7 @@ where // Or snapshot was empty and we construct a placeholder state. debug_assert_ne!(current_pos, None); + // FIXME persist_state( barrier.epoch, &mut self.state_table, @@ -492,13 +493,7 @@ where ) { let mut streams = Vec::with_capacity(upstream_table.vnodes().len()); for vnode in upstream_table.vnodes().iter_vnodes() { - let backfill_progress = match backfill_state.get_progress(&vnode) { - None => bail!( - "Backfill progress for vnode {:#?} not found, backfill_state not initialized properly", - vnode, - ), - Some(p) => p, - }; + let backfill_progress = backfill_state.get_progress_infallible(&vnode)?; let current_pos = match backfill_progress { BackfillProgressPerVnode::Completed => { continue; diff --git a/src/stream/src/executor/backfill/no_shuffle_backfill.rs b/src/stream/src/executor/backfill/no_shuffle_backfill.rs index 36b79fa57c62a..7a0399ea007ad 100644 --- a/src/stream/src/executor/backfill/no_shuffle_backfill.rs +++ b/src/stream/src/executor/backfill/no_shuffle_backfill.rs @@ -259,6 +259,8 @@ where // upstream buffer chunk // Consume upstream buffer chunk + // If no current_pos, means we did not process any snapshot yet. + // In that case we can just ignore the upstream buffer chunk. if let Some(current_pos) = ¤t_pos { for chunk in upstream_chunk_buffer.drain(..) { cur_barrier_upstream_processed_rows += diff --git a/src/stream/src/executor/backfill/utils.rs b/src/stream/src/executor/backfill/utils.rs index 5764bcd249baf..f900f54c6a361 100644 --- a/src/stream/src/executor/backfill/utils.rs +++ b/src/stream/src/executor/backfill/utils.rs @@ -21,6 +21,7 @@ use futures::Stream; use futures_async_stream::try_stream; use risingwave_common::array::stream_record::Record; use risingwave_common::array::{Op, StreamChunk}; +use risingwave_common::bail; use risingwave_common::buffer::BitmapBuilder; use risingwave_common::catalog::Schema; use risingwave_common::hash::{VirtualNode, VnodeBitmapExt}; @@ -56,8 +57,18 @@ impl BackfillState { self.inner.is_empty() } - pub fn get_progress(&self, vnode: &VirtualNode) -> Option<&BackfillProgressPerVnode> { - self.inner.get(vnode) + // Expects the vnode to always have progress, otherwise it will return an error. + pub fn get_progress_infallible( + &self, + vnode: &VirtualNode, + ) -> StreamExecutorResult<&BackfillProgressPerVnode> { + match self.inner.get(vnode) { + Some(p) => Ok(p), + None => bail!( + "Backfill progress for vnode {:#?} not found, backfill_state not initialized properly", + vnode, + ), + } } pub fn update_progress( @@ -109,31 +120,39 @@ pub(crate) fn mark_chunk( /// TODO(kwannoel): We should always forward rows with status `FINISHED`. pub(crate) fn mark_chunk_ref_by_vnode( chunk: &StreamChunk, - current_pos_map: &HashMap, + backfill_state: &BackfillState, pk_in_output_indices: PkIndicesRef<'_>, pk_order: &[OrderType], -) -> StreamChunk { +) -> StreamExecutorResult { let chunk = chunk.clone(); let (data, ops) = chunk.into_parts(); let mut new_visibility = BitmapBuilder::with_capacity(ops.len()); // Use project to avoid allocation. - for v in data.rows().map(|row| { + for row in data.rows() { // TODO(kwannoel): Is this logic correct for computing vnode? // I will revisit it again when arrangement_backfill is implemented e2e. let vnode = VirtualNode::compute_row(row, pk_in_output_indices); - let current_pos = current_pos_map.get(&vnode).unwrap(); - let lhs = row.project(pk_in_output_indices); - let rhs = current_pos.project(pk_in_output_indices); - let order = cmp_datum_iter(lhs.iter(), rhs.iter(), pk_order.iter().copied()); - match order { - Ordering::Less | Ordering::Equal => true, - Ordering::Greater => false, - } - }) { + let v = match backfill_state.get_progress_infallible(&vnode)? { + BackfillProgressPerVnode::Completed => true, + BackfillProgressPerVnode::NotStarted => false, + BackfillProgressPerVnode::InProgress(current_pos) => { + let lhs = row.project(pk_in_output_indices); + let rhs = current_pos.project(pk_in_output_indices); + let order = cmp_datum_iter(lhs.iter(), rhs.iter(), pk_order.iter().copied()); + match order { + Ordering::Less | Ordering::Equal => true, + Ordering::Greater => false, + } + } + }; new_visibility.append(v); } let (columns, _) = data.into_parts(); - StreamChunk::new(ops, columns, Some(new_visibility.finish())) + Ok(StreamChunk::new( + ops, + columns, + Some(new_visibility.finish()), + )) } /// Mark chunk: From 56d853dc820ffbeb67eeb16cc759806140e567f1 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Tue, 4 Jul 2023 23:28:52 +0800 Subject: [PATCH 86/90] interim commit: persist_state_per_vnode --- .../src/executor/backfill/arrangement_backfill.rs | 11 +++-------- 1 file changed, 3 insertions(+), 8 deletions(-) diff --git a/src/stream/src/executor/backfill/arrangement_backfill.rs b/src/stream/src/executor/backfill/arrangement_backfill.rs index f02f9ba8e092b..2b103e020d607 100644 --- a/src/stream/src/executor/backfill/arrangement_backfill.rs +++ b/src/stream/src/executor/backfill/arrangement_backfill.rs @@ -30,11 +30,7 @@ use risingwave_common::util::select_all; use risingwave_storage::StateStore; use crate::common::table::state_table::ReplicatedStateTable; -use crate::executor::backfill::utils::{ - check_all_vnode_finished, compute_bounds, construct_initial_finished_state, - get_progress_per_vnode, iter_chunks, mapping_chunk, mapping_message, mark_chunk_ref_by_vnode, - persist_state, update_pos_by_vnode, BackfillProgressPerVnode, BackfillState, CurrentPosMap, -}; +use crate::executor::backfill::utils::{check_all_vnode_finished, compute_bounds, construct_initial_finished_state, get_progress_per_vnode, iter_chunks, mapping_chunk, mapping_message, mark_chunk_ref_by_vnode, persist_state, update_pos_by_vnode, BackfillProgressPerVnode, BackfillState, CurrentPosMap, persist_state_per_vnode}; use crate::executor::monitor::StreamingMetrics; use crate::executor::{ expect_first_barrier, Barrier, BoxedExecutor, BoxedMessageStream, Executor, ExecutorInfo, @@ -390,15 +386,14 @@ where // Persist state on barrier // FIXME: This should persist state per vnode. - persist_state( + persist_state_per_vnode( barrier.epoch, &mut self.state_table, false, ¤t_pos, &mut old_state, &mut current_state, - ) - .await?; + ).await?; yield Message::Barrier(barrier); } From 259c27820a84e2ac5588adb9f08061fba4aaace5 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Tue, 4 Jul 2023 23:47:38 +0800 Subject: [PATCH 87/90] use persist_state_per_vnode --- .../executor/backfill/arrangement_backfill.rs | 40 +++++++++---------- 1 file changed, 20 insertions(+), 20 deletions(-) diff --git a/src/stream/src/executor/backfill/arrangement_backfill.rs b/src/stream/src/executor/backfill/arrangement_backfill.rs index 2b103e020d607..7fae933686f72 100644 --- a/src/stream/src/executor/backfill/arrangement_backfill.rs +++ b/src/stream/src/executor/backfill/arrangement_backfill.rs @@ -30,7 +30,12 @@ use risingwave_common::util::select_all; use risingwave_storage::StateStore; use crate::common::table::state_table::ReplicatedStateTable; -use crate::executor::backfill::utils::{check_all_vnode_finished, compute_bounds, construct_initial_finished_state, get_progress_per_vnode, iter_chunks, mapping_chunk, mapping_message, mark_chunk_ref_by_vnode, persist_state, update_pos_by_vnode, BackfillProgressPerVnode, BackfillState, CurrentPosMap, persist_state_per_vnode}; +use crate::executor::backfill::utils::{ + check_all_vnode_finished, compute_bounds, construct_initial_finished_state, + get_progress_per_vnode, iter_chunks, mapping_chunk, mapping_message, mark_chunk_ref_by_vnode, + persist_state, persist_state_per_vnode, update_pos_by_vnode, BackfillProgressPerVnode, + BackfillState, CurrentPosMap, +}; use crate::executor::monitor::StreamingMetrics; use crate::executor::{ expect_first_barrier, Barrier, BoxedExecutor, BoxedMessageStream, Executor, ExecutorInfo, @@ -133,11 +138,7 @@ where } let mut backfill_state: BackfillState = progress_per_vnode.into(); - - // Current position of upstream_table primary key. - // Current position is computed **per vnode**. - // FIXME: This should be backfill state instead. We need to know which vnode is finished. - let mut current_pos_map: CurrentPosMap = HashMap::new(); + let mut committed_progress = HashMap::new(); // If the snapshot is empty, we don't need to backfill. // We cannot complete progress now, as we want to persist @@ -174,7 +175,7 @@ where // They contain the backfill position, and the progress. // However, they do not contain the vnode key (index 0). // That is filled in when we flush the state table. - let mut current_state: Vec = vec![None; state_len]; + let mut temporary_state: Vec = vec![None; state_len]; let mut old_state: Option> = None; // The first barrier message should be propagated. @@ -385,15 +386,15 @@ where ); // Persist state on barrier - // FIXME: This should persist state per vnode. persist_state_per_vnode( barrier.epoch, &mut self.state_table, false, - ¤t_pos, - &mut old_state, - &mut current_state, - ).await?; + &mut backfill_state, + &mut committed_progress, + &mut temporary_state, + ) + .await?; yield Message::Barrier(barrier); } @@ -426,16 +427,15 @@ where // Or snapshot was empty and we construct a placeholder state. debug_assert_ne!(current_pos, None); - // FIXME - persist_state( + persist_state_per_vnode( barrier.epoch, &mut self.state_table, - true, - ¤t_pos, - &mut old_state, - &mut current_state, - ) - .await?; + false, + &mut backfill_state, + &mut committed_progress, + &mut temporary_state, + ).await?; + self.progress.finish(barrier.epoch.curr); yield msg; break; From d1ff383c27adfb4796dbd1f2c56bdeda9f32f394 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Wed, 5 Jul 2023 00:37:59 +0800 Subject: [PATCH 88/90] clean --- .../executor/backfill/arrangement_backfill.rs | 34 +++++--------- .../executor/backfill/no_shuffle_backfill.rs | 2 +- src/stream/src/executor/backfill/utils.rs | 47 ++++++------------- 3 files changed, 27 insertions(+), 56 deletions(-) diff --git a/src/stream/src/executor/backfill/arrangement_backfill.rs b/src/stream/src/executor/backfill/arrangement_backfill.rs index 7fae933686f72..8370efa85f91d 100644 --- a/src/stream/src/executor/backfill/arrangement_backfill.rs +++ b/src/stream/src/executor/backfill/arrangement_backfill.rs @@ -24,17 +24,15 @@ use risingwave_common::array::StreamChunk; use risingwave_common::bail; use risingwave_common::catalog::Schema; use risingwave_common::hash::{VirtualNode, VnodeBitmapExt}; -use risingwave_common::row::OwnedRow; use risingwave_common::types::Datum; use risingwave_common::util::select_all; use risingwave_storage::StateStore; use crate::common::table::state_table::ReplicatedStateTable; use crate::executor::backfill::utils::{ - check_all_vnode_finished, compute_bounds, construct_initial_finished_state, - get_progress_per_vnode, iter_chunks, mapping_chunk, mapping_message, mark_chunk_ref_by_vnode, - persist_state, persist_state_per_vnode, update_pos_by_vnode, BackfillProgressPerVnode, - BackfillState, CurrentPosMap, + compute_bounds, construct_initial_finished_state, get_progress_per_vnode, iter_chunks, + mapping_chunk, mapping_message, mark_chunk_ref_by_vnode, persist_state_per_vnode, + update_pos_by_vnode, BackfillProgressPerVnode, BackfillState, }; use crate::executor::monitor::StreamingMetrics; use crate::executor::{ @@ -166,17 +164,11 @@ where // | f | f | -> | t | let to_backfill = !is_completely_finished && !is_snapshot_empty; - // Current position of the upstream_table primary key. - // `None` means it starts from the beginning. - // FIXME: Remove it - let mut current_pos: Option = None; - // Use these to persist state. // They contain the backfill position, and the progress. // However, they do not contain the vnode key (index 0). // That is filled in when we flush the state table. let mut temporary_state: Vec = vec![None; state_len]; - let mut old_state: Option> = None; // The first barrier message should be propagated. yield Message::Barrier(first_barrier); @@ -302,7 +294,6 @@ where // Raise the current position. // As snapshot read streams are ordered by pk, so we can // just use the last row to update `current_pos`. - // current_pos = up(&chunk, &pk_in_output_indices); update_pos_by_vnode( vnode, &chunk, @@ -339,7 +330,7 @@ where // Flush downstream. // If no current_pos, means no snapshot processed yet. // Also means we don't need propagate any updates <= current_pos. - if let Some(_current_pos) = ¤t_pos { + if backfill_state.has_progress() { yield Message::Chunk(mapping_chunk( mark_chunk_ref_by_vnode( &chunk, @@ -418,15 +409,12 @@ where // since it expects to have been initialized in previous epoch // (there's no epoch before the first epoch). if is_snapshot_empty { - current_pos = - construct_initial_finished_state(pk_in_output_indices.len()) + let finished_state = construct_initial_finished_state(pk_in_output_indices.len()); + for vnode in upstream_table.vnodes().iter_vnodes() { + backfill_state.update_progress(vnode, BackfillProgressPerVnode::InProgress(finished_state.clone())); + } } - // We will update current_pos at least once, - // since snapshot read has to be non-empty, - // Or snapshot was empty and we construct a placeholder state. - debug_assert_ne!(current_pos, None); - persist_state_per_vnode( barrier.epoch, &mut self.state_table, @@ -481,14 +469,14 @@ where /// /// For now we only support the case where all iterators are complete. #[try_stream(ok = Option<(VirtualNode, StreamChunk)>, error = StreamExecutorError)] - async fn snapshot_read_per_vnode<'a>( + async fn snapshot_read_per_vnode( schema: Arc, - upstream_table: &'a ReplicatedStateTable, + upstream_table: &ReplicatedStateTable, backfill_state: BackfillState, ) { let mut streams = Vec::with_capacity(upstream_table.vnodes().len()); for vnode in upstream_table.vnodes().iter_vnodes() { - let backfill_progress = backfill_state.get_progress_infallible(&vnode)?; + let backfill_progress = backfill_state.get_progress(&vnode)?; let current_pos = match backfill_progress { BackfillProgressPerVnode::Completed => { continue; diff --git a/src/stream/src/executor/backfill/no_shuffle_backfill.rs b/src/stream/src/executor/backfill/no_shuffle_backfill.rs index 7a0399ea007ad..878bff5dd072b 100644 --- a/src/stream/src/executor/backfill/no_shuffle_backfill.rs +++ b/src/stream/src/executor/backfill/no_shuffle_backfill.rs @@ -387,7 +387,7 @@ where // (there's no epoch before the first epoch). if is_snapshot_empty { current_pos = - construct_initial_finished_state(pk_in_output_indices.len()) + Some(construct_initial_finished_state(pk_in_output_indices.len())) } // We will update current_pos at least once, diff --git a/src/stream/src/executor/backfill/utils.rs b/src/stream/src/executor/backfill/utils.rs index f900f54c6a361..810e737f04da5 100644 --- a/src/stream/src/executor/backfill/utils.rs +++ b/src/stream/src/executor/backfill/utils.rs @@ -38,8 +38,6 @@ use crate::executor::{ Message, PkIndicesRef, StreamExecutorError, StreamExecutorResult, Watermark, }; -pub type CurrentPosMap = HashMap; - #[derive(Clone, Debug)] pub struct BackfillState { /// Used to track backfill progress. @@ -47,18 +45,18 @@ pub struct BackfillState { } impl BackfillState { - fn new() -> Self { - Self { - inner: HashMap::new(), - } + fn has_no_progress(&self) -> bool { + self.inner + .values() + .all(|p| !matches!(p, BackfillProgressPerVnode::InProgress(_))) } - fn has_no_progress(&self) -> bool { - self.inner.is_empty() + pub(crate) fn has_progress(&self) -> bool { + !self.has_no_progress() } // Expects the vnode to always have progress, otherwise it will return an error. - pub fn get_progress_infallible( + pub(crate) fn get_progress( &self, vnode: &VirtualNode, ) -> StreamExecutorResult<&BackfillProgressPerVnode> { @@ -71,7 +69,7 @@ impl BackfillState { } } - pub fn update_progress( + pub(crate) fn update_progress( &mut self, vnode: VirtualNode, progress: BackfillProgressPerVnode, @@ -132,7 +130,7 @@ pub(crate) fn mark_chunk_ref_by_vnode( // TODO(kwannoel): Is this logic correct for computing vnode? // I will revisit it again when arrangement_backfill is implemented e2e. let vnode = VirtualNode::compute_row(row, pk_in_output_indices); - let v = match backfill_state.get_progress_infallible(&vnode)? { + let v = match backfill_state.get_progress(&vnode)? { BackfillProgressPerVnode::Completed => true, BackfillProgressPerVnode::NotStarted => false, BackfillProgressPerVnode::InProgress(current_pos) => { @@ -325,21 +323,6 @@ pub(crate) fn build_temporary_state( row_state[current_pos.len() + 1] = Some(is_finished.into()); } -pub(crate) fn update_pos_per_vnode( - chunk: &StreamChunk, - pk_in_output_indices: &[usize], -) -> Option { - Some( - chunk - .rows() - .last() - .unwrap() - .1 - .project(pk_in_output_indices) - .into_owned_row(), - ) -} - /// Update backfill pos by vnode. pub(crate) fn update_pos_by_vnode( vnode: VirtualNode, @@ -368,8 +351,8 @@ pub(crate) fn get_new_pos(chunk: &StreamChunk, pk_in_output_indices: &[usize]) - // This is so we can persist backfill state as "finished". // It won't be confused with another case where pk position comprised of nulls, // because they both record that backfill is finished. -pub(crate) fn construct_initial_finished_state(pos_len: usize) -> Option { - Some(OwnedRow::new(vec![None; pos_len])) +pub(crate) fn construct_initial_finished_state(pos_len: usize) -> OwnedRow { + OwnedRow::new(vec![None; pos_len]) } pub(crate) fn compute_bounds( @@ -419,9 +402,9 @@ pub(crate) async fn iter_chunks<'a, S, E>( /// Schema /// | vnode | pk | `backfill_finished` | /// Persists the state per vnode. -/// 1. For each (vnode, current_pos), -/// either insert OR update the state, -/// depending if there was an old state. +/// 1. For each (`vnode`, `current_pos`), +/// Either insert if no old state, +/// Or update the state if have old state. pub(crate) async fn persist_state_per_vnode( epoch: EpochPair, table: &mut StateTableInner, @@ -444,7 +427,7 @@ pub(crate) async fn persist_state_per_vnode Date: Wed, 5 Jul 2023 00:40:31 +0800 Subject: [PATCH 89/90] minor --- src/stream/src/executor/backfill/utils.rs | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/stream/src/executor/backfill/utils.rs b/src/stream/src/executor/backfill/utils.rs index 810e737f04da5..1d4f1ce90bd5c 100644 --- a/src/stream/src/executor/backfill/utils.rs +++ b/src/stream/src/executor/backfill/utils.rs @@ -114,8 +114,6 @@ pub(crate) fn mark_chunk( /// For each row of the chunk, forward it to downstream if its pk <= `current_pos` for the /// corresponding `vnode`, otherwise ignore it. /// We implement it by changing the visibility bitmap. -/// -/// TODO(kwannoel): We should always forward rows with status `FINISHED`. pub(crate) fn mark_chunk_ref_by_vnode( chunk: &StreamChunk, backfill_state: &BackfillState, From 9cc6ad26b19a67e4aa9c7e15246b0e885135f243 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Sat, 8 Jul 2023 01:54:07 +0800 Subject: [PATCH 90/90] use try_join_all --- src/stream/src/executor/backfill/utils.rs | 19 +++++++++++++------ 1 file changed, 13 insertions(+), 6 deletions(-) diff --git a/src/stream/src/executor/backfill/utils.rs b/src/stream/src/executor/backfill/utils.rs index 1d4f1ce90bd5c..ad2f3be64e075 100644 --- a/src/stream/src/executor/backfill/utils.rs +++ b/src/stream/src/executor/backfill/utils.rs @@ -17,6 +17,7 @@ use std::collections::HashMap; use std::ops::Bound; use await_tree::InstrumentAwait; +use futures::future::try_join_all; use futures::Stream; use futures_async_stream::try_stream; use risingwave_common::array::stream_record::Record; @@ -28,6 +29,7 @@ use risingwave_common::hash::{VirtualNode, VnodeBitmapExt}; use risingwave_common::row::{OwnedRow, Row, RowExt}; use risingwave_common::types::Datum; use risingwave_common::util::epoch::EpochPair; +use risingwave_common::util::iter_util::ZipEqDebug; use risingwave_common::util::sort_util::{cmp_datum_iter, OrderType}; use risingwave_common::util::value_encoding::BasicSerde; use risingwave_storage::table::collect_data_chunk; @@ -207,12 +209,17 @@ pub(crate) async fn get_progress_per_vnode {