common_recordbatch/
cursor.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
// Copyright 2023 Greptime Team
//
// 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 futures::StreamExt;
use tokio::sync::Mutex;

use crate::error::Result;
use crate::recordbatch::merge_record_batches;
use crate::{RecordBatch, SendableRecordBatchStream};

struct Inner {
    stream: SendableRecordBatchStream,
    current_row_index: usize,
    current_batch: Option<RecordBatch>,
    total_rows_in_current_batch: usize,
}

/// A cursor on RecordBatchStream that fetches data batch by batch
pub struct RecordBatchStreamCursor {
    inner: Mutex<Inner>,
}

impl RecordBatchStreamCursor {
    pub fn new(stream: SendableRecordBatchStream) -> RecordBatchStreamCursor {
        Self {
            inner: Mutex::new(Inner {
                stream,
                current_row_index: 0,
                current_batch: None,
                total_rows_in_current_batch: 0,
            }),
        }
    }

    /// Take `size` of row from the `RecordBatchStream` and create a new
    /// `RecordBatch` for these rows.
    pub async fn take(&self, size: usize) -> Result<RecordBatch> {
        let mut remaining_rows_to_take = size;
        let mut accumulated_rows = Vec::new();

        let mut inner = self.inner.lock().await;

        while remaining_rows_to_take > 0 {
            // Ensure we have a current batch or fetch the next one
            if inner.current_batch.is_none()
                || inner.current_row_index >= inner.total_rows_in_current_batch
            {
                match inner.stream.next().await {
                    Some(Ok(batch)) => {
                        inner.total_rows_in_current_batch = batch.num_rows();
                        inner.current_batch = Some(batch);
                        inner.current_row_index = 0;
                    }
                    Some(Err(e)) => return Err(e),
                    None => {
                        // Stream is exhausted
                        break;
                    }
                }
            }

            // If we still have no batch after attempting to fetch
            let current_batch = match &inner.current_batch {
                Some(batch) => batch,
                None => break,
            };

            // Calculate how many rows we can take from this batch
            let rows_to_take_from_batch = remaining_rows_to_take
                .min(inner.total_rows_in_current_batch - inner.current_row_index);

            // Slice the current batch to get the desired rows
            let taken_batch =
                current_batch.slice(inner.current_row_index, rows_to_take_from_batch)?;

            // Add the taken batch to accumulated rows
            accumulated_rows.push(taken_batch);

            // Update cursor and remaining rows
            inner.current_row_index += rows_to_take_from_batch;
            remaining_rows_to_take -= rows_to_take_from_batch;
        }

        // If no rows were accumulated, return empty
        if accumulated_rows.is_empty() {
            return Ok(RecordBatch::new_empty(inner.stream.schema()));
        }

        // If only one batch was accumulated, return it directly
        if accumulated_rows.len() == 1 {
            return Ok(accumulated_rows.remove(0));
        }

        // Merge multiple batches
        merge_record_batches(inner.stream.schema(), &accumulated_rows)
    }
}

#[cfg(test)]
mod tests {
    use std::sync::Arc;

    use datatypes::prelude::ConcreteDataType;
    use datatypes::schema::{ColumnSchema, Schema};
    use datatypes::vectors::StringVector;

    use super::*;
    use crate::RecordBatches;

    #[tokio::test]
    async fn test_cursor() {
        let schema = Arc::new(Schema::new(vec![ColumnSchema::new(
            "a",
            ConcreteDataType::string_datatype(),
            false,
        )]));

        let rbs = RecordBatches::try_from_columns(
            schema.clone(),
            vec![Arc::new(StringVector::from(vec!["hello", "world"])) as _],
        )
        .unwrap();

        let cursor = RecordBatchStreamCursor::new(rbs.as_stream());
        let result_rb = cursor.take(1).await.expect("take from cursor failed");
        assert_eq!(result_rb.num_rows(), 1);

        let result_rb = cursor.take(1).await.expect("take from cursor failed");
        assert_eq!(result_rb.num_rows(), 1);

        let result_rb = cursor.take(1).await.expect("take from cursor failed");
        assert_eq!(result_rb.num_rows(), 0);

        let rb = RecordBatch::new(
            schema.clone(),
            vec![Arc::new(StringVector::from(vec!["hello", "world"])) as _],
        )
        .unwrap();
        let rbs2 =
            RecordBatches::try_new(schema.clone(), vec![rb.clone(), rb.clone(), rb]).unwrap();
        let cursor = RecordBatchStreamCursor::new(rbs2.as_stream());
        let result_rb = cursor.take(3).await.expect("take from cursor failed");
        assert_eq!(result_rb.num_rows(), 3);
        let result_rb = cursor.take(2).await.expect("take from cursor failed");
        assert_eq!(result_rb.num_rows(), 2);
        let result_rb = cursor.take(2).await.expect("take from cursor failed");
        assert_eq!(result_rb.num_rows(), 1);
        let result_rb = cursor.take(2).await.expect("take from cursor failed");
        assert_eq!(result_rb.num_rows(), 0);

        let rb = RecordBatch::new(
            schema.clone(),
            vec![Arc::new(StringVector::from(vec!["hello", "world"])) as _],
        )
        .unwrap();
        let rbs3 =
            RecordBatches::try_new(schema.clone(), vec![rb.clone(), rb.clone(), rb]).unwrap();
        let cursor = RecordBatchStreamCursor::new(rbs3.as_stream());
        let result_rb = cursor.take(10).await.expect("take from cursor failed");
        assert_eq!(result_rb.num_rows(), 6);
    }
}