1#![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#[derive(Debug, thiserror::Error)]
45pub enum Error {
46 #[error(transparent)]
48 RocksDB(#[from] RocksDBError),
49
50 #[error("RocksDB thread has been shut down or errored")]
53 RocksDBThreadGoneAway,
54
55 #[error("failed to decode value")]
57 DecodeError(#[from] bincode::Error),
58
59 #[error("tokio thread panicked")]
61 TokioPanic(#[from] tokio::task::JoinError),
62
63 #[error("failed to cleanup in time")]
65 CleanupTimeout(#[from] tokio::time::error::Elapsed),
66
67 #[error("error with value: {0}")]
69 ValueError(String),
70}
71
72pub 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
99pub type StubMergeOperator<V> =
102 fn(key: &[u8], operands: ValueIterator<bincode::DefaultOptions, V>) -> V;
103
104pub struct InstanceOptions<O, V, F> {
107 pub cleanup_on_new: bool,
110
111 pub cleanup_tries: usize,
113
114 pub use_wal: bool,
118
119 pub env: Env,
121
122 pub bincode: O,
124
125 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 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 let mut options = rocksdb::Options::default();
163 options.create_if_missing(true);
164
165 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 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 Some(bincode.serialize(&result).unwrap())
185 });
186 }
187
188 let write_buffer_handle = config::apply_to_options(tuning_config, &mut options);
189 (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
201pub struct RocksDBSharedMetrics {
204 pub multi_get_latency: DeleteOnDropHistogram<Vec<String>>,
206 pub multi_put_latency: DeleteOnDropHistogram<Vec<String>>,
208}
209
210pub struct RocksDBInstanceMetrics {
213 pub multi_get_size: DeleteOnDropCounter<AtomicU64, Vec<String>>,
215 pub multi_get_result_count: DeleteOnDropCounter<AtomicU64, Vec<String>>,
217 pub multi_get_result_bytes: DeleteOnDropCounter<AtomicU64, Vec<String>>,
219 pub multi_get_count: DeleteOnDropCounter<AtomicU64, Vec<String>>,
221 pub multi_put_count: DeleteOnDropCounter<AtomicU64, Vec<String>>,
223 pub multi_put_size: DeleteOnDropCounter<AtomicU64, Vec<String>>,
225}
226
227#[derive(Default, Debug)]
229pub struct MultiGetResult {
230 pub processed_gets: u64,
232 pub processed_gets_size: u64,
234 pub returned_gets: u64,
236}
237
238#[derive(Debug, Default, Clone)]
240pub struct GetResult<V> {
241 pub value: V,
243 pub size: u64,
246}
247
248#[derive(Default, Debug)]
250pub struct MultiUpdateResult {
251 pub processed_updates: u64,
253 pub size_written: u64,
256 pub size_diff: Option<Diff>,
260}
261
262#[derive(Debug)]
264pub enum KeyUpdate<V> {
265 Put(V),
267 Merge(V),
270 Delete,
272}
273
274#[derive(Debug)]
275enum Command<K, V> {
276 MultiGet {
277 batch: Vec<K>,
278 results_scratch: Vec<Option<GetResult<V>>>,
280 response_sender: oneshot::Sender<
281 Result<
282 (
283 MultiGetResult,
284 Vec<K>,
286 Vec<Option<GetResult<V>>>,
287 ),
288 Error,
289 >,
290 >,
291 },
292 MultiUpdate {
293 batch: Vec<(K, KeyUpdate<V>, Option<Diff>)>,
297 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
310pub struct RocksDBInstance<K, V> {
312 tx: mpsc::Sender<Command<K, V>>,
313
314 multi_get_scratch: Vec<K>,
317
318 multi_get_results_scratch: Vec<Option<GetResult<V>>>,
321
322 multi_update_scratch: Vec<(K, KeyUpdate<V>, Option<Diff>)>,
325
326 dynamic_config: config::RocksDBDynamicConfig,
328
329 pub supports_merges: bool,
332
333 handle: Option<std::thread::JoinHandle<()>>,
335}
336
337impl<K, V> RocksDBInstance<K, V>
338where
339 K: AsRef<[u8]> + Send + Sync + 'static,
340 V: Serialize + DeserializeOwned + Send + Sync + 'static,
341{
342 pub fn new<M, O, IM, F>(
350 instance_path: &Path,
351 options: InstanceOptions<O, V, F>,
352 tuning_config: RocksDBConfig,
353 shared_metrics: M,
354 instance_metrics: IM,
355 ) -> Result<Self, Error>
356 where
357 O: bincode::Options + Copy + Send + Sync + 'static,
358 M: Deref<Target = RocksDBSharedMetrics> + Send + 'static,
359 IM: Deref<Target = RocksDBInstanceMetrics> + Send + 'static,
360 F: for<'a> Fn(&'a [u8], ValueIterator<'a, O, V>) -> V + Copy + Send + Sync + 'static,
361 {
362 let dynamic_config = tuning_config.dynamic.clone();
363 let supports_merges = options.merge_operator.is_some();
364
365 let (tx, rx): (mpsc::Sender<Command<K, V>>, _) = mpsc::channel(10);
367
368 let instance_path = instance_path.to_owned();
369 let handle = std::thread::spawn(move || {
375 rocksdb_core_loop(
376 options,
377 tuning_config,
378 instance_path,
379 rx,
380 shared_metrics,
381 instance_metrics,
382 )
383 });
384
385 Ok(Self {
386 tx,
387 multi_get_scratch: Vec::new(),
388 multi_get_results_scratch: Vec::new(),
389 multi_update_scratch: Vec::new(),
390 dynamic_config,
391 supports_merges,
392 handle: Some(handle),
393 })
394 }
395
396 pub fn take_core_loop_handle(&mut self) -> Option<std::thread::JoinHandle<()>> {
399 self.handle.take()
400 }
401
402 pub async fn multi_get<'r, G, R, Ret, Placement>(
406 &mut self,
407 gets: G,
408 results_out: R,
409 placement: Placement,
410 ) -> Result<MultiGetResult, Error>
411 where
412 G: IntoIterator<Item = K>,
413 R: IntoIterator<Item = &'r mut Ret>,
414 Ret: 'r,
415 Placement: Fn(Option<GetResult<V>>) -> Ret,
416 {
417 let batch_size = self.dynamic_config.batch_size();
418 let mut stats = MultiGetResult::default();
419
420 let mut gets = gets.into_iter().peekable();
421 if gets.peek().is_some() {
422 let gets = gets.chunks(batch_size);
423 let results_out = results_out.into_iter().chunks(batch_size);
424
425 for (gets, results_out) in gets.into_iter().zip_eq(results_out.into_iter()) {
426 let ret = self.multi_get_inner(gets, results_out, &placement).await?;
427 stats.processed_gets += ret.processed_gets;
428 }
429 }
430
431 Ok(stats)
432 }
433
434 async fn multi_get_inner<'r, G, R, Ret, Placement>(
435 &mut self,
436 gets: G,
437 results_out: R,
438 placement: &Placement,
439 ) -> Result<MultiGetResult, Error>
440 where
441 G: IntoIterator<Item = K>,
442 R: IntoIterator<Item = &'r mut Ret>,
443 Ret: 'r,
444 Placement: Fn(Option<GetResult<V>>) -> Ret,
445 {
446 let mut multi_get_vec = std::mem::take(&mut self.multi_get_scratch);
447 let mut results_vec = std::mem::take(&mut self.multi_get_results_scratch);
448 multi_get_vec.clear();
449 results_vec.clear();
450
451 multi_get_vec.extend(gets);
452 if multi_get_vec.is_empty() {
453 self.multi_get_scratch = multi_get_vec;
454 self.multi_get_results_scratch = results_vec;
455 return Ok(MultiGetResult {
456 processed_gets: 0,
457 processed_gets_size: 0,
458 returned_gets: 0,
459 });
460 }
461
462 let (tx, rx) = oneshot::channel();
463 self.tx
464 .send(Command::MultiGet {
465 batch: multi_get_vec,
466 results_scratch: results_vec,
467 response_sender: tx,
468 })
469 .await
470 .map_err(|_| Error::RocksDBThreadGoneAway)?;
471
472 match rx.await.map_err(|_| Error::RocksDBThreadGoneAway)? {
474 Ok((ret, get_scratch, mut results_scratch)) => {
475 for (place, get) in results_out.into_iter().zip_eq(results_scratch.drain(..)) {
476 *place = placement(get);
477 }
478 self.multi_get_scratch = get_scratch;
479 self.multi_get_results_scratch = results_scratch;
480 Ok(ret)
481 }
482 Err(e) => {
483 Err(e)
485 }
486 }
487 }
488
489 pub async fn multi_update<P>(&mut self, puts: P) -> Result<MultiUpdateResult, Error>
496 where
497 P: IntoIterator<Item = (K, KeyUpdate<V>, Option<Diff>)>,
498 {
499 let batch_size = self.dynamic_config.batch_size();
500 let mut stats = MultiUpdateResult::default();
501
502 let mut puts = puts.into_iter().peekable();
503 if puts.peek().is_some() {
504 let puts = puts.chunks(batch_size);
505
506 for puts in puts.into_iter() {
507 let ret = self.multi_update_inner(puts).await?;
508 stats.processed_updates += ret.processed_updates;
509 stats.size_written += ret.size_written;
510 if let Some(diff) = ret.size_diff {
511 stats.size_diff = Some(stats.size_diff.unwrap_or(Diff::ZERO) + diff);
512 }
513 }
514 }
515
516 Ok(stats)
517 }
518
519 async fn multi_update_inner<P>(&mut self, updates: P) -> Result<MultiUpdateResult, Error>
520 where
521 P: IntoIterator<Item = (K, KeyUpdate<V>, Option<Diff>)>,
522 {
523 let mut multi_put_vec = std::mem::take(&mut self.multi_update_scratch);
524 multi_put_vec.clear();
525
526 multi_put_vec.extend(updates);
527 if multi_put_vec.is_empty() {
528 self.multi_update_scratch = multi_put_vec;
529 return Ok(MultiUpdateResult {
530 processed_updates: 0,
531 size_written: 0,
532 size_diff: None,
533 });
534 }
535
536 let (tx, rx) = oneshot::channel();
537 self.tx
538 .send(Command::MultiUpdate {
539 batch: multi_put_vec,
540 response_sender: tx,
541 })
542 .await
543 .map_err(|_| Error::RocksDBThreadGoneAway)?;
544
545 match rx.await.map_err(|_| Error::RocksDBThreadGoneAway)? {
547 Ok((ret, scratch)) => {
548 self.multi_update_scratch = scratch;
549 Ok(ret)
550 }
551 Err(e) => {
552 Err(e)
554 }
555 }
556 }
557
558 pub async fn manual_compaction(&self) -> Result<(), Error> {
560 let (tx, rx) = oneshot::channel();
561 self.tx
562 .send(Command::ManualCompaction { done_sender: tx })
563 .await
564 .map_err(|_| Error::RocksDBThreadGoneAway)?;
565
566 rx.await.map_err(|_| Error::RocksDBThreadGoneAway)
567 }
568
569 pub async fn close(self) -> Result<(), Error> {
572 let (tx, rx) = oneshot::channel();
573 self.tx
574 .send(Command::Shutdown { done_sender: tx })
575 .await
576 .map_err(|_| Error::RocksDBThreadGoneAway)?;
577
578 let _ = rx.await;
579
580 Ok(())
581 }
582}
583
584fn rocksdb_core_loop<K, V, M, O, IM, F>(
585 options: InstanceOptions<O, V, F>,
586 tuning_config: RocksDBConfig,
587 instance_path: PathBuf,
588 mut cmd_rx: mpsc::Receiver<Command<K, V>>,
589 shared_metrics: M,
590 instance_metrics: IM,
591) where
592 K: AsRef<[u8]> + Send + Sync + 'static,
593 V: Serialize + DeserializeOwned + Send + Sync + 'static,
594 M: Deref<Target = RocksDBSharedMetrics> + Send + 'static,
595 O: bincode::Options + Copy + Send + Sync + 'static,
596 F: for<'a> Fn(&'a [u8], ValueIterator<'a, O, V>) -> V + Send + Sync + Copy + 'static,
597 IM: Deref<Target = RocksDBInstanceMetrics> + Send + 'static,
598{
599 if options.cleanup_on_new && instance_path.exists() {
600 let retry = mz_ore::retry::Retry::default()
604 .max_tries(options.cleanup_tries)
605 .initial_backoff(std::time::Duration::from_secs(1));
607
608 let destroy_result = retry.retry(|_rs| {
609 if let Err(e) = DB::destroy(&RocksDBOptions::default(), &*instance_path) {
610 tracing::warn!(
611 "failed to cleanup rocksdb dir on creation {}: {}",
612 instance_path.display(),
613 e.display_with_causes(),
614 );
615 RetryResult::RetryableErr(Error::from(e))
616 } else {
617 RetryResult::Ok(())
618 }
619 });
620 if let Err(e) = destroy_result {
621 tracing::error!(
622 "retries exhausted trying to cleanup rocksdb dir on creation {}: {}",
623 instance_path.display(),
624 e.display_with_causes(),
625 );
626 return;
627 }
628 }
629
630 let retry_max_duration = tuning_config.retry_max_duration;
631
632 let (rocksdb_options, write_buffer_handle) = options.as_rocksdb_options(&tuning_config);
637 tracing::info!(
638 "Starting rocksdb at {:?} with write_buffer_manager: {:?}",
639 instance_path,
640 write_buffer_handle
641 );
642
643 let retry_result = Retry::default()
644 .max_duration(retry_max_duration)
645 .retry(|_| match DB::open(&rocksdb_options, &instance_path) {
646 Ok(db) => RetryResult::Ok(db),
647 Err(e) => match e.kind() {
648 ErrorKind::TryAgain => RetryResult::RetryableErr(Error::RocksDB(e)),
649 _ => RetryResult::FatalErr(Error::RocksDB(e)),
650 },
651 });
652
653 let db: DB = match retry_result {
654 Ok(db) => db,
655 Err(e) => {
656 tracing::error!(
657 "failed to create rocksdb at {}: {}",
658 instance_path.display(),
659 e.display_with_causes(),
660 );
661 return;
662 }
663 };
664
665 let mut encoded_batch_buffers: Vec<Option<Vec<u8>>> = Vec::new();
666 let mut encoded_batch: Vec<(K, KeyUpdate<Vec<u8>>)> = Vec::new();
667
668 let wo = options.as_rocksdb_write_options();
669 while let Some(cmd) = cmd_rx.blocking_recv() {
670 match cmd {
671 Command::Shutdown { done_sender } => {
672 shutdown_and_cleanup(db, &instance_path);
673 drop(write_buffer_handle);
674 let _ = done_sender.send(());
675 return;
676 }
677 Command::ManualCompaction { done_sender } => {
678 db.compact_range::<&[u8], &[u8]>(None, None);
680 let _ = done_sender.send(());
681 }
682 Command::MultiGet {
683 mut batch,
684 mut results_scratch,
685 response_sender,
686 } => {
687 let batch_size = batch.len();
688
689 let now = Instant::now();
691 let retry_result = Retry::default()
692 .max_duration(retry_max_duration)
693 .retry(|_| {
694 let gets = db.multi_get(batch.iter());
695 let latency = now.elapsed();
696
697 let gets: Result<Vec<_>, _> = gets.into_iter().collect();
698 match gets {
699 Ok(gets) => {
700 shared_metrics
701 .multi_get_latency
702 .observe(latency.as_secs_f64());
703 instance_metrics
704 .multi_get_size
705 .inc_by(batch_size.try_into().unwrap());
706 instance_metrics.multi_get_count.inc();
707
708 RetryResult::Ok(gets)
709 }
710 Err(e) => match e.kind() {
711 ErrorKind::TryAgain => RetryResult::RetryableErr(Error::RocksDB(e)),
712 _ => RetryResult::FatalErr(Error::RocksDB(e)),
713 },
714 }
715 });
716
717 let _ = match retry_result {
718 Ok(gets) => {
719 let processed_gets: u64 = gets.len().try_into().unwrap();
720 let mut processed_gets_size = 0;
721 let mut returned_gets: u64 = 0;
722 for previous_value in gets {
723 let get_result = match previous_value {
724 Some(previous_value) => {
725 match options.bincode.deserialize(&previous_value) {
726 Ok(value) => {
727 let size = u64::cast_from(previous_value.len());
728 processed_gets_size += size;
729 returned_gets += 1;
730 Some(GetResult { value, size })
731 }
732 Err(e) => {
733 let _ =
734 response_sender.send(Err(Error::DecodeError(e)));
735 return;
736 }
737 }
738 }
739 None => None,
740 };
741 results_scratch.push(get_result);
742 }
743
744 instance_metrics
745 .multi_get_result_count
746 .inc_by(returned_gets);
747 instance_metrics
748 .multi_get_result_bytes
749 .inc_by(processed_gets_size);
750 batch.clear();
751 response_sender.send(Ok((
752 MultiGetResult {
753 processed_gets,
754 processed_gets_size,
755 returned_gets,
756 },
757 batch,
758 results_scratch,
759 )))
760 }
761 Err(e) => response_sender.send(Err(e)),
762 };
763 }
764 Command::MultiUpdate {
765 mut batch,
766 response_sender,
767 } => {
768 let batch_size = batch.len();
769
770 let mut ret = MultiUpdateResult {
771 processed_updates: 0,
772 size_written: 0,
773 size_diff: None,
774 };
775
776 let buf_size = encoded_batch_buffers.len();
778 for _ in buf_size..batch_size {
779 encoded_batch_buffers.push(Some(Vec::new()));
780 }
781 if tuning_config.shrink_buffers_by_ratio > 0 {
784 let reduced_capacity =
785 encoded_batch_buffers.capacity() / tuning_config.shrink_buffers_by_ratio;
786 if reduced_capacity > batch_size {
787 encoded_batch_buffers.truncate(reduced_capacity);
788 encoded_batch_buffers.shrink_to(reduced_capacity);
789
790 encoded_batch.truncate(reduced_capacity);
791 encoded_batch.shrink_to(reduced_capacity);
792 }
793 }
794
795 let Some(encode_bufs) = encoded_batch_buffers.get_mut(0..batch_size) else {
796 panic!(
797 "Encoded buffers over-truncated. expected >= {batch_size} actual: {}",
798 encoded_batch_buffers.len()
799 );
800 };
801
802 for ((key, value, diff), encode_buf) in batch.drain(..).zip_eq(encode_bufs) {
804 ret.processed_updates += 1;
805
806 match &value {
807 update_type @ (KeyUpdate::Put(update) | KeyUpdate::Merge(update)) => {
808 let mut encode_buf =
809 encode_buf.take().expect("encode_buf should not be empty");
810 encode_buf.clear();
811 match options
812 .bincode
813 .serialize_into::<&mut Vec<u8>, _>(&mut encode_buf, update)
814 {
815 Ok(()) => {
816 ret.size_written += u64::cast_from(encode_buf.len());
817 if let Some(diff) = diff {
819 let encoded_len = Diff::try_from(encode_buf.len())
820 .expect("less than i64 size");
821 ret.size_diff = Some(
822 ret.size_diff.unwrap_or(Diff::ZERO)
823 + (diff * encoded_len),
824 );
825 }
826 }
827 Err(e) => {
828 let _ = response_sender.send(Err(Error::DecodeError(e)));
829 return;
830 }
831 };
832 if matches!(update_type, KeyUpdate::Put(_)) {
833 encoded_batch.push((key, KeyUpdate::Put(encode_buf)));
834 } else {
835 encoded_batch.push((key, KeyUpdate::Merge(encode_buf)));
836 }
837 }
838 KeyUpdate::Delete => encoded_batch.push((key, KeyUpdate::Delete)),
839 }
840 }
841 let now = Instant::now();
843 let retry_result = Retry::default()
844 .max_duration(retry_max_duration)
845 .retry(|_| {
846 let mut writes = rocksdb::WriteBatch::default();
847
848 for (key, value) in encoded_batch.iter() {
849 match value {
850 KeyUpdate::Put(update) => writes.put(key, update),
851 KeyUpdate::Merge(update) => writes.merge(key, update),
852 KeyUpdate::Delete => writes.delete(key),
853 }
854 }
855
856 match db.write_opt(writes, &wo) {
857 Ok(()) => {
858 let latency = now.elapsed();
859 shared_metrics
860 .multi_put_latency
861 .observe(latency.as_secs_f64());
862 instance_metrics
863 .multi_put_size
864 .inc_by(batch_size.try_into().unwrap());
865 instance_metrics.multi_put_count.inc();
866 RetryResult::Ok(())
867 }
868 Err(e) => match e.kind() {
869 ErrorKind::TryAgain => RetryResult::RetryableErr(Error::RocksDB(e)),
870 _ => RetryResult::FatalErr(Error::RocksDB(e)),
871 },
872 }
873 });
874
875 for (i, (_, encoded_buffer)) in encoded_batch.drain(..).enumerate() {
877 if let KeyUpdate::Put(encoded_buffer) | KeyUpdate::Merge(encoded_buffer) =
878 encoded_buffer
879 {
880 encoded_batch_buffers[i] = Some(encoded_buffer);
881 }
882 }
883
884 match retry_result {
885 Ok(()) => {
886 batch.clear();
887 let _ = response_sender.send(Ok((ret, batch)));
888 }
889 Err(e) => {
890 let db_err = match e {
891 Error::RocksDB(ref inner) => Some(inner.clone()),
892 _ => None,
893 };
894 let _ = response_sender.send(Err(e));
895 if let Some(db_err) = db_err {
896 if !matches!(db_err.kind(), ErrorKind::TryAgain) {
897 tracing::warn!(
898 "exiting on fatal rocksdb error at {}: {}",
899 instance_path.display(),
900 db_err.display_with_causes(),
901 );
902 break;
903 }
904 }
905 }
906 };
907 }
908 }
909 }
910 shutdown_and_cleanup(db, &instance_path);
911}
912
913fn shutdown_and_cleanup(db: DB, instance_path: &PathBuf) {
914 db.cancel_all_background_work(true);
916 drop(db);
917 tracing::info!("dropped rocksdb at {}", instance_path.display());
918
919 if let Err(e) = DB::destroy(&RocksDBOptions::default(), &*instance_path) {
921 tracing::warn!(
922 "failed to cleanup rocksdb dir at {}: {}",
923 instance_path.display(),
924 e.display_with_causes(),
925 );
926 }
927}