Skip to content
New issue

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

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

Already on GitHub? Sign in to your account

fix(storage): generate correct user key range in iter_log #17805

Merged
merged 3 commits into from
Jul 31, 2024
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
38 changes: 26 additions & 12 deletions src/batch/src/executor/log_row_seq_scan.rs
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,7 @@ use risingwave_common::catalog::{ColumnId, Field, Schema};
use risingwave_common::row::{Row, RowExt};
use risingwave_common::types::ScalarImpl;
use risingwave_pb::batch_plan::plan_node::NodeBody;
use risingwave_pb::common::BatchQueryEpoch;
use risingwave_pb::common::{batch_query_epoch, BatchQueryEpoch};
use risingwave_pb::plan_common::StorageTableDesc;
use risingwave_storage::table::batch_table::storage_table::StorageTable;
use risingwave_storage::table::{collect_data_chunk, TableDistribution};
Expand All @@ -47,15 +47,15 @@ pub struct LogRowSeqScanExecutor<S: StateStore> {
metrics: Option<BatchMetricsWithTaskLabels>,

table: StorageTable<S>,
old_epoch: BatchQueryEpoch,
new_epoch: BatchQueryEpoch,
old_epoch: u64,
new_epoch: u64,
}

impl<S: StateStore> LogRowSeqScanExecutor<S> {
pub fn new(
table: StorageTable<S>,
old_epoch: BatchQueryEpoch,
new_epoch: BatchQueryEpoch,
old_epoch: u64,
new_epoch: u64,
chunk_size: usize,
identity: String,
metrics: Option<BatchMetricsWithTaskLabels>,
Expand Down Expand Up @@ -112,12 +112,26 @@ impl BoxedExecutorBuilder for LogStoreRowSeqScanExecutorBuilder {
let chunk_size = source.context.get_config().developer.chunk_size as u32;
let metrics = source.context().batch_metrics();

let Some(BatchQueryEpoch {
epoch: Some(batch_query_epoch::Epoch::Committed(old_epoch)),
}) = &log_store_seq_scan_node.old_epoch
else {
unreachable!("invalid old epoch: {:?}", log_store_seq_scan_node.old_epoch)
};

let Some(BatchQueryEpoch {
epoch: Some(batch_query_epoch::Epoch::Committed(new_epoch)),
}) = &log_store_seq_scan_node.new_epoch
else {
unreachable!("invalid new epoch: {:?}", log_store_seq_scan_node.new_epoch)
};

dispatch_state_store!(source.context().state_store(), state_store, {
let table = StorageTable::new_partial(state_store, column_ids, vnodes, table_desc);
Ok(Box::new(LogRowSeqScanExecutor::new(
table,
log_store_seq_scan_node.old_epoch.clone().unwrap(),
log_store_seq_scan_node.new_epoch.clone().unwrap(),
*old_epoch,
*new_epoch,
chunk_size as usize,
source.plan_node().get_identity().clone(),
metrics,
Expand Down Expand Up @@ -165,8 +179,8 @@ impl<S: StateStore> LogRowSeqScanExecutor<S> {
// it can consume too much memory if there're too many ranges.
let stream = Self::execute_range(
table.clone(),
old_epoch.clone(),
new_epoch.clone(),
old_epoch,
new_epoch,
chunk_size,
histogram.clone(),
Arc::new(schema.clone()),
Expand All @@ -181,15 +195,15 @@ impl<S: StateStore> LogRowSeqScanExecutor<S> {
#[try_stream(ok = DataChunk, error = BatchError)]
async fn execute_range(
table: Arc<StorageTable<S>>,
old_epoch: BatchQueryEpoch,
new_epoch: BatchQueryEpoch,
old_epoch: u64,
new_epoch: u64,
chunk_size: usize,
histogram: Option<impl Deref<Target = Histogram>>,
schema: Arc<Schema>,
) {
// Range Scan.
let iter = table
.batch_iter_log_with_pk_bounds(old_epoch.into(), new_epoch.into())
.batch_iter_log_with_pk_bounds(old_epoch, new_epoch)
.await?
.flat_map(|r| {
futures::stream::iter(std::iter::from_coroutine(move || {
Expand Down
17 changes: 9 additions & 8 deletions src/storage/src/hummock/iterator/change_log.rs
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,9 @@ use std::ops::Bound::{Excluded, Included, Unbounded};
use risingwave_common::catalog::TableId;
use risingwave_common::must_match;
use risingwave_common::util::epoch::MAX_SPILL_TIMES;
use risingwave_hummock_sdk::key::{FullKey, SetSlice, TableKeyRange, UserKey, UserKeyRange};
use risingwave_hummock_sdk::key::{
bound_table_key_range, FullKey, SetSlice, TableKeyRange, UserKey, UserKeyRange,
};
use risingwave_hummock_sdk::EpochWithGap;

use crate::error::StorageResult;
Expand Down Expand Up @@ -347,17 +349,16 @@ pub struct ChangeLogIterator {
impl ChangeLogIterator {
pub async fn new(
epoch_range: (u64, u64),
(start_bound, end_bound): TableKeyRange,
table_key_range: TableKeyRange,
new_value_iter: MergeIterator<SstableIterator>,
old_value_iter: MergeIterator<SstableIterator>,
table_id: TableId,
) -> HummockResult<Self> {
let make_user_key = |table_key| UserKey {
table_id,
table_key,
};
let start_bound = start_bound.map(make_user_key);
let end_bound = end_bound.map(make_user_key);
let user_key_range_ref = bound_table_key_range(table_id, &table_key_range);
let (start_bound, end_bound) = (
user_key_range_ref.0.map(|key| key.cloned()),
user_key_range_ref.1.map(|key| key.cloned()),
);
let mut inner = ChangeLogIteratorInner::new(
epoch_range,
(start_bound, end_bound),
Expand Down
11 changes: 11 additions & 0 deletions src/storage/src/hummock/store/version.rs
Original file line number Diff line number Diff line change
Expand Up @@ -973,6 +973,17 @@ impl HummockVersionReader {
static EMPTY_VEC: Vec<EpochNewChangeLog> = Vec::new();
&EMPTY_VEC[..]
};
if let Some(max_epoch_change_log) = change_log.last() {
let (_, max_epoch) = epoch_range;
if !max_epoch_change_log.epochs.contains(&max_epoch) {
warn!(
max_epoch,
change_log_epochs = ?change_log.iter().flat_map(|epoch_log| epoch_log.epochs.iter()).collect_vec(),
table_id = options.table_id.table_id,
"max_epoch does not exist"
);
}
}
let read_options = Arc::new(SstableIteratorReadOptions {
cache_policy: Default::default(),
must_iterated_end_user_key: None,
Expand Down
22 changes: 9 additions & 13 deletions src/storage/src/table/batch_table/storage_table.rs
Original file line number Diff line number Diff line change
Expand Up @@ -749,8 +749,8 @@ impl<S: StateStore, SD: ValueRowSerde> StorageTableInner<S, SD> {

pub async fn batch_iter_log_with_pk_bounds(
&self,
satrt_epoch: HummockReadEpoch,
end_epoch: HummockReadEpoch,
start_epoch: u64,
end_epoch: u64,
) -> StorageResult<impl Stream<Item = StorageResult<ChangeLogRow>> + Send> {
let pk_prefix = OwnedRow::default();
let start_key = self.serialize_pk_bound(&pk_prefix, Unbounded, true);
Expand Down Expand Up @@ -779,7 +779,7 @@ impl<S: StateStore, SD: ValueRowSerde> StorageTableInner<S, SD> {
self.row_serde.clone(),
table_key_range,
read_options,
satrt_epoch,
start_epoch,
end_epoch,
)
.await?
Expand Down Expand Up @@ -974,18 +974,14 @@ impl<S: StateStore, SD: ValueRowSerde> StorageTableInnerIterLogInner<S, SD> {
row_deserializer: Arc<SD>,
table_key_range: TableKeyRange,
read_options: ReadLogOptions,
satrt_epoch: HummockReadEpoch,
end_epoch: HummockReadEpoch,
start_epoch: u64,
end_epoch: u64,
) -> StorageResult<Self> {
let raw_satrt_epoch = satrt_epoch.get_epoch();
let raw_end_epoch = end_epoch.get_epoch();
store.try_wait_epoch(end_epoch).await?;
store
.try_wait_epoch(HummockReadEpoch::Committed(end_epoch))
.await?;
let iter = store
.iter_log(
(raw_satrt_epoch, raw_end_epoch),
table_key_range,
read_options,
)
.iter_log((start_epoch, end_epoch), table_key_range, read_options)
.await?;
let iter = Self {
iter,
Expand Down
Loading