risingwave_batch/executor/
merge_sort_exchange.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
// 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::sync::Arc;

use futures_async_stream::try_stream;
use risingwave_common::array::DataChunk;
use risingwave_common::catalog::{Field, Schema};
use risingwave_common::memory::MemoryContext;
use risingwave_common::util::sort_util::ColumnOrder;
use risingwave_pb::batch_plan::plan_node::NodeBody;
use risingwave_pb::batch_plan::PbExchangeSource;

use crate::error::{BatchError, Result};
use crate::executor::{
    BoxedDataChunkStream, BoxedExecutor, BoxedExecutorBuilder, CreateSource, DefaultCreateSource,
    Executor, ExecutorBuilder, MergeSortExecutor, WrapStreamExecutor,
};
use crate::task::{BatchTaskContext, TaskId};

pub type MergeSortExchangeExecutor<C> = MergeSortExchangeExecutorImpl<DefaultCreateSource, C>;

/// `MergeSortExchangeExecutor2` takes inputs from multiple sources and
/// The outputs of all the sources have been sorted in the same way.
pub struct MergeSortExchangeExecutorImpl<CS, C> {
    context: C,
    column_orders: Arc<Vec<ColumnOrder>>,
    proto_sources: Vec<PbExchangeSource>,
    /// Mock-able `CreateSource`.
    source_creators: Vec<CS>,
    schema: Schema,
    task_id: TaskId,
    identity: String,
    /// The maximum size of the chunk produced by executor at a time.
    chunk_size: usize,
    mem_ctx: MemoryContext,
}

impl<CS: 'static + Send + CreateSource, C: BatchTaskContext> MergeSortExchangeExecutorImpl<CS, C> {
    #[allow(clippy::too_many_arguments)]
    pub fn new(
        context: C,
        column_orders: Arc<Vec<ColumnOrder>>,
        proto_sources: Vec<PbExchangeSource>,
        source_creators: Vec<CS>,
        schema: Schema,
        task_id: TaskId,
        identity: String,
        chunk_size: usize,
    ) -> Self {
        let mem_ctx = context.create_executor_mem_context(&identity);

        Self {
            context,
            column_orders,
            proto_sources,
            source_creators,
            schema,
            task_id,
            identity,
            chunk_size,
            mem_ctx,
        }
    }
}

impl<CS: 'static + Send + CreateSource, C: BatchTaskContext> Executor
    for MergeSortExchangeExecutorImpl<CS, C>
{
    fn schema(&self) -> &Schema {
        &self.schema
    }

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

    fn execute(self: Box<Self>) -> BoxedDataChunkStream {
        self.do_execute()
    }
}
/// Everytime `execute` is called, it tries to produce a chunk of size
/// `self.chunk_size`. It is possible that the chunk's size is smaller than the
/// `self.chunk_size` as the executor runs out of input from `sources`.
impl<CS: 'static + Send + CreateSource, C: BatchTaskContext> MergeSortExchangeExecutorImpl<CS, C> {
    #[try_stream(boxed, ok = DataChunk, error = BatchError)]
    async fn do_execute(self: Box<Self>) {
        let mut sources: Vec<BoxedExecutor> = vec![];
        for source_idx in 0..self.proto_sources.len() {
            let new_source = self.source_creators[source_idx]
                .create_source(self.context.clone(), &self.proto_sources[source_idx])
                .await?;

            sources.push(Box::new(WrapStreamExecutor::new(
                self.schema.clone(),
                new_source.take_data_stream(),
            )));
        }

        let merge_sort_executor = Box::new(MergeSortExecutor::new(
            sources,
            self.column_orders.clone(),
            self.schema,
            format!("MergeSortExecutor{}", &self.task_id.task_id),
            self.chunk_size,
            self.mem_ctx,
        ));

        #[for_await]
        for chunk in merge_sort_executor.execute() {
            yield chunk?;
        }
    }
}

pub struct MergeSortExchangeExecutorBuilder {}

#[async_trait::async_trait]
impl BoxedExecutorBuilder for MergeSortExchangeExecutorBuilder {
    async fn new_boxed_executor<C: BatchTaskContext>(
        source: &ExecutorBuilder<'_, C>,
        inputs: Vec<BoxedExecutor>,
    ) -> Result<BoxedExecutor> {
        ensure!(
            inputs.is_empty(),
            "MergeSortExchangeExecutor should not have child!"
        );
        let sort_merge_node = try_match_expand!(
            source.plan_node().get_node_body().unwrap(),
            NodeBody::MergeSortExchange
        )?;

        let column_orders = sort_merge_node
            .column_orders
            .iter()
            .map(ColumnOrder::from_protobuf)
            .collect();
        let column_orders = Arc::new(column_orders);

        let exchange_node = sort_merge_node.get_exchange()?;
        let proto_sources: Vec<PbExchangeSource> = exchange_node.get_sources().to_vec();
        let source_creators =
            vec![DefaultCreateSource::new(source.context().client_pool()); proto_sources.len()];
        ensure!(!exchange_node.get_sources().is_empty());
        let fields = exchange_node
            .get_input_schema()
            .iter()
            .map(Field::from)
            .collect::<Vec<Field>>();

        Ok(Box::new(MergeSortExchangeExecutor::<C>::new(
            source.context().clone(),
            column_orders,
            proto_sources,
            source_creators,
            Schema { fields },
            source.task_id.clone(),
            source.plan_node().get_identity().clone(),
            source.context.get_config().developer.chunk_size,
        )))
    }
}

#[cfg(test)]
mod tests {
    use futures::StreamExt;
    use risingwave_common::array::Array;
    use risingwave_common::test_prelude::DataChunkTestExt;
    use risingwave_common::types::DataType;
    use risingwave_common::util::sort_util::OrderType;

    use super::*;
    use crate::executor::test_utils::{FakeCreateSource, FakeExchangeSource};
    use crate::task::ComputeNodeContext;

    const CHUNK_SIZE: usize = 1024;

    #[tokio::test]
    async fn test_exchange_multiple_sources() {
        let chunk = DataChunk::from_pretty(
            "i
                     1
                     2
                     3",
        );
        let fake_exchange_source = FakeExchangeSource::new(vec![Some(chunk)]);
        let fake_create_source = FakeCreateSource::new(fake_exchange_source);

        let mut proto_sources: Vec<PbExchangeSource> = vec![];
        let mut source_creators = vec![];
        let num_sources = 2;
        for _ in 0..num_sources {
            proto_sources.push(PbExchangeSource::default());
            source_creators.push(fake_create_source.clone());
        }
        let column_orders = Arc::new(vec![ColumnOrder {
            column_index: 0,
            order_type: OrderType::ascending(),
        }]);

        let executor = Box::new(MergeSortExchangeExecutorImpl::<
            FakeCreateSource,
            ComputeNodeContext,
        >::new(
            ComputeNodeContext::for_test(),
            column_orders,
            proto_sources,
            source_creators,
            Schema {
                fields: vec![Field::unnamed(DataType::Int32)],
            },
            TaskId::default(),
            "MergeSortExchangeExecutor2".to_string(),
            CHUNK_SIZE,
        ));

        let mut stream = executor.execute();
        let res = stream.next().await;
        assert!(res.is_some());
        if let Some(res) = res {
            let res = res.unwrap();
            assert_eq!(res.capacity(), 3 * num_sources);
            let col0 = res.column_at(0);
            assert_eq!(col0.as_int32().value_at(0), Some(1));
            assert_eq!(col0.as_int32().value_at(1), Some(1));
            assert_eq!(col0.as_int32().value_at(2), Some(2));
            assert_eq!(col0.as_int32().value_at(3), Some(2));
            assert_eq!(col0.as_int32().value_at(4), Some(3));
            assert_eq!(col0.as_int32().value_at(5), Some(3));
        }
        let res = stream.next().await;
        assert!(res.is_none());
    }
}