risingwave_batch/executor/
log_row_seq_scan.rs

1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
// Copyright 2024 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::ops::Deref;
use std::sync::Arc;

use futures::prelude::stream::StreamExt;
use futures_async_stream::try_stream;
use futures_util::pin_mut;
use prometheus::Histogram;
use risingwave_common::array::{DataChunk, Op};
use risingwave_common::bitmap::Bitmap;
use risingwave_common::catalog::{ColumnId, Field, Schema};
use risingwave_common::hash::VnodeCountCompat;
use risingwave_common::row::{Row, RowExt};
use risingwave_common::types::ScalarImpl;
use risingwave_hummock_sdk::{HummockReadEpoch, HummockVersionId};
use risingwave_pb::batch_plan::plan_node::NodeBody;
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;
use risingwave_storage::{dispatch_state_store, StateStore};

use super::{BoxedDataChunkStream, BoxedExecutor, BoxedExecutorBuilder, Executor, ExecutorBuilder};
use crate::error::{BatchError, Result};
use crate::monitor::BatchMetrics;
use crate::task::BatchTaskContext;

pub struct LogRowSeqScanExecutor<S: StateStore> {
    chunk_size: usize,
    identity: String,
    // It is table schema + op column
    schema: Schema,

    /// Batch metrics.
    /// None: Local mode don't record mertics.
    metrics: Option<BatchMetrics>,

    table: StorageTable<S>,
    old_epoch: u64,
    new_epoch: u64,
    version_id: HummockVersionId,
    ordered: bool,
}

impl<S: StateStore> LogRowSeqScanExecutor<S> {
    pub fn new(
        table: StorageTable<S>,
        old_epoch: u64,
        new_epoch: u64,
        version_id: HummockVersionId,
        chunk_size: usize,
        identity: String,
        metrics: Option<BatchMetrics>,
        ordered: bool,
    ) -> Self {
        let mut schema = table.schema().clone();
        schema.fields.push(Field::with_name(
            risingwave_common::types::DataType::Varchar,
            "op",
        ));
        Self {
            chunk_size,
            identity,
            schema,
            metrics,
            table,
            old_epoch,
            new_epoch,
            version_id,
            ordered,
        }
    }
}

pub struct LogStoreRowSeqScanExecutorBuilder {}

#[async_trait::async_trait]
impl BoxedExecutorBuilder for LogStoreRowSeqScanExecutorBuilder {
    async fn new_boxed_executor<C: BatchTaskContext>(
        source: &ExecutorBuilder<'_, C>,
        inputs: Vec<BoxedExecutor>,
    ) -> Result<BoxedExecutor> {
        ensure!(
            inputs.is_empty(),
            "LogStore row sequential scan should not have input executor!"
        );
        let log_store_seq_scan_node = try_match_expand!(
            source.plan_node().get_node_body().unwrap(),
            NodeBody::LogRowSeqScan
        )?;

        let table_desc: &StorageTableDesc = log_store_seq_scan_node.get_table_desc()?;
        let column_ids = log_store_seq_scan_node
            .column_ids
            .iter()
            .copied()
            .map(ColumnId::from)
            .collect();

        let vnodes = match &log_store_seq_scan_node.vnode_bitmap {
            Some(vnodes) => Some(Bitmap::from(vnodes).into()),
            // This is possible for dml. vnode_bitmap is not filled by scheduler.
            // Or it's single distribution, e.g., distinct agg. We scan in a single executor.
            None => Some(Bitmap::ones(table_desc.vnode_count()).into()),
        };

        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)
        };

        assert_eq!(old_epoch.hummock_version_id, new_epoch.hummock_version_id);
        let version_id = old_epoch.hummock_version_id;
        let old_epoch = old_epoch.epoch;
        let new_epoch = new_epoch.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,
                old_epoch,
                new_epoch,
                HummockVersionId::new(version_id),
                chunk_size as usize,
                source.plan_node().get_identity().clone(),
                metrics,
                log_store_seq_scan_node.ordered,
            )))
        })
    }
}
impl<S: StateStore> Executor for LogRowSeqScanExecutor<S> {
    fn schema(&self) -> &Schema {
        &self.schema
    }

    fn identity(&self) -> &str {
        &self.identity
    }

    fn execute(self: Box<Self>) -> BoxedDataChunkStream {
        self.do_execute().boxed()
    }
}

impl<S: StateStore> LogRowSeqScanExecutor<S> {
    #[try_stream(ok = DataChunk, error = BatchError)]
    async fn do_execute(self: Box<Self>) {
        let Self {
            chunk_size,
            metrics,
            table,
            old_epoch,
            new_epoch,
            version_id,
            schema,
            ordered,
            ..
        } = *self;
        let table = std::sync::Arc::new(table);

        // Create collector.
        let histogram = metrics
            .as_ref()
            .map(|metrics| &metrics.executor_metrics().row_seq_scan_next_duration);
        // Range Scan
        // WARN: DO NOT use `select` to execute range scans concurrently
        //       it can consume too much memory if there're too many ranges.
        let stream = Self::execute_range(
            table.clone(),
            old_epoch,
            new_epoch,
            version_id,
            chunk_size,
            histogram,
            Arc::new(schema.clone()),
            ordered,
        );
        #[for_await]
        for chunk in stream {
            let chunk = chunk?;
            yield chunk;
        }
    }

    #[try_stream(ok = DataChunk, error = BatchError)]
    async fn execute_range(
        table: Arc<StorageTable<S>>,
        old_epoch: u64,
        new_epoch: u64,
        version_id: HummockVersionId,
        chunk_size: usize,
        histogram: Option<impl Deref<Target = Histogram>>,
        schema: Arc<Schema>,
        ordered: bool,
    ) {
        // Range Scan.
        let iter = table
            .batch_iter_log_with_pk_bounds(
                old_epoch,
                HummockReadEpoch::BatchQueryCommitted(new_epoch, version_id),
                ordered,
            )
            .await?
            .flat_map(|r| {
                futures::stream::iter(std::iter::from_coroutine(
                    #[coroutine]
                    move || {
                        match r {
                            Ok(change_log_row) => {
                                fn with_op(op: Op, row: impl Row) -> impl Row {
                                    row.chain([Some(ScalarImpl::Utf8(op.to_varchar().into()))])
                                }
                                for (op, row) in change_log_row.into_op_value_iter() {
                                    yield Ok(with_op(op, row));
                                }
                            }
                            Err(e) => {
                                yield Err(e);
                            }
                        };
                    },
                ))
            });

        pin_mut!(iter);
        loop {
            let timer = histogram.as_ref().map(|histogram| histogram.start_timer());

            let chunk = collect_data_chunk(&mut iter, &schema, Some(chunk_size))
                .await
                .map_err(BatchError::from)?;
            if let Some(timer) = timer {
                timer.observe_duration()
            }

            if let Some(chunk) = chunk {
                yield chunk
            } else {
                break;
            }
        }
    }
}