risingwave_batch/executor/
top_n.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
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
// 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::cmp::Ordering;
use std::sync::Arc;

use futures_async_stream::try_stream;
use risingwave_common::array::DataChunk;
use risingwave_common::catalog::Schema;
use risingwave_common::memory::{MemMonitoredHeap, MemoryContext};
use risingwave_common::row::{OwnedRow, Row};
use risingwave_common::util::chunk_coalesce::DataChunkBuilder;
use risingwave_common::util::memcmp_encoding::{encode_chunk, MemcmpEncoded};
use risingwave_common::util::sort_util::ColumnOrder;
use risingwave_common_estimate_size::EstimateSize;
use risingwave_pb::batch_plan::plan_node::NodeBody;

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

/// Top-N Executor
///
/// Use a N-heap to store the smallest N rows.
pub struct TopNExecutor {
    child: BoxedExecutor,
    column_orders: Vec<ColumnOrder>,
    offset: usize,
    limit: usize,
    with_ties: bool,
    schema: Schema,
    identity: String,
    chunk_size: usize,
    mem_ctx: MemoryContext,
}

#[async_trait::async_trait]
impl BoxedExecutorBuilder for TopNExecutor {
    async fn new_boxed_executor<C: BatchTaskContext>(
        source: &ExecutorBuilder<'_, C>,
        inputs: Vec<BoxedExecutor>,
    ) -> Result<BoxedExecutor> {
        let [child]: [_; 1] = inputs.try_into().unwrap();

        let top_n_node =
            try_match_expand!(source.plan_node().get_node_body().unwrap(), NodeBody::TopN)?;

        let column_orders = top_n_node
            .column_orders
            .iter()
            .map(ColumnOrder::from_protobuf)
            .collect();

        let identity = source.plan_node().get_identity();

        Ok(Box::new(Self::new(
            child,
            column_orders,
            top_n_node.get_offset() as usize,
            top_n_node.get_limit() as usize,
            top_n_node.get_with_ties(),
            identity.clone(),
            source.context.get_config().developer.chunk_size,
            source.context().create_executor_mem_context(identity),
        )))
    }
}

impl TopNExecutor {
    pub fn new(
        child: BoxedExecutor,
        column_orders: Vec<ColumnOrder>,
        offset: usize,
        limit: usize,
        with_ties: bool,
        identity: String,
        chunk_size: usize,
        mem_ctx: MemoryContext,
    ) -> Self {
        let schema = child.schema().clone();
        Self {
            child,
            column_orders,
            offset,
            limit,
            with_ties,
            schema,
            identity,
            chunk_size,
            mem_ctx,
        }
    }
}

impl Executor for TopNExecutor {
    fn schema(&self) -> &Schema {
        &self.schema
    }

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

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

pub const MAX_TOPN_INIT_HEAP_CAPACITY: usize = 1024;

/// A max-heap used to find the smallest `limit+offset` items.
pub struct TopNHeap {
    heap: MemMonitoredHeap<HeapElem>,
    limit: usize,
    offset: usize,
    with_ties: bool,
}

impl TopNHeap {
    pub fn new(limit: usize, offset: usize, with_ties: bool, mem_ctx: MemoryContext) -> Self {
        assert!(limit > 0);
        Self {
            heap: MemMonitoredHeap::with_capacity(
                (limit + offset).min(MAX_TOPN_INIT_HEAP_CAPACITY),
                mem_ctx,
            ),
            limit,
            offset,
            with_ties,
        }
    }

    // Only used for swapping out the heap in hashmap, due to a bug in hashmap which forbids us from
    // using `into_iter`. We should remove this after Hashmap upgraded and fixed the bug.
    pub fn empty() -> Self {
        Self {
            heap: MemMonitoredHeap::with_capacity(0, MemoryContext::none()),
            limit: 0,
            offset: 0,
            with_ties: false,
        }
    }

    pub fn push(&mut self, elem: HeapElem) {
        if self.heap.len() < self.limit + self.offset {
            self.heap.push(elem);
        } else {
            // heap is full
            if !self.with_ties {
                let peek = self.heap.pop().unwrap();
                if elem < peek {
                    self.heap.push(elem);
                } else {
                    self.heap.push(peek);
                }
                // let inner = self.heap.inner();
                // let mut peek = inner.peek_mut().unwrap();
                // if elem < *peek {
                //     *peek = elem;
                // }
            } else {
                let peek = self.heap.peek().unwrap().clone();
                match elem.cmp(&peek) {
                    Ordering::Less => {
                        let mut ties_with_peek = vec![];
                        // pop all the ties with peek
                        ties_with_peek.push(self.heap.pop().unwrap());
                        while let Some(e) = self.heap.peek()
                            && e.encoded_row == peek.encoded_row
                        {
                            ties_with_peek.push(self.heap.pop().unwrap());
                        }
                        self.heap.push(elem);
                        // If the size is smaller than limit, we can push all the elements back.
                        if self.heap.len() < self.limit {
                            self.heap.extend(ties_with_peek);
                        }
                    }
                    Ordering::Equal => {
                        // It's a tie.
                        self.heap.push(elem);
                    }
                    Ordering::Greater => {}
                }
            }
        }
    }

    /// Returns the elements in the range `[offset, offset+limit)`.
    pub fn dump(self) -> impl Iterator<Item = HeapElem> {
        self.heap
            .into_sorted_vec()
            .into_iter()
            .rev()
            .skip(self.offset)
    }
}

#[derive(Clone, EstimateSize)]
pub struct HeapElem {
    encoded_row: MemcmpEncoded,
    row: OwnedRow,
}

impl PartialEq for HeapElem {
    fn eq(&self, other: &Self) -> bool {
        self.encoded_row.eq(&other.encoded_row)
    }
}

impl Eq for HeapElem {}

impl PartialOrd for HeapElem {
    fn partial_cmp(&self, other: &Self) -> Option<std::cmp::Ordering> {
        Some(self.cmp(other))
    }
}

impl Ord for HeapElem {
    fn cmp(&self, other: &Self) -> std::cmp::Ordering {
        self.encoded_row.cmp(&other.encoded_row)
    }
}

impl HeapElem {
    pub fn new(encoded_row: MemcmpEncoded, row: impl Row) -> Self {
        Self {
            encoded_row,
            row: row.into_owned_row(),
        }
    }

    pub fn row(&self) -> impl Row + '_ {
        &self.row
    }
}

impl TopNExecutor {
    #[try_stream(boxed, ok = DataChunk, error = BatchError)]
    async fn do_execute(self: Box<Self>) {
        if self.limit == 0 {
            return Ok(());
        }
        let mut heap = TopNHeap::new(
            self.limit,
            self.offset,
            self.with_ties,
            self.mem_ctx.clone(),
        );

        #[for_await]
        for chunk in self.child.execute() {
            let chunk = Arc::new(chunk?.compact());
            for (row_id, encoded_row) in encode_chunk(&chunk, &self.column_orders)?
                .into_iter()
                .enumerate()
            {
                heap.push(HeapElem {
                    encoded_row,
                    row: chunk.row_at(row_id).0.to_owned_row(),
                });
            }
        }

        let mut chunk_builder = DataChunkBuilder::new(self.schema.data_types(), self.chunk_size);
        for HeapElem { row, .. } in heap.dump() {
            if let Some(spilled) = chunk_builder.append_one_row(row) {
                yield spilled
            }
        }
        if let Some(spilled) = chunk_builder.consume_all() {
            yield spilled
        }
    }
}

#[cfg(test)]
mod tests {
    use futures::stream::StreamExt;
    use itertools::Itertools;
    use risingwave_common::array::Array;
    use risingwave_common::catalog::Field;
    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::MockExecutor;

    const CHUNK_SIZE: usize = 1024;

    #[tokio::test]
    async fn test_simple_top_n_executor() {
        let schema = Schema {
            fields: vec![
                Field::unnamed(DataType::Int32),
                Field::unnamed(DataType::Int32),
            ],
        };
        let mut mock_executor = MockExecutor::new(schema);
        mock_executor.add(DataChunk::from_pretty(
            "i i
             1 5
             2 4
             3 3
             4 2
             5 1",
        ));
        let column_orders = vec![
            ColumnOrder {
                column_index: 1,
                order_type: OrderType::ascending(),
            },
            ColumnOrder {
                column_index: 0,
                order_type: OrderType::ascending(),
            },
        ];
        let top_n_executor = Box::new(TopNExecutor::new(
            Box::new(mock_executor),
            column_orders,
            1,
            3,
            false,
            "TopNExecutor".to_string(),
            CHUNK_SIZE,
            MemoryContext::none(),
        ));
        let fields = &top_n_executor.schema().fields;
        assert_eq!(fields[0].data_type, DataType::Int32);
        assert_eq!(fields[1].data_type, DataType::Int32);

        let mut stream = top_n_executor.execute();
        let res = stream.next().await;

        assert!(res.is_some());
        if let Some(res) = res {
            let res = res.unwrap();
            assert_eq!(res.cardinality(), 3);
            assert_eq!(
                res.column_at(0).as_int32().iter().collect_vec(),
                vec![Some(4), Some(3), Some(2)]
            );
        }

        let res = stream.next().await;
        assert!(res.is_none());
    }

    #[tokio::test]
    async fn test_limit_0() {
        let schema = Schema {
            fields: vec![
                Field::unnamed(DataType::Int32),
                Field::unnamed(DataType::Int32),
            ],
        };
        let mut mock_executor = MockExecutor::new(schema);
        mock_executor.add(DataChunk::from_pretty(
            "i i
             1 5
             2 4
             3 3
             4 2
             5 1",
        ));
        let column_orders = vec![
            ColumnOrder {
                column_index: 1,
                order_type: OrderType::ascending(),
            },
            ColumnOrder {
                column_index: 0,
                order_type: OrderType::ascending(),
            },
        ];
        let top_n_executor = Box::new(TopNExecutor::new(
            Box::new(mock_executor),
            column_orders,
            1,
            0,
            false,
            "TopNExecutor".to_string(),
            CHUNK_SIZE,
            MemoryContext::none(),
        ));
        let fields = &top_n_executor.schema().fields;
        assert_eq!(fields[0].data_type, DataType::Int32);
        assert_eq!(fields[1].data_type, DataType::Int32);

        let mut stream = top_n_executor.execute();
        let res = stream.next().await;

        assert!(res.is_none());
    }
}