mz_rocksdb/
lib.rs

1// Copyright Materialize, Inc. and contributors. All rights reserved.
2//
3// Use of this software is governed by the Business Source License
4// included in the LICENSE file.
5//
6// As of the Change Date specified in that file, in accordance with
7// the Business Source License, use of this software will be governed
8// by the Apache License, Version 2.0.
9
10//! An async wrapper around RocksDB, that does IO on a separate thread.
11//!
12//! This crate offers a limited API to communicate with RocksDB, to get
13//! the best performance possible (most importantly, by batching operations).
14//! Currently this API is only `upsert`, which replaces (or deletes) values for
15//! a set of keys, and returns the previous values.
16
17#![warn(missing_docs)]
18
19use std::convert::AsRef;
20use std::ops::Deref;
21use std::path::{Path, PathBuf};
22use std::time::Instant;
23
24use itertools::Itertools;
25use mz_ore::cast::CastFrom;
26use mz_ore::error::ErrorExt;
27use mz_ore::metrics::{DeleteOnDropCounter, DeleteOnDropHistogram};
28use mz_ore::retry::{Retry, RetryResult};
29use prometheus::core::AtomicU64;
30use rocksdb::merge_operator::MergeOperandsIter;
31use rocksdb::{DB, Env, Error as RocksDBError, ErrorKind, Options as RocksDBOptions, WriteOptions};
32use serde::Serialize;
33use serde::de::DeserializeOwned;
34use tokio::sync::{mpsc, oneshot};
35
36pub mod config;
37pub use config::{RocksDBConfig, RocksDBTuningParameters, defaults};
38
39use crate::config::WriteBufferManagerHandle;
40
41type Diff = mz_ore::Overflowing<i64>;
42
43/// An error using this RocksDB wrapper.
44#[derive(Debug, thiserror::Error)]
45pub enum Error {
46    /// An error from the underlying Kafka library.
47    #[error(transparent)]
48    RocksDB(#[from] RocksDBError),
49
50    /// Error when using the instance after RocksDB as errored
51    /// or been shutdown.
52    #[error("RocksDB thread has been shut down or errored")]
53    RocksDBThreadGoneAway,
54
55    /// Error decoding a value previously written.
56    #[error("failed to decode value")]
57    DecodeError(#[from] bincode::Error),
58
59    /// A tokio thread used by the implementation panicked.
60    #[error("tokio thread panicked")]
61    TokioPanic(#[from] tokio::task::JoinError),
62
63    /// A tokio thread used by the implementation panicked.
64    #[error("failed to cleanup in time")]
65    CleanupTimeout(#[from] tokio::time::error::Elapsed),
66
67    /// An error occured with a provided value.
68    #[error("error with value: {0}")]
69    ValueError(String),
70}
71
72/// An iterator over operand values to merge for a key in RocksDB.
73/// By convention the first value will be the existing value
74/// if it was present.
75pub struct ValueIterator<'a, O, V>
76where
77    O: bincode::Options + Copy + Send + Sync + 'static,
78    V: DeserializeOwned + Serialize + Send + Sync + 'static,
79{
80    iter: std::iter::Chain<std::option::IntoIter<&'a [u8]>, MergeOperandsIter<'a>>,
81    bincode: &'a O,
82    v: std::marker::PhantomData<V>,
83}
84
85impl<O, V> Iterator for ValueIterator<'_, O, V>
86where
87    O: bincode::Options + Copy + Send + Sync + 'static,
88    V: DeserializeOwned + Serialize + Send + Sync + 'static,
89{
90    type Item = V;
91
92    fn next(&mut self) -> Option<Self::Item> {
93        self.iter
94            .next()
95            .map(|v| self.bincode.deserialize(v).unwrap())
96    }
97}
98
99/// Helper type stub to satisfy generic bounds when initializing a `InstanceOptions` without a
100/// defined merge operator.
101pub type StubMergeOperator<V> =
102    fn(key: &[u8], operands: ValueIterator<bincode::DefaultOptions, V>) -> V;
103
104/// Fixed options to configure a [`RocksDBInstance`]. These are not tuning parameters,
105/// see the `config` modules for tuning. These are generally fixed within the binary.
106pub struct InstanceOptions<O, V, F> {
107    /// Whether or not to clear state at the instance
108    /// path before starting.
109    pub cleanup_on_new: bool,
110
111    /// If `cleanup_on_new`, how many times to try.
112    pub cleanup_tries: usize,
113
114    /// Whether or not to write writes
115    /// to the wal. This is not in `RocksDBTuningParameters` because it
116    /// applies to `WriteOptions` when creating `WriteBatch`es.
117    pub use_wal: bool,
118
119    /// A possibly shared RocksDB `Env`.
120    pub env: Env,
121
122    /// The bincode options to use for serializing and deserializing values.
123    pub bincode: O,
124
125    /// A merge operator to use for associative merges, if any. The first
126    /// item is the name of the operator to store in RocksDB for
127    /// compatibility checks, and the second is the merge function.
128    pub merge_operator: Option<(String, F)>,
129
130    v: std::marker::PhantomData<V>,
131}
132
133impl<O, V, F> InstanceOptions<O, V, F>
134where
135    O: bincode::Options + Copy + Send + Sync + 'static,
136    V: DeserializeOwned + Serialize + Send + Sync + 'static,
137    F: for<'a> Fn(&'a [u8], ValueIterator<'a, O, V>) -> V + Copy + Send + Sync + 'static,
138{
139    /// A new `Options` object with reasonable defaults.
140    pub fn new(
141        env: rocksdb::Env,
142        cleanup_tries: usize,
143        merge_operator: Option<(String, F)>,
144        bincode: O,
145    ) -> Self {
146        InstanceOptions {
147            cleanup_on_new: true,
148            cleanup_tries,
149            use_wal: false,
150            env,
151            merge_operator,
152            bincode,
153            v: std::marker::PhantomData,
154        }
155    }
156
157    fn as_rocksdb_options(
158        &self,
159        tuning_config: &RocksDBConfig,
160    ) -> (RocksDBOptions, Option<WriteBufferManagerHandle>) {
161        // Defaults + `create_if_missing`
162        let mut options = rocksdb::Options::default();
163        options.create_if_missing(true);
164
165        // Set the env first so tuning applies to the shared `Env`.
166        options.set_env(&self.env);
167
168        if let Some((fn_name, merge_fn)) = &self.merge_operator {
169            let bincode = self.bincode.clone();
170            let merge_fn = merge_fn.clone();
171            // We use an associative merge operator which is used for both full/partial merges
172            // since the value type is always the same for puts and merges.
173            // See `https://github.com/facebook/rocksdb/wiki/Merge-Operator#associativity-vs-non-associativity`
174            // for more info.
175            options.set_merge_operator_associative(fn_name, move |key, existing, operands| {
176                let operands = ValueIterator {
177                    iter: existing.into_iter().chain(operands.iter()),
178                    bincode: &bincode,
179                    v: std::marker::PhantomData::<V>,
180                };
181                let result = merge_fn(key, operands);
182                // NOTE: While the API specifies the return type as Option<Vec<u8>>, returning a None
183                // will cause rocksdb to throw a corruption error and SIGABRT the process.
184                Some(bincode.serialize(&result).unwrap())
185            });
186        }
187
188        let write_buffer_handle = config::apply_to_options(tuning_config, &mut options);
189        // Returns the rocksdb options and an optional `WriteBufferManagerHandle`
190        // if write buffer manager was enabled in the configs.
191        (options, write_buffer_handle)
192    }
193
194    fn as_rocksdb_write_options(&self) -> WriteOptions {
195        let mut wo = rocksdb::WriteOptions::new();
196        wo.disable_wal(!self.use_wal);
197        wo
198    }
199}
200
201/// Shared metrics about an instances usage of RocksDB. User-provided
202/// so the user can choose the labels.
203pub struct RocksDBSharedMetrics {
204    /// Latency of multi_gets, in fractional seconds.
205    pub multi_get_latency: DeleteOnDropHistogram<Vec<String>>,
206    /// Latency of write batch writes, in fractional seconds.
207    pub multi_put_latency: DeleteOnDropHistogram<Vec<String>>,
208}
209
210/// Worker metrics about an instances usage of RocksDB. User-provided
211/// so the user can choose the labels.
212pub struct RocksDBInstanceMetrics {
213    /// Size of multi_get batches.
214    pub multi_get_size: DeleteOnDropCounter<AtomicU64, Vec<String>>,
215    /// Size of multi_get non-empty results.
216    pub multi_get_result_count: DeleteOnDropCounter<AtomicU64, Vec<String>>,
217    /// Total size of bytes returned in the result
218    pub multi_get_result_bytes: DeleteOnDropCounter<AtomicU64, Vec<String>>,
219    /// The number of calls to rocksdb multi_get
220    pub multi_get_count: DeleteOnDropCounter<AtomicU64, Vec<String>>,
221    /// The number of calls to rocksdb multi_put
222    pub multi_put_count: DeleteOnDropCounter<AtomicU64, Vec<String>>,
223    /// Size of write batches.
224    pub multi_put_size: DeleteOnDropCounter<AtomicU64, Vec<String>>,
225}
226
227/// The result type for `multi_get`.
228#[derive(Default, Debug)]
229pub struct MultiGetResult {
230    /// The number of keys we fetched.
231    pub processed_gets: u64,
232    /// The total size of values fetched.
233    pub processed_gets_size: u64,
234    /// The number of records returns.
235    pub returned_gets: u64,
236}
237
238/// The result type for individual gets.
239#[derive(Debug, Default, Clone)]
240pub struct GetResult<V> {
241    /// The previous value, if there was one.
242    pub value: V,
243    /// The size of `value` as persisted, if there was one.
244    /// Useful for users keeping track of statistics.
245    pub size: u64,
246}
247
248/// The result type for `multi_update`.
249#[derive(Default, Debug)]
250pub struct MultiUpdateResult {
251    /// The number of puts, merges, and deletes.
252    pub processed_updates: u64,
253    /// The total size of values we wrote to the database.
254    /// Does not contain any information about deletes.
255    pub size_written: u64,
256    /// The 'diff' size of the values we wrote to the database,
257    /// returned when the `MultiUpdate` command included a multiplier 'diff'
258    /// for at least one update value.
259    pub size_diff: Option<Diff>,
260}
261
262/// The type of update to perform on a key.
263#[derive(Debug)]
264pub enum KeyUpdate<V> {
265    /// Put a value into the database under the given key.
266    Put(V),
267    /// Merge the database value with the given value.
268    /// Will error if the merge operator is not set.
269    Merge(V),
270    /// Delete the value from the database.
271    Delete,
272}
273
274#[derive(Debug)]
275enum Command<K, V> {
276    MultiGet {
277        batch: Vec<K>,
278        // Scratch vector to return results in.
279        results_scratch: Vec<Option<GetResult<V>>>,
280        response_sender: oneshot::Sender<
281            Result<
282                (
283                    MultiGetResult,
284                    // The batch scratch vector being given back.
285                    Vec<K>,
286                    Vec<Option<GetResult<V>>>,
287                ),
288                Error,
289            >,
290        >,
291    },
292    MultiUpdate {
293        // The batch of updates to perform. The 3rd item in each tuple is an optional diff
294        // multiplier that when present, will be multiplied by the size of the encoded
295        // value written to the database and summed into the `MultiUpdateResult::size_diff` field.
296        batch: Vec<(K, KeyUpdate<V>, Option<Diff>)>,
297        // Scratch vector to return results in.
298        response_sender: oneshot::Sender<
299            Result<(MultiUpdateResult, Vec<(K, KeyUpdate<V>, Option<Diff>)>), Error>,
300        >,
301    },
302    Shutdown {
303        done_sender: oneshot::Sender<()>,
304    },
305    ManualCompaction {
306        done_sender: oneshot::Sender<()>,
307    },
308}
309
310/// An async wrapper around RocksDB.
311pub struct RocksDBInstance<K, V> {
312    tx: mpsc::Sender<Command<K, V>>,
313
314    // Scratch vector to send keys to the RocksDB thread
315    // during `MultiGet`.
316    multi_get_scratch: Vec<K>,
317
318    // Scratch vector to return results from the RocksDB thread
319    // during `MultiGet`.
320    multi_get_results_scratch: Vec<Option<GetResult<V>>>,
321
322    // Scratch vector to send updates to the RocksDB thread
323    // during `MultiUpdate`.
324    multi_update_scratch: Vec<(K, KeyUpdate<V>, Option<Diff>)>,
325
326    // Configuration that can change dynamically.
327    dynamic_config: config::RocksDBDynamicConfig,
328
329    /// Whether this instance supports merge operations (whether a
330    /// merge operator was set at creation time)
331    pub supports_merges: bool,
332}
333
334impl<K, V> RocksDBInstance<K, V>
335where
336    K: AsRef<[u8]> + Send + Sync + 'static,
337    V: Serialize + DeserializeOwned + Send + Sync + 'static,
338{
339    /// Start a new RocksDB instance at the path, using
340    /// the `Options` and `RocksDBTuningParameters` to
341    /// configure the instance.
342    ///
343    /// `metrics` is a set of metric types that this type will keep
344    /// up to date. `enc_opts` is the `bincode` options used to
345    /// serialize and deserialize the keys and values.
346    pub async fn new<M, O, IM, F>(
347        instance_path: &Path,
348        options: InstanceOptions<O, V, F>,
349        tuning_config: RocksDBConfig,
350        shared_metrics: M,
351        instance_metrics: IM,
352    ) -> Result<Self, Error>
353    where
354        O: bincode::Options + Copy + Send + Sync + 'static,
355        M: Deref<Target = RocksDBSharedMetrics> + Send + 'static,
356        IM: Deref<Target = RocksDBInstanceMetrics> + Send + 'static,
357        F: for<'a> Fn(&'a [u8], ValueIterator<'a, O, V>) -> V + Copy + Send + Sync + 'static,
358    {
359        let dynamic_config = tuning_config.dynamic.clone();
360        let supports_merges = options.merge_operator.is_some();
361        if options.cleanup_on_new && instance_path.exists() {
362            let instance_path_owned = instance_path.to_owned();
363
364            // We require that cleanup of the DB succeeds. Otherwise, we could open a DB with old,
365            // incorrect data. Because of races with dataflow shutdown, we retry a few times here.
366            // 1s with a 2x backoff is ~30s after 5 tries.
367            //
368            // TODO(guswynn): remove this when we can wait on dataflow cleanup asynchronously in
369            // the controller.
370            let retry = mz_ore::retry::Retry::default()
371                .max_tries(options.cleanup_tries)
372                // Large DB's could take multiple seconds to run.
373                .initial_backoff(std::time::Duration::from_secs(1));
374
375            retry
376                .retry_async_canceling(|_rs| async {
377                    let instance_path_owned = instance_path_owned.clone();
378                    mz_ore::task::spawn_blocking(
379                        || {
380                            format!(
381                                "RocksDB instance at {}: cleanup on creation",
382                                instance_path.display()
383                            )
384                        },
385                        move || {
386                            if let Err(e) =
387                                DB::destroy(&RocksDBOptions::default(), &*instance_path_owned)
388                            {
389                                tracing::warn!(
390                                    "failed to cleanup rocksdb dir on creation {}: {}",
391                                    instance_path_owned.display(),
392                                    e.display_with_causes(),
393                                );
394                                Err(Error::from(e))
395                            } else {
396                                Ok(())
397                            }
398                        },
399                    )
400                    .await?
401                })
402                .await?;
403        }
404
405        // The buffer can be small here, as all interactions with it take `&mut self`.
406        let (tx, rx): (mpsc::Sender<Command<K, V>>, _) = mpsc::channel(10);
407
408        let instance_path = instance_path.to_owned();
409        let (creation_error_tx, creation_error_rx) = oneshot::channel();
410        std::thread::spawn(move || {
411            rocksdb_core_loop(
412                options,
413                tuning_config,
414                instance_path,
415                rx,
416                shared_metrics,
417                instance_metrics,
418                creation_error_tx,
419            )
420        });
421
422        if let Ok(creation_error) = creation_error_rx.await {
423            return Err(creation_error);
424        }
425
426        Ok(Self {
427            tx,
428            multi_get_scratch: Vec::new(),
429            multi_get_results_scratch: Vec::new(),
430            multi_update_scratch: Vec::new(),
431            dynamic_config,
432            supports_merges,
433        })
434    }
435
436    /// For each _unique_ key in `gets`, place the stored value (if any) in `results_out`.
437    ///
438    /// Panics if `gets` and `results_out` are not the same length.
439    pub async fn multi_get<'r, G, R, Ret, Placement>(
440        &mut self,
441        gets: G,
442        results_out: R,
443        placement: Placement,
444    ) -> Result<MultiGetResult, Error>
445    where
446        G: IntoIterator<Item = K>,
447        R: IntoIterator<Item = &'r mut Ret>,
448        Ret: 'r,
449        Placement: Fn(Option<GetResult<V>>) -> Ret,
450    {
451        let batch_size = self.dynamic_config.batch_size();
452        let mut stats = MultiGetResult::default();
453
454        let mut gets = gets.into_iter().peekable();
455        if gets.peek().is_some() {
456            let gets = gets.chunks(batch_size);
457            let results_out = results_out.into_iter().chunks(batch_size);
458
459            for (gets, results_out) in gets.into_iter().zip_eq(results_out.into_iter()) {
460                let ret = self.multi_get_inner(gets, results_out, &placement).await?;
461                stats.processed_gets += ret.processed_gets;
462            }
463        }
464
465        Ok(stats)
466    }
467
468    async fn multi_get_inner<'r, G, R, Ret, Placement>(
469        &mut self,
470        gets: G,
471        results_out: R,
472        placement: &Placement,
473    ) -> Result<MultiGetResult, Error>
474    where
475        G: IntoIterator<Item = K>,
476        R: IntoIterator<Item = &'r mut Ret>,
477        Ret: 'r,
478        Placement: Fn(Option<GetResult<V>>) -> Ret,
479    {
480        let mut multi_get_vec = std::mem::take(&mut self.multi_get_scratch);
481        let mut results_vec = std::mem::take(&mut self.multi_get_results_scratch);
482        multi_get_vec.clear();
483        results_vec.clear();
484
485        multi_get_vec.extend(gets);
486        if multi_get_vec.is_empty() {
487            self.multi_get_scratch = multi_get_vec;
488            self.multi_get_results_scratch = results_vec;
489            return Ok(MultiGetResult {
490                processed_gets: 0,
491                processed_gets_size: 0,
492                returned_gets: 0,
493            });
494        }
495
496        let (tx, rx) = oneshot::channel();
497        self.tx
498            .send(Command::MultiGet {
499                batch: multi_get_vec,
500                results_scratch: results_vec,
501                response_sender: tx,
502            })
503            .await
504            .map_err(|_| Error::RocksDBThreadGoneAway)?;
505
506        // We also unwrap all rocksdb errors here.
507        match rx.await.map_err(|_| Error::RocksDBThreadGoneAway)? {
508            Ok((ret, get_scratch, mut results_scratch)) => {
509                for (place, get) in results_out.into_iter().zip_eq(results_scratch.drain(..)) {
510                    *place = placement(get);
511                }
512                self.multi_get_scratch = get_scratch;
513                self.multi_get_results_scratch = results_scratch;
514                Ok(ret)
515            }
516            Err(e) => {
517                // Note we don't attempt to preserve the scratch allocations here.
518                Err(e)
519            }
520        }
521    }
522
523    /// For each key in puts, store the given value, or delete it if
524    /// the value is `None`. If the same `key` appears multiple times,
525    /// the last value for the key wins.
526    /// The third item in each tuple is an optional diff multiplier that when present,
527    /// will be multiplied by the size of the encoded value written to the database and
528    /// summed into the `MultiUpdateResult::size_diff` field.
529    pub async fn multi_update<P>(&mut self, puts: P) -> Result<MultiUpdateResult, Error>
530    where
531        P: IntoIterator<Item = (K, KeyUpdate<V>, Option<Diff>)>,
532    {
533        let batch_size = self.dynamic_config.batch_size();
534        let mut stats = MultiUpdateResult::default();
535
536        let mut puts = puts.into_iter().peekable();
537        if puts.peek().is_some() {
538            let puts = puts.chunks(batch_size);
539
540            for puts in puts.into_iter() {
541                let ret = self.multi_update_inner(puts).await?;
542                stats.processed_updates += ret.processed_updates;
543                stats.size_written += ret.size_written;
544                if let Some(diff) = ret.size_diff {
545                    stats.size_diff = Some(stats.size_diff.unwrap_or(Diff::ZERO) + diff);
546                }
547            }
548        }
549
550        Ok(stats)
551    }
552
553    async fn multi_update_inner<P>(&mut self, updates: P) -> Result<MultiUpdateResult, Error>
554    where
555        P: IntoIterator<Item = (K, KeyUpdate<V>, Option<Diff>)>,
556    {
557        let mut multi_put_vec = std::mem::take(&mut self.multi_update_scratch);
558        multi_put_vec.clear();
559
560        multi_put_vec.extend(updates);
561        if multi_put_vec.is_empty() {
562            self.multi_update_scratch = multi_put_vec;
563            return Ok(MultiUpdateResult {
564                processed_updates: 0,
565                size_written: 0,
566                size_diff: None,
567            });
568        }
569
570        let (tx, rx) = oneshot::channel();
571        self.tx
572            .send(Command::MultiUpdate {
573                batch: multi_put_vec,
574                response_sender: tx,
575            })
576            .await
577            .map_err(|_| Error::RocksDBThreadGoneAway)?;
578
579        // We also unwrap all rocksdb errors here.
580        match rx.await.map_err(|_| Error::RocksDBThreadGoneAway)? {
581            Ok((ret, scratch)) => {
582                self.multi_update_scratch = scratch;
583                Ok(ret)
584            }
585            Err(e) => {
586                // Note we don't attempt to preserve the allocation here.
587                Err(e)
588            }
589        }
590    }
591
592    /// Trigger manual compaction of the RocksDB instance.
593    pub async fn manual_compaction(&self) -> Result<(), Error> {
594        let (tx, rx) = oneshot::channel();
595        self.tx
596            .send(Command::ManualCompaction { done_sender: tx })
597            .await
598            .map_err(|_| Error::RocksDBThreadGoneAway)?;
599
600        rx.await.map_err(|_| Error::RocksDBThreadGoneAway)
601    }
602
603    /// Gracefully shut down RocksDB. Can error if the instance
604    /// is already shut down or errored.
605    pub async fn close(self) -> Result<(), Error> {
606        let (tx, rx) = oneshot::channel();
607        self.tx
608            .send(Command::Shutdown { done_sender: tx })
609            .await
610            .map_err(|_| Error::RocksDBThreadGoneAway)?;
611
612        let _ = rx.await;
613
614        Ok(())
615    }
616}
617
618fn rocksdb_core_loop<K, V, M, O, IM, F>(
619    options: InstanceOptions<O, V, F>,
620    tuning_config: RocksDBConfig,
621    instance_path: PathBuf,
622    mut cmd_rx: mpsc::Receiver<Command<K, V>>,
623    shared_metrics: M,
624    instance_metrics: IM,
625    creation_error_tx: oneshot::Sender<Error>,
626) where
627    K: AsRef<[u8]> + Send + Sync + 'static,
628    V: Serialize + DeserializeOwned + Send + Sync + 'static,
629    M: Deref<Target = RocksDBSharedMetrics> + Send + 'static,
630    O: bincode::Options + Copy + Send + Sync + 'static,
631    F: for<'a> Fn(&'a [u8], ValueIterator<'a, O, V>) -> V + Send + Sync + Copy + 'static,
632    IM: Deref<Target = RocksDBInstanceMetrics> + Send + 'static,
633{
634    let retry_max_duration = tuning_config.retry_max_duration;
635
636    // Handle to an optional reference of a write buffer manager which
637    // should be valid till the rocksdb thread is running.
638    // The shared write buffer manager will be cleaned up if all
639    // the handles are dropped across all the rocksdb instances.
640    let (rocksdb_options, write_buffer_handle) = options.as_rocksdb_options(&tuning_config);
641    tracing::info!(
642        "Starting rocksdb at {:?} with write_buffer_manager: {:?}",
643        instance_path,
644        write_buffer_handle
645    );
646
647    let retry_result = Retry::default()
648        .max_duration(retry_max_duration)
649        .retry(|_| match DB::open(&rocksdb_options, &instance_path) {
650            Ok(db) => RetryResult::Ok(db),
651            Err(e) => match e.kind() {
652                ErrorKind::TryAgain => RetryResult::RetryableErr(Error::RocksDB(e)),
653                _ => RetryResult::FatalErr(Error::RocksDB(e)),
654            },
655        });
656
657    let db: DB = match retry_result {
658        Ok(db) => {
659            drop(creation_error_tx);
660            db
661        }
662        Err(e) => {
663            // Communicate the error back to `new`.
664            let _ = creation_error_tx.send(e);
665            return;
666        }
667    };
668
669    let mut encoded_batch_buffers: Vec<Option<Vec<u8>>> = Vec::new();
670    let mut encoded_batch: Vec<(K, KeyUpdate<Vec<u8>>)> = Vec::new();
671
672    let wo = options.as_rocksdb_write_options();
673
674    while let Some(cmd) = cmd_rx.blocking_recv() {
675        match cmd {
676            Command::Shutdown { done_sender } => {
677                db.cancel_all_background_work(true);
678                drop(db);
679                drop(write_buffer_handle);
680                let _ = done_sender.send(());
681                return;
682            }
683            Command::ManualCompaction { done_sender } => {
684                // Compact the full key-range.
685                db.compact_range::<&[u8], &[u8]>(None, None);
686                let _ = done_sender.send(());
687            }
688            Command::MultiGet {
689                mut batch,
690                mut results_scratch,
691                response_sender,
692            } => {
693                let batch_size = batch.len();
694
695                // Perform the multi_get and record metrics, if there wasn't an error.
696                let now = Instant::now();
697                let retry_result = Retry::default()
698                    .max_duration(retry_max_duration)
699                    .retry(|_| {
700                        let gets = db.multi_get(batch.iter());
701                        let latency = now.elapsed();
702
703                        let gets: Result<Vec<_>, _> = gets.into_iter().collect();
704                        match gets {
705                            Ok(gets) => {
706                                shared_metrics
707                                    .multi_get_latency
708                                    .observe(latency.as_secs_f64());
709                                instance_metrics
710                                    .multi_get_size
711                                    .inc_by(batch_size.try_into().unwrap());
712                                instance_metrics.multi_get_count.inc();
713
714                                RetryResult::Ok(gets)
715                            }
716                            Err(e) => match e.kind() {
717                                ErrorKind::TryAgain => RetryResult::RetryableErr(Error::RocksDB(e)),
718                                _ => RetryResult::FatalErr(Error::RocksDB(e)),
719                            },
720                        }
721                    });
722
723                let _ = match retry_result {
724                    Ok(gets) => {
725                        let processed_gets: u64 = gets.len().try_into().unwrap();
726                        let mut processed_gets_size = 0;
727                        let mut returned_gets: u64 = 0;
728                        for previous_value in gets {
729                            let get_result = match previous_value {
730                                Some(previous_value) => {
731                                    match options.bincode.deserialize(&previous_value) {
732                                        Ok(value) => {
733                                            let size = u64::cast_from(previous_value.len());
734                                            processed_gets_size += size;
735                                            returned_gets += 1;
736                                            Some(GetResult { value, size })
737                                        }
738                                        Err(e) => {
739                                            let _ =
740                                                response_sender.send(Err(Error::DecodeError(e)));
741                                            return;
742                                        }
743                                    }
744                                }
745                                None => None,
746                            };
747                            results_scratch.push(get_result);
748                        }
749
750                        instance_metrics
751                            .multi_get_result_count
752                            .inc_by(returned_gets);
753                        instance_metrics
754                            .multi_get_result_bytes
755                            .inc_by(processed_gets_size);
756                        batch.clear();
757                        response_sender.send(Ok((
758                            MultiGetResult {
759                                processed_gets,
760                                processed_gets_size,
761                                returned_gets,
762                            },
763                            batch,
764                            results_scratch,
765                        )))
766                    }
767                    Err(e) => response_sender.send(Err(e)),
768                };
769            }
770            Command::MultiUpdate {
771                mut batch,
772                response_sender,
773            } => {
774                let batch_size = batch.len();
775
776                let mut ret = MultiUpdateResult {
777                    processed_updates: 0,
778                    size_written: 0,
779                    size_diff: None,
780                };
781
782                // initialize and push values into the buffer to match the batch size
783                let buf_size = encoded_batch_buffers.len();
784                for _ in buf_size..batch_size {
785                    encoded_batch_buffers.push(Some(Vec::new()));
786                }
787                // shrinking the buffers in case the scratch buffer's capacity is significantly
788                // more than the size of batch
789                if tuning_config.shrink_buffers_by_ratio > 0 {
790                    let reduced_capacity =
791                        encoded_batch_buffers.capacity() / tuning_config.shrink_buffers_by_ratio;
792                    if reduced_capacity > batch_size {
793                        encoded_batch_buffers.truncate(reduced_capacity);
794                        encoded_batch_buffers.shrink_to(reduced_capacity);
795
796                        encoded_batch.truncate(reduced_capacity);
797                        encoded_batch.shrink_to(reduced_capacity);
798                    }
799                }
800                assert!(encoded_batch_buffers.len() >= batch_size);
801
802                // TODO(guswynn): sort by key before writing.
803                for ((key, value, diff), encode_buf) in
804                    batch.drain(..).zip(encoded_batch_buffers.iter_mut())
805                {
806                    ret.processed_updates += 1;
807
808                    match &value {
809                        update_type @ (KeyUpdate::Put(update) | KeyUpdate::Merge(update)) => {
810                            let mut encode_buf =
811                                encode_buf.take().expect("encode_buf should not be empty");
812                            encode_buf.clear();
813                            match options
814                                .bincode
815                                .serialize_into::<&mut Vec<u8>, _>(&mut encode_buf, update)
816                            {
817                                Ok(()) => {
818                                    ret.size_written += u64::cast_from(encode_buf.len());
819                                    // calculate the diff size if the diff multiplier is present
820                                    if let Some(diff) = diff {
821                                        let encoded_len = Diff::try_from(encode_buf.len())
822                                            .expect("less than i64 size");
823                                        ret.size_diff = Some(
824                                            ret.size_diff.unwrap_or(Diff::ZERO)
825                                                + (diff * encoded_len),
826                                        );
827                                    }
828                                }
829                                Err(e) => {
830                                    let _ = response_sender.send(Err(Error::DecodeError(e)));
831                                    return;
832                                }
833                            };
834                            if matches!(update_type, KeyUpdate::Put(_)) {
835                                encoded_batch.push((key, KeyUpdate::Put(encode_buf)));
836                            } else {
837                                encoded_batch.push((key, KeyUpdate::Merge(encode_buf)));
838                            }
839                        }
840                        KeyUpdate::Delete => encoded_batch.push((key, KeyUpdate::Delete)),
841                    }
842                }
843                // Perform the multi_update and record metrics, if there wasn't an error.
844                let now = Instant::now();
845                let retry_result = Retry::default()
846                    .max_duration(retry_max_duration)
847                    .retry(|_| {
848                        let mut writes = rocksdb::WriteBatch::default();
849
850                        for (key, value) in encoded_batch.iter() {
851                            match value {
852                                KeyUpdate::Put(update) => writes.put(key, update),
853                                KeyUpdate::Merge(update) => writes.merge(key, update),
854                                KeyUpdate::Delete => writes.delete(key),
855                            }
856                        }
857
858                        match db.write_opt(writes, &wo) {
859                            Ok(()) => {
860                                let latency = now.elapsed();
861                                shared_metrics
862                                    .multi_put_latency
863                                    .observe(latency.as_secs_f64());
864                                instance_metrics
865                                    .multi_put_size
866                                    .inc_by(batch_size.try_into().unwrap());
867                                instance_metrics.multi_put_count.inc();
868                                RetryResult::Ok(())
869                            }
870                            Err(e) => match e.kind() {
871                                ErrorKind::TryAgain => RetryResult::RetryableErr(Error::RocksDB(e)),
872                                _ => RetryResult::FatalErr(Error::RocksDB(e)),
873                            },
874                        }
875                    });
876
877                // put back the values in the buffer so we don't lose allocation
878                for (i, (_, encoded_buffer)) in encoded_batch.drain(..).enumerate() {
879                    if let KeyUpdate::Put(encoded_buffer) | KeyUpdate::Merge(encoded_buffer) =
880                        encoded_buffer
881                    {
882                        encoded_batch_buffers[i] = Some(encoded_buffer);
883                    }
884                }
885
886                let _ = match retry_result {
887                    Ok(()) => {
888                        batch.clear();
889                        response_sender.send(Ok((ret, batch)))
890                    }
891                    Err(e) => response_sender.send(Err(e)),
892                };
893            }
894        }
895    }
896    // Gracefully cleanup if the `RocksDBInstance` has gone away.
897    db.cancel_all_background_work(true);
898    drop(db);
899
900    // Note that we don't retry, as we already may race here with a source being restarted.
901    if let Err(e) = DB::destroy(&RocksDBOptions::default(), &*instance_path) {
902        tracing::warn!(
903            "failed to cleanup rocksdb dir at {}: {}",
904            instance_path.display(),
905            e.display_with_causes(),
906        );
907    }
908}