mito2/wal/
raw_entry_reader.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
// 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 std::sync::Arc;

use async_stream::try_stream;
use common_error::ext::BoxedError;
use futures::stream::BoxStream;
use snafu::ResultExt;
use store_api::logstore::entry::Entry;
use store_api::logstore::provider::Provider;
use store_api::logstore::{LogStore, WalIndex};
use store_api::storage::RegionId;
use tokio_stream::StreamExt;

use crate::error::{self, Result};
use crate::wal::EntryId;

/// A stream that yields [Entry].
pub type EntryStream<'a> = BoxStream<'a, Result<Entry>>;

/// [RawEntryReader] provides the ability to read [Entry] from the underlying [LogStore].
pub(crate) trait RawEntryReader: Send + Sync {
    fn read(&self, provider: &Provider, start_id: EntryId) -> Result<EntryStream<'static>>;
}

/// Implement the [RawEntryReader] for the [LogStore].
pub struct LogStoreRawEntryReader<S> {
    store: Arc<S>,
    wal_index: Option<WalIndex>,
}

impl<S> LogStoreRawEntryReader<S> {
    pub fn new(store: Arc<S>) -> Self {
        Self {
            store,
            wal_index: None,
        }
    }

    pub fn with_wal_index(mut self, wal_index: WalIndex) -> Self {
        self.wal_index = Some(wal_index);
        self
    }
}

impl<S: LogStore> RawEntryReader for LogStoreRawEntryReader<S> {
    fn read(&self, provider: &Provider, start_id: EntryId) -> Result<EntryStream<'static>> {
        let store = self.store.clone();
        let provider = provider.clone();
        let wal_index = self.wal_index;
        let stream = try_stream!({
            let mut stream = store
                .read(&provider, start_id, wal_index)
                .await
                .map_err(BoxedError::new)
                .with_context(|_| error::ReadWalSnafu {
                    provider: provider.clone(),
                })?;

            while let Some(entries) = stream.next().await {
                let entries =
                    entries
                        .map_err(BoxedError::new)
                        .with_context(|_| error::ReadWalSnafu {
                            provider: provider.clone(),
                        })?;

                for entry in entries {
                    yield entry
                }
            }
        });

        Ok(Box::pin(stream))
    }
}

/// A [RawEntryReader] reads [RawEntry] belongs to a specific region.
pub struct RegionRawEntryReader<R> {
    reader: R,
    region_id: RegionId,
}

impl<R> RegionRawEntryReader<R>
where
    R: RawEntryReader,
{
    pub fn new(reader: R, region_id: RegionId) -> Self {
        Self { reader, region_id }
    }
}

impl<R> RawEntryReader for RegionRawEntryReader<R>
where
    R: RawEntryReader,
{
    fn read(&self, ctx: &Provider, start_id: EntryId) -> Result<EntryStream<'static>> {
        let mut stream = self.reader.read(ctx, start_id)?;
        let region_id = self.region_id;

        let stream = try_stream!({
            while let Some(entry) = stream.next().await {
                let entry = entry?;
                if entry.region_id() == region_id {
                    yield entry
                }
            }
        });

        Ok(Box::pin(stream))
    }
}

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

    use futures::{stream, TryStreamExt};
    use store_api::logstore::entry::{Entry, NaiveEntry};
    use store_api::logstore::{
        AppendBatchResponse, EntryId, LogStore, SendableEntryStream, WalIndex,
    };
    use store_api::storage::RegionId;

    use super::*;
    use crate::error;

    #[derive(Debug)]
    struct MockLogStore {
        entries: Vec<Entry>,
    }

    #[async_trait::async_trait]
    impl LogStore for MockLogStore {
        type Error = error::Error;

        async fn stop(&self) -> Result<(), Self::Error> {
            unreachable!()
        }

        async fn append_batch(
            &self,
            _entries: Vec<Entry>,
        ) -> Result<AppendBatchResponse, Self::Error> {
            unreachable!()
        }

        async fn read(
            &self,
            _provider: &Provider,
            _id: EntryId,
            _index: Option<WalIndex>,
        ) -> Result<SendableEntryStream<'static, Entry, Self::Error>, Self::Error> {
            Ok(Box::pin(stream::iter(vec![Ok(self.entries.clone())])))
        }

        async fn create_namespace(&self, _ns: &Provider) -> Result<(), Self::Error> {
            unreachable!()
        }

        async fn delete_namespace(&self, _ns: &Provider) -> Result<(), Self::Error> {
            unreachable!()
        }

        async fn list_namespaces(&self) -> Result<Vec<Provider>, Self::Error> {
            unreachable!()
        }

        async fn obsolete(
            &self,
            _provider: &Provider,
            _region_id: RegionId,
            _entry_id: EntryId,
        ) -> Result<(), Self::Error> {
            unreachable!()
        }

        fn entry(
            &self,
            _data: &mut Vec<u8>,
            _entry_id: EntryId,
            _region_id: RegionId,
            _provider: &Provider,
        ) -> Result<Entry, Self::Error> {
            unreachable!()
        }
    }

    #[tokio::test]
    async fn test_raw_entry_reader() {
        let provider = Provider::raft_engine_provider(RegionId::new(1024, 1).as_u64());
        let expected_entries = vec![Entry::Naive(NaiveEntry {
            provider: provider.clone(),
            region_id: RegionId::new(1024, 1),
            entry_id: 1,
            data: vec![1],
        })];
        let store = MockLogStore {
            entries: expected_entries.clone(),
        };

        let reader = LogStoreRawEntryReader::new(Arc::new(store));
        let entries = reader
            .read(&provider, 0)
            .unwrap()
            .try_collect::<Vec<_>>()
            .await
            .unwrap();
        assert_eq!(expected_entries, entries);
    }

    #[tokio::test]
    async fn test_raw_entry_reader_filter() {
        let provider = Provider::raft_engine_provider(RegionId::new(1024, 1).as_u64());
        let all_entries = vec![
            Entry::Naive(NaiveEntry {
                provider: provider.clone(),
                region_id: RegionId::new(1024, 1),
                entry_id: 1,
                data: vec![1],
            }),
            Entry::Naive(NaiveEntry {
                provider: provider.clone(),
                region_id: RegionId::new(1024, 2),
                entry_id: 2,
                data: vec![2],
            }),
            Entry::Naive(NaiveEntry {
                provider: provider.clone(),
                region_id: RegionId::new(1024, 3),
                entry_id: 3,
                data: vec![3],
            }),
        ];
        let store = MockLogStore {
            entries: all_entries.clone(),
        };

        let expected_region_id = RegionId::new(1024, 3);
        let reader = RegionRawEntryReader::new(
            LogStoreRawEntryReader::new(Arc::new(store)),
            expected_region_id,
        );
        let entries = reader
            .read(&provider, 0)
            .unwrap()
            .try_collect::<Vec<_>>()
            .await
            .unwrap();
        assert_eq!(
            all_entries
                .into_iter()
                .filter(|entry| entry.region_id() == expected_region_id)
                .collect::<Vec<_>>(),
            entries
        );
    }
}