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
// Copyright Materialize, Inc. and contributors. All rights reserved.
//
// Use of this software is governed by the Business Source License
// included in the LICENSE file.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0.

//! In-memory implementations for testing and benchmarking.

use std::collections::BTreeMap;
use std::future::Future;
use std::pin::Pin;
use std::sync::{Arc, Mutex};
use std::task::{Context, Poll};

use anyhow::anyhow;
use async_trait::async_trait;
use bytes::Bytes;
use futures_util::{stream, StreamExt};
use mz_ore::bytes::SegmentedBytes;
use mz_ore::cast::CastFrom;

use crate::error::Error;
use crate::location::{
    Blob, BlobMetadata, CaSResult, Consensus, Determinate, ExternalError, ResultStream, SeqNo,
    VersionedData,
};

// A snapshot of the old tokio::task::yield_now() implementation, from before it
// had sneaky TLS shenangans.
//
// TODO: Move this into mz_ore somewhere so others can use it, too.
async fn yield_now() {
    struct YieldNow {
        yielded: bool,
    }

    impl Future for YieldNow {
        type Output = ();

        fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<()> {
            if self.yielded {
                return Poll::Ready(());
            }

            self.yielded = true;
            cx.waker().wake_by_ref();
            Poll::Pending
        }
    }

    YieldNow { yielded: false }.await
}

/// An in-memory representation of a set of [Log]s and [Blob]s that can be reused
/// across dataflows
#[cfg(test)]
#[derive(Debug)]
pub struct MemMultiRegistry {
    blob_by_path: BTreeMap<String, Arc<tokio::sync::Mutex<MemBlobCore>>>,
    tombstone: bool,
}

#[cfg(test)]
impl MemMultiRegistry {
    /// Constructs a new, empty [MemMultiRegistry].
    pub fn new(tombstone: bool) -> Self {
        MemMultiRegistry {
            blob_by_path: BTreeMap::new(),
            tombstone,
        }
    }

    /// Opens a [MemBlob] associated with `path`.
    ///
    /// TODO: Replace this with PersistClientCache once they're in the same
    /// crate.
    pub fn blob(&mut self, path: &str) -> MemBlob {
        if let Some(blob) = self.blob_by_path.get(path) {
            MemBlob::open(MemBlobConfig {
                core: Arc::clone(blob),
            })
        } else {
            let blob = Arc::new(tokio::sync::Mutex::new(MemBlobCore {
                dataz: Default::default(),
                tombstone: self.tombstone,
            }));
            self.blob_by_path
                .insert(path.to_string(), Arc::clone(&blob));
            MemBlob::open(MemBlobConfig { core: blob })
        }
    }
}

#[derive(Debug, Default)]
struct MemBlobCore {
    dataz: BTreeMap<String, (Bytes, bool)>,
    tombstone: bool,
}

impl MemBlobCore {
    fn get(&self, key: &str) -> Result<Option<Bytes>, ExternalError> {
        Ok(self
            .dataz
            .get(key)
            .and_then(|(x, exists)| exists.then(|| Bytes::clone(x))))
    }

    fn set(&mut self, key: &str, value: Bytes) -> Result<(), ExternalError> {
        self.dataz.insert(key.to_owned(), (value, true));
        Ok(())
    }

    fn list_keys_and_metadata(
        &self,
        key_prefix: &str,
        f: &mut (dyn FnMut(BlobMetadata) + Send + Sync),
    ) -> Result<(), ExternalError> {
        for (key, (value, exists)) in &self.dataz {
            if !*exists || !key.starts_with(key_prefix) {
                continue;
            }

            f(BlobMetadata {
                key,
                size_in_bytes: u64::cast_from(value.len()),
            });
        }

        Ok(())
    }

    fn delete(&mut self, key: &str) -> Result<Option<usize>, ExternalError> {
        let bytes = if self.tombstone {
            self.dataz.get_mut(key).and_then(|(x, exists)| {
                let deleted_size = exists.then(|| x.len());
                *exists = false;
                deleted_size
            })
        } else {
            self.dataz.remove(key).map(|(x, _)| x.len())
        };
        Ok(bytes)
    }

    fn restore(&mut self, key: &str) -> Result<(), ExternalError> {
        match self.dataz.get_mut(key) {
            None => Err(
                Determinate::new(anyhow!("unable to restore {key} from in-memory state")).into(),
            ),
            Some((_, exists)) => {
                *exists = true;
                Ok(())
            }
        }
    }
}

/// Configuration for opening a [MemBlob].
#[derive(Debug, Default)]
pub struct MemBlobConfig {
    core: Arc<tokio::sync::Mutex<MemBlobCore>>,
}

impl MemBlobConfig {
    /// Create a new instance.
    pub fn new(tombstone: bool) -> Self {
        Self {
            core: Arc::new(tokio::sync::Mutex::new(MemBlobCore {
                dataz: Default::default(),
                tombstone,
            })),
        }
    }
}

/// An in-memory implementation of [Blob].
#[derive(Debug)]
pub struct MemBlob {
    core: Arc<tokio::sync::Mutex<MemBlobCore>>,
}

impl MemBlob {
    /// Opens the given location for non-exclusive read-write access.
    pub fn open(config: MemBlobConfig) -> Self {
        MemBlob { core: config.core }
    }
}

#[async_trait]
impl Blob for MemBlob {
    async fn get(&self, key: &str) -> Result<Option<SegmentedBytes>, ExternalError> {
        // Yield to maximize our chances for getting interesting orderings.
        let () = yield_now().await;
        let maybe_bytes = self.core.lock().await.get(key)?;
        Ok(maybe_bytes.map(SegmentedBytes::from))
    }

    async fn list_keys_and_metadata(
        &self,
        key_prefix: &str,
        f: &mut (dyn FnMut(BlobMetadata) + Send + Sync),
    ) -> Result<(), ExternalError> {
        // Yield to maximize our chances for getting interesting orderings.
        let () = yield_now().await;
        self.core.lock().await.list_keys_and_metadata(key_prefix, f)
    }

    async fn set(&self, key: &str, value: Bytes) -> Result<(), ExternalError> {
        // Yield to maximize our chances for getting interesting orderings.
        let () = yield_now().await;
        // NB: This is always atomic, so we're free to ignore the atomic param.
        self.core.lock().await.set(key, value)
    }

    async fn delete(&self, key: &str) -> Result<Option<usize>, ExternalError> {
        // Yield to maximize our chances for getting interesting orderings.
        let () = yield_now().await;
        self.core.lock().await.delete(key)
    }

    async fn restore(&self, key: &str) -> Result<(), ExternalError> {
        // Yield to maximize our chances for getting interesting orderings.
        let () = yield_now().await;
        self.core.lock().await.restore(key)
    }
}

/// An in-memory implementation of [Consensus].
#[derive(Debug)]
pub struct MemConsensus {
    // TODO: This was intended to be a tokio::sync::Mutex but that seems to
    // regularly deadlock in the `concurrency` test.
    data: Arc<Mutex<BTreeMap<String, Vec<VersionedData>>>>,
}

impl Default for MemConsensus {
    fn default() -> Self {
        Self {
            data: Arc::new(Mutex::new(BTreeMap::new())),
        }
    }
}

impl MemConsensus {
    fn scan_store(
        store: &BTreeMap<String, Vec<VersionedData>>,
        key: &str,
        from: SeqNo,
        limit: usize,
    ) -> Result<Vec<VersionedData>, ExternalError> {
        let results = if let Some(values) = store.get(key) {
            let from_idx = values.partition_point(|x| x.seqno < from);
            let from_values = &values[from_idx..];
            let from_values = &from_values[..usize::min(limit, from_values.len())];
            from_values.to_vec()
        } else {
            Vec::new()
        };
        Ok(results)
    }
}

#[async_trait]
impl Consensus for MemConsensus {
    fn list_keys(&self) -> ResultStream<String> {
        // Yield to maximize our chances for getting interesting orderings.
        let store = self.data.lock().expect("lock poisoned");
        let keys: Vec<_> = store.keys().cloned().collect();
        Box::pin(stream::iter(keys).map(Ok))
    }

    async fn head(&self, key: &str) -> Result<Option<VersionedData>, ExternalError> {
        // Yield to maximize our chances for getting interesting orderings.
        let () = yield_now().await;
        let store = self.data.lock().map_err(Error::from)?;
        let values = match store.get(key) {
            None => return Ok(None),
            Some(values) => values,
        };

        Ok(values.last().cloned())
    }

    async fn compare_and_set(
        &self,
        key: &str,
        expected: Option<SeqNo>,
        new: VersionedData,
    ) -> Result<CaSResult, ExternalError> {
        // Yield to maximize our chances for getting interesting orderings.
        let () = yield_now().await;
        if let Some(expected) = expected {
            if new.seqno <= expected {
                return Err(ExternalError::from(
                        anyhow!("new seqno must be strictly greater than expected. Got new: {:?} expected: {:?}",
                                 new.seqno, expected)));
            }
        }

        if new.seqno.0 > i64::MAX.try_into().expect("i64::MAX known to fit in u64") {
            return Err(ExternalError::from(anyhow!(
                "sequence numbers must fit within [0, i64::MAX], received: {:?}",
                new.seqno
            )));
        }
        let mut store = self.data.lock().map_err(Error::from)?;

        let data = match store.get(key) {
            None => None,
            Some(values) => values.last(),
        };

        let seqno = data.as_ref().map(|data| data.seqno);

        if seqno != expected {
            return Ok(CaSResult::ExpectationMismatch);
        }

        store.entry(key.to_string()).or_default().push(new);

        Ok(CaSResult::Committed)
    }

    async fn scan(
        &self,
        key: &str,
        from: SeqNo,
        limit: usize,
    ) -> Result<Vec<VersionedData>, ExternalError> {
        // Yield to maximize our chances for getting interesting orderings.
        let () = yield_now().await;
        let store = self.data.lock().map_err(Error::from)?;
        Self::scan_store(&store, key, from, limit)
    }

    async fn truncate(&self, key: &str, seqno: SeqNo) -> Result<usize, ExternalError> {
        // Yield to maximize our chances for getting interesting orderings.
        let () = yield_now().await;
        let current = self.head(key).await?;
        if current.map_or(true, |data| data.seqno < seqno) {
            return Err(ExternalError::from(anyhow!(
                "upper bound too high for truncate: {:?}",
                seqno
            )));
        }

        let mut store = self.data.lock().map_err(Error::from)?;

        let mut deleted = 0;
        if let Some(values) = store.get_mut(key) {
            let count_before = values.len();
            values.retain(|val| val.seqno >= seqno);
            deleted += count_before - values.len();
        }

        Ok(deleted)
    }
}

#[cfg(test)]
mod tests {
    use crate::location::tests::{blob_impl_test, consensus_impl_test};

    use super::*;

    #[mz_ore::test(tokio::test)]
    #[cfg_attr(miri, ignore)] // unsupported operation: returning ready events from epoll_wait is not yet implemented
    async fn mem_blob() -> Result<(), ExternalError> {
        let registry = Arc::new(tokio::sync::Mutex::new(MemMultiRegistry::new(false)));
        blob_impl_test(move |path| {
            let path = path.to_owned();
            let registry = Arc::clone(&registry);
            async move { Ok(registry.lock().await.blob(&path)) }
        })
        .await?;

        let registry = Arc::new(tokio::sync::Mutex::new(MemMultiRegistry::new(true)));
        blob_impl_test(move |path| {
            let path = path.to_owned();
            let registry = Arc::clone(&registry);
            async move { Ok(registry.lock().await.blob(&path)) }
        })
        .await?;

        Ok(())
    }

    #[mz_ore::test(tokio::test)]
    #[cfg_attr(miri, ignore)] // unsupported operation: returning ready events from epoll_wait is not yet implemented
    async fn mem_consensus() -> Result<(), ExternalError> {
        consensus_impl_test(|| async { Ok(MemConsensus::default()) }).await
    }
}