2021-03-12 20:52:19 +00:00
|
|
|
use std::collections::HashMap;
|
2021-11-03 21:07:43 +00:00
|
|
|
use std::convert::TryInto;
|
2021-03-12 18:57:37 +00:00
|
|
|
use std::sync::Arc;
|
|
|
|
use std::time::Duration;
|
|
|
|
|
2021-10-14 09:50:12 +00:00
|
|
|
use async_trait::async_trait;
|
2021-03-12 18:57:37 +00:00
|
|
|
use serde::{Deserialize, Serialize};
|
|
|
|
use serde_bytes::ByteBuf;
|
|
|
|
|
|
|
|
use futures::future::join_all;
|
|
|
|
use tokio::sync::watch;
|
|
|
|
|
2022-06-08 08:01:44 +00:00
|
|
|
use garage_db::counted_tree_hack::CountedTree;
|
|
|
|
|
2022-07-08 11:30:26 +00:00
|
|
|
use garage_util::background::*;
|
2021-03-12 18:57:37 +00:00
|
|
|
use garage_util::data::*;
|
2021-10-28 10:49:37 +00:00
|
|
|
use garage_util::error::*;
|
2021-11-03 21:07:43 +00:00
|
|
|
use garage_util::time::*;
|
2021-03-12 18:57:37 +00:00
|
|
|
|
2021-10-14 09:50:12 +00:00
|
|
|
use garage_rpc::system::System;
|
|
|
|
use garage_rpc::*;
|
2021-03-12 18:57:37 +00:00
|
|
|
|
|
|
|
use crate::data::*;
|
|
|
|
use crate::replication::*;
|
2021-03-12 20:52:19 +00:00
|
|
|
use crate::schema::*;
|
2021-03-12 18:57:37 +00:00
|
|
|
|
|
|
|
const TABLE_GC_BATCH_SIZE: usize = 1024;
|
2022-09-01 14:30:44 +00:00
|
|
|
// Same timeout as NEED_BLOCK_QUERY_TIMEOUT in block manager
|
|
|
|
const TABLE_GC_RPC_TIMEOUT: Duration = Duration::from_secs(15);
|
2021-03-12 18:57:37 +00:00
|
|
|
|
2021-11-03 21:07:43 +00:00
|
|
|
// GC delay for table entries: 1 day (24 hours)
|
|
|
|
// (the delay before the entry is added in the GC todo list
|
|
|
|
// and the moment the garbage collection actually happens)
|
|
|
|
const TABLE_GC_DELAY: Duration = Duration::from_secs(24 * 3600);
|
|
|
|
|
2021-10-28 10:49:37 +00:00
|
|
|
pub(crate) struct TableGc<F: TableSchema + 'static, R: TableReplication + 'static> {
|
2021-03-16 10:43:58 +00:00
|
|
|
system: Arc<System>,
|
2021-03-16 10:47:39 +00:00
|
|
|
data: Arc<TableData<F, R>>,
|
2021-03-12 18:57:37 +00:00
|
|
|
|
2021-10-14 09:50:12 +00:00
|
|
|
endpoint: Arc<Endpoint<GcRpc, Self>>,
|
2021-03-12 18:57:37 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
#[derive(Serialize, Deserialize)]
|
2021-05-02 21:13:08 +00:00
|
|
|
enum GcRpc {
|
2021-03-12 18:57:37 +00:00
|
|
|
Update(Vec<ByteBuf>),
|
|
|
|
DeleteIfEqualHash(Vec<(ByteBuf, Hash)>),
|
|
|
|
Ok,
|
|
|
|
}
|
|
|
|
|
2021-10-15 09:05:09 +00:00
|
|
|
impl Rpc for GcRpc {
|
|
|
|
type Response = Result<GcRpc, Error>;
|
2021-10-14 09:50:12 +00:00
|
|
|
}
|
2021-03-12 18:57:37 +00:00
|
|
|
|
2021-05-02 21:13:08 +00:00
|
|
|
impl<F, R> TableGc<F, R>
|
2021-03-12 18:57:37 +00:00
|
|
|
where
|
|
|
|
F: TableSchema + 'static,
|
|
|
|
R: TableReplication + 'static,
|
|
|
|
{
|
2021-10-14 09:50:12 +00:00
|
|
|
pub(crate) fn launch(system: Arc<System>, data: Arc<TableData<F, R>>) -> Arc<Self> {
|
|
|
|
let endpoint = system
|
|
|
|
.netapp
|
2021-12-14 11:34:01 +00:00
|
|
|
.endpoint(format!("garage_table/gc.rs/Rpc:{}", F::TABLE_NAME));
|
2021-03-12 18:57:37 +00:00
|
|
|
|
|
|
|
let gc = Arc::new(Self {
|
2021-03-16 10:43:58 +00:00
|
|
|
system: system.clone(),
|
2021-12-14 11:34:01 +00:00
|
|
|
data,
|
2021-10-14 09:50:12 +00:00
|
|
|
endpoint,
|
2021-03-12 18:57:37 +00:00
|
|
|
});
|
|
|
|
|
2021-10-14 09:50:12 +00:00
|
|
|
gc.endpoint.set_handler(gc.clone());
|
2021-03-12 18:57:37 +00:00
|
|
|
|
2022-07-08 11:30:26 +00:00
|
|
|
system.background.spawn_worker(GcWorker::new(gc.clone()));
|
2021-03-12 18:57:37 +00:00
|
|
|
|
|
|
|
gc
|
|
|
|
}
|
|
|
|
|
2021-11-03 21:07:43 +00:00
|
|
|
async fn gc_loop_iter(&self) -> Result<Option<Duration>, Error> {
|
|
|
|
let now = now_msec();
|
|
|
|
|
2021-10-28 10:49:37 +00:00
|
|
|
// List entries in the GC todo list
|
|
|
|
// These entries are put there when a tombstone is inserted in the table
|
2021-11-03 21:07:43 +00:00
|
|
|
// (see update_entry in data.rs)
|
2022-06-08 08:01:44 +00:00
|
|
|
let mut candidates = vec![];
|
|
|
|
for entry_kv in self.data.gc_todo.iter()? {
|
2021-10-28 10:49:37 +00:00
|
|
|
let (k, vhash) = entry_kv?;
|
2022-06-08 08:01:44 +00:00
|
|
|
let todo_entry = GcTodoEntry::parse(&k, &vhash);
|
2021-03-12 18:57:37 +00:00
|
|
|
|
2021-11-03 21:07:43 +00:00
|
|
|
if todo_entry.deletion_time() > now {
|
2022-06-08 08:01:44 +00:00
|
|
|
if candidates.is_empty() {
|
2021-11-03 21:07:43 +00:00
|
|
|
// If the earliest entry in the todo list shouldn't yet be processed,
|
|
|
|
// return a duration to wait in the loop
|
|
|
|
return Ok(Some(Duration::from_millis(
|
|
|
|
todo_entry.deletion_time() - now,
|
|
|
|
)));
|
|
|
|
} else {
|
|
|
|
// Otherwise we have some entries to process, do a normal iteration.
|
|
|
|
break;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2022-06-08 08:01:44 +00:00
|
|
|
candidates.push(todo_entry);
|
|
|
|
if candidates.len() >= 2 * TABLE_GC_BATCH_SIZE {
|
|
|
|
break;
|
|
|
|
}
|
|
|
|
}
|
2021-03-12 18:57:37 +00:00
|
|
|
|
2022-06-08 08:01:44 +00:00
|
|
|
let mut entries = vec![];
|
|
|
|
let mut excluded = vec![];
|
|
|
|
for mut todo_entry in candidates {
|
2021-10-28 10:49:37 +00:00
|
|
|
// Check if the tombstone is still the current value of the entry.
|
|
|
|
// If not, we don't actually want to GC it, and we will remove it
|
|
|
|
// from the gc_todo table later (below).
|
2022-06-08 08:01:44 +00:00
|
|
|
let vhash = todo_entry.value_hash;
|
2021-10-28 10:49:37 +00:00
|
|
|
todo_entry.value = self
|
2021-03-12 20:52:19 +00:00
|
|
|
.data
|
|
|
|
.store
|
2022-06-08 08:01:44 +00:00
|
|
|
.get(&todo_entry.key[..])?
|
2021-10-28 10:49:37 +00:00
|
|
|
.filter(|v| blake2sum(&v[..]) == vhash)
|
|
|
|
.map(|v| v.to_vec());
|
2021-03-12 18:57:37 +00:00
|
|
|
|
2021-10-28 10:49:37 +00:00
|
|
|
if todo_entry.value.is_some() {
|
|
|
|
entries.push(todo_entry);
|
2021-03-12 18:57:37 +00:00
|
|
|
if entries.len() >= TABLE_GC_BATCH_SIZE {
|
|
|
|
break;
|
|
|
|
}
|
|
|
|
} else {
|
2021-10-28 10:49:37 +00:00
|
|
|
excluded.push(todo_entry);
|
2021-03-12 18:57:37 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-10-28 10:49:37 +00:00
|
|
|
// Remove from gc_todo entries for tombstones where we have
|
|
|
|
// detected that the current value has changed and
|
|
|
|
// is no longer a tombstone.
|
|
|
|
for entry in excluded {
|
|
|
|
entry.remove_if_equal(&self.data.gc_todo)?;
|
2021-03-12 18:57:37 +00:00
|
|
|
}
|
|
|
|
|
2021-10-28 10:49:37 +00:00
|
|
|
// Remaining in `entries` is the list of entries we want to GC,
|
|
|
|
// and for which they are still currently tombstones in the table.
|
|
|
|
|
2021-04-23 19:42:52 +00:00
|
|
|
if entries.is_empty() {
|
2021-11-03 21:07:43 +00:00
|
|
|
// Nothing to do in this iteration (no entries present)
|
|
|
|
// Wait for a default delay of 60 seconds
|
|
|
|
return Ok(Some(Duration::from_secs(60)));
|
2021-03-12 18:57:37 +00:00
|
|
|
}
|
|
|
|
|
2021-12-14 11:34:01 +00:00
|
|
|
debug!("({}) GC: doing {} items", F::TABLE_NAME, entries.len());
|
2021-03-12 18:57:37 +00:00
|
|
|
|
2021-10-28 10:49:37 +00:00
|
|
|
// Split entries to GC by the set of nodes on which they are stored.
|
|
|
|
// Here we call them partitions but they are not exactly
|
|
|
|
// the same as partitions as defined in the ring: those partitions
|
|
|
|
// are defined by the first 8 bits of the hash, but two of these
|
|
|
|
// partitions can be stored on the same set of nodes.
|
|
|
|
// Here we detect when entries are stored on the same set of nodes:
|
|
|
|
// even if they are not in the same 8-bit partition, we can still
|
|
|
|
// handle them together.
|
2021-03-12 18:57:37 +00:00
|
|
|
let mut partitions = HashMap::new();
|
2021-10-28 10:49:37 +00:00
|
|
|
for entry in entries {
|
|
|
|
let pkh = Hash::try_from(&entry.key[..32]).unwrap();
|
2021-03-16 10:43:58 +00:00
|
|
|
let mut nodes = self.data.replication.write_nodes(&pkh);
|
|
|
|
nodes.retain(|x| *x != self.system.id);
|
2021-03-12 18:57:37 +00:00
|
|
|
nodes.sort();
|
|
|
|
|
|
|
|
if !partitions.contains_key(&nodes) {
|
|
|
|
partitions.insert(nodes.clone(), vec![]);
|
|
|
|
}
|
2021-10-28 10:49:37 +00:00
|
|
|
partitions.get_mut(&nodes).unwrap().push(entry);
|
2021-03-12 18:57:37 +00:00
|
|
|
}
|
|
|
|
|
2021-10-28 10:49:37 +00:00
|
|
|
// For each set of nodes that contains some items,
|
|
|
|
// ensure they are aware of the tombstone status, and once they
|
|
|
|
// are, instruct them to delete the entries.
|
2021-03-12 20:52:19 +00:00
|
|
|
let resps = join_all(
|
|
|
|
partitions
|
|
|
|
.into_iter()
|
|
|
|
.map(|(nodes, items)| self.try_send_and_delete(nodes, items)),
|
|
|
|
)
|
|
|
|
.await;
|
2021-03-16 15:35:10 +00:00
|
|
|
|
2021-10-28 10:49:37 +00:00
|
|
|
// Collect errors and return a single error value even if several
|
|
|
|
// errors occurred.
|
2021-03-16 15:35:10 +00:00
|
|
|
let mut errs = vec![];
|
2021-03-12 18:57:37 +00:00
|
|
|
for resp in resps {
|
|
|
|
if let Err(e) = resp {
|
2021-03-16 15:35:10 +00:00
|
|
|
errs.push(e);
|
2021-03-12 18:57:37 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-03-16 15:35:10 +00:00
|
|
|
if errs.is_empty() {
|
2021-11-03 21:07:43 +00:00
|
|
|
Ok(None)
|
2021-03-16 15:35:10 +00:00
|
|
|
} else {
|
2021-04-05 17:55:53 +00:00
|
|
|
Err(Error::Message(
|
|
|
|
errs.into_iter()
|
|
|
|
.map(|x| format!("{}", x))
|
|
|
|
.collect::<Vec<_>>()
|
|
|
|
.join(", "),
|
|
|
|
))
|
2021-11-03 21:07:43 +00:00
|
|
|
.err_context("in try_send_and_delete in table GC:")
|
2021-03-16 15:35:10 +00:00
|
|
|
}
|
2021-03-12 18:57:37 +00:00
|
|
|
}
|
|
|
|
|
2021-03-12 20:52:19 +00:00
|
|
|
async fn try_send_and_delete(
|
|
|
|
&self,
|
2021-10-15 09:05:09 +00:00
|
|
|
nodes: Vec<Uuid>,
|
2021-11-03 21:07:43 +00:00
|
|
|
mut items: Vec<GcTodoEntry>,
|
2021-03-12 20:52:19 +00:00
|
|
|
) -> Result<(), Error> {
|
2021-03-12 18:57:37 +00:00
|
|
|
let n_items = items.len();
|
|
|
|
|
2021-10-28 10:49:37 +00:00
|
|
|
// Strategy: we first send all of the values to the remote nodes,
|
2021-11-03 21:07:43 +00:00
|
|
|
// to ensure that they are aware of the tombstone state,
|
|
|
|
// and that the previous state was correctly overwritten
|
2021-10-28 10:49:37 +00:00
|
|
|
// (if they have a newer state that overrides the tombstone, that's fine).
|
|
|
|
// Second, once everyone is at least at the tombstone state,
|
|
|
|
// we instruct everyone to delete the tombstone IF that is still their current state.
|
|
|
|
// If they are now at a different state, it means that that state overrides the
|
|
|
|
// tombstone in the CRDT lattice, and it will be propagated back to us at some point
|
|
|
|
// (either just a regular update that hasn't reached us yet, or later when the
|
|
|
|
// table is synced).
|
2021-11-03 21:07:43 +00:00
|
|
|
|
2021-10-28 10:49:37 +00:00
|
|
|
// Here, we store in updates all of the tombstones to send for step 1,
|
|
|
|
// and in deletes the list of keys and hashes of value for step 2.
|
2021-03-12 18:57:37 +00:00
|
|
|
let mut updates = vec![];
|
|
|
|
let mut deletes = vec![];
|
2021-11-03 21:07:43 +00:00
|
|
|
for item in items.iter_mut() {
|
|
|
|
updates.push(ByteBuf::from(item.value.take().unwrap()));
|
|
|
|
deletes.push((ByteBuf::from(item.key.clone()), item.value_hash));
|
2021-03-12 18:57:37 +00:00
|
|
|
}
|
|
|
|
|
2021-10-28 10:49:37 +00:00
|
|
|
// Step 1: ensure everyone is at least at tombstone in CRDT lattice
|
|
|
|
// Here the quorum is nodes.len(): we cannot tolerate even a single failure,
|
|
|
|
// otherwise old values before the tombstone might come back in the data.
|
|
|
|
// GC'ing is not a critical function of the system, so it's not a big
|
|
|
|
// deal if we can't do it right now.
|
2021-10-14 09:50:12 +00:00
|
|
|
self.system
|
|
|
|
.rpc
|
2021-03-12 20:52:19 +00:00
|
|
|
.try_call_many(
|
2021-10-14 09:50:12 +00:00
|
|
|
&self.endpoint,
|
2021-03-12 20:52:19 +00:00
|
|
|
&nodes[..],
|
2021-05-02 21:13:08 +00:00
|
|
|
GcRpc::Update(updates),
|
2021-10-14 09:50:12 +00:00
|
|
|
RequestStrategy::with_priority(PRIO_BACKGROUND)
|
|
|
|
.with_quorum(nodes.len())
|
|
|
|
.with_timeout(TABLE_GC_RPC_TIMEOUT),
|
2021-03-12 20:52:19 +00:00
|
|
|
)
|
2021-10-28 10:49:37 +00:00
|
|
|
.await
|
|
|
|
.err_context("GC: send tombstones")?;
|
2021-03-12 20:52:19 +00:00
|
|
|
|
|
|
|
info!(
|
|
|
|
"({}) GC: {} items successfully pushed, will try to delete.",
|
2021-12-14 11:34:01 +00:00
|
|
|
F::TABLE_NAME,
|
|
|
|
n_items
|
2021-03-12 20:52:19 +00:00
|
|
|
);
|
2021-03-12 18:57:37 +00:00
|
|
|
|
2021-10-28 10:49:37 +00:00
|
|
|
// Step 2: delete tombstones everywhere.
|
|
|
|
// Here we also fail if even a single node returns a failure:
|
|
|
|
// it means that the garbage collection wasn't completed and has
|
|
|
|
// to be retried later.
|
2021-10-14 09:50:12 +00:00
|
|
|
self.system
|
|
|
|
.rpc
|
2021-03-12 20:52:19 +00:00
|
|
|
.try_call_many(
|
2021-10-14 09:50:12 +00:00
|
|
|
&self.endpoint,
|
2021-03-12 20:52:19 +00:00
|
|
|
&nodes[..],
|
2021-11-03 21:07:43 +00:00
|
|
|
GcRpc::DeleteIfEqualHash(deletes),
|
2021-10-14 09:50:12 +00:00
|
|
|
RequestStrategy::with_priority(PRIO_BACKGROUND)
|
|
|
|
.with_quorum(nodes.len())
|
|
|
|
.with_timeout(TABLE_GC_RPC_TIMEOUT),
|
2021-03-12 20:52:19 +00:00
|
|
|
)
|
2021-10-28 10:49:37 +00:00
|
|
|
.await
|
|
|
|
.err_context("GC: remote delete tombstones")?;
|
2021-03-12 18:57:37 +00:00
|
|
|
|
2021-10-28 10:49:37 +00:00
|
|
|
// GC has been successfull for all of these entries.
|
|
|
|
// We now remove them all from our local table and from the GC todo list.
|
2021-11-03 21:07:43 +00:00
|
|
|
for item in items {
|
2021-10-28 10:49:37 +00:00
|
|
|
self.data
|
2021-11-03 21:07:43 +00:00
|
|
|
.delete_if_equal_hash(&item.key[..], item.value_hash)
|
2021-10-28 10:49:37 +00:00
|
|
|
.err_context("GC: local delete tombstones")?;
|
2021-11-03 21:07:43 +00:00
|
|
|
item.remove_if_equal(&self.data.gc_todo)
|
2021-10-28 10:49:37 +00:00
|
|
|
.err_context("GC: remove from todo list after successfull GC")?;
|
2021-03-12 18:57:37 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
Ok(())
|
|
|
|
}
|
2021-10-15 09:05:09 +00:00
|
|
|
}
|
2021-03-12 21:06:56 +00:00
|
|
|
|
2021-10-15 09:05:09 +00:00
|
|
|
#[async_trait]
|
|
|
|
impl<F, R> EndpointHandler<GcRpc> for TableGc<F, R>
|
|
|
|
where
|
|
|
|
F: TableSchema + 'static,
|
|
|
|
R: TableReplication + 'static,
|
|
|
|
{
|
|
|
|
async fn handle(self: &Arc<Self>, message: &GcRpc, _from: NodeID) -> Result<GcRpc, Error> {
|
2021-03-12 18:57:37 +00:00
|
|
|
match message {
|
2021-05-02 21:13:08 +00:00
|
|
|
GcRpc::Update(items) => {
|
2021-03-12 18:57:37 +00:00
|
|
|
self.data.update_many(items)?;
|
2021-05-02 21:13:08 +00:00
|
|
|
Ok(GcRpc::Ok)
|
2021-03-12 18:57:37 +00:00
|
|
|
}
|
2021-05-02 21:13:08 +00:00
|
|
|
GcRpc::DeleteIfEqualHash(items) => {
|
2021-03-12 18:57:37 +00:00
|
|
|
for (key, vhash) in items.iter() {
|
|
|
|
self.data.delete_if_equal_hash(&key[..], *vhash)?;
|
|
|
|
}
|
2021-05-02 21:13:08 +00:00
|
|
|
Ok(GcRpc::Ok)
|
2021-03-12 18:57:37 +00:00
|
|
|
}
|
2022-01-03 12:58:05 +00:00
|
|
|
m => Err(Error::unexpected_rpc_message(m)),
|
2021-03-12 18:57:37 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
2021-10-28 10:49:37 +00:00
|
|
|
|
2022-07-08 11:30:26 +00:00
|
|
|
struct GcWorker<F, R>
|
|
|
|
where
|
|
|
|
F: TableSchema + 'static,
|
|
|
|
R: TableReplication + 'static,
|
|
|
|
{
|
|
|
|
gc: Arc<TableGc<F, R>>,
|
|
|
|
wait_delay: Duration,
|
|
|
|
}
|
|
|
|
|
|
|
|
impl<F, R> GcWorker<F, R>
|
|
|
|
where
|
|
|
|
F: TableSchema + 'static,
|
|
|
|
R: TableReplication + 'static,
|
|
|
|
{
|
|
|
|
fn new(gc: Arc<TableGc<F, R>>) -> Self {
|
|
|
|
Self {
|
|
|
|
gc,
|
|
|
|
wait_delay: Duration::from_secs(0),
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
#[async_trait]
|
|
|
|
impl<F, R> Worker for GcWorker<F, R>
|
|
|
|
where
|
|
|
|
F: TableSchema + 'static,
|
|
|
|
R: TableReplication + 'static,
|
|
|
|
{
|
|
|
|
fn name(&self) -> String {
|
|
|
|
format!("{} GC", F::TABLE_NAME)
|
|
|
|
}
|
|
|
|
|
|
|
|
fn info(&self) -> Option<String> {
|
|
|
|
let l = self.gc.data.gc_todo_len().unwrap_or(0);
|
|
|
|
if l > 0 {
|
|
|
|
Some(format!("{} items in queue", l))
|
|
|
|
} else {
|
|
|
|
None
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
async fn work(&mut self, _must_exit: &mut watch::Receiver<bool>) -> Result<WorkerState, Error> {
|
|
|
|
match self.gc.gc_loop_iter().await? {
|
|
|
|
None => Ok(WorkerState::Busy),
|
|
|
|
Some(delay) => {
|
|
|
|
self.wait_delay = delay;
|
|
|
|
Ok(WorkerState::Idle)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
async fn wait_for_work(&mut self, must_exit: &watch::Receiver<bool>) -> WorkerState {
|
|
|
|
if *must_exit.borrow() {
|
|
|
|
return WorkerState::Done;
|
|
|
|
}
|
|
|
|
tokio::time::sleep(self.wait_delay).await;
|
|
|
|
WorkerState::Busy
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-10-28 10:49:37 +00:00
|
|
|
/// An entry stored in the gc_todo Sled tree associated with the table
|
|
|
|
/// Contains helper function for parsing, saving, and removing
|
|
|
|
/// such entry in Sled
|
2021-11-03 21:07:43 +00:00
|
|
|
///
|
|
|
|
/// Format of an entry:
|
|
|
|
/// - key = 8 bytes: timestamp of tombstone
|
|
|
|
/// (used to implement GC delay)
|
|
|
|
/// n bytes: key in the main data table
|
|
|
|
/// - value = hash of the table entry to delete (the tombstone)
|
|
|
|
/// for verification purpose, because we don't want to delete
|
|
|
|
/// things that aren't tombstones
|
2021-10-28 10:49:37 +00:00
|
|
|
pub(crate) struct GcTodoEntry {
|
2021-11-03 21:07:43 +00:00
|
|
|
tombstone_timestamp: u64,
|
2021-10-28 10:49:37 +00:00
|
|
|
key: Vec<u8>,
|
|
|
|
value_hash: Hash,
|
|
|
|
value: Option<Vec<u8>>,
|
|
|
|
}
|
|
|
|
|
|
|
|
impl GcTodoEntry {
|
|
|
|
/// Creates a new GcTodoEntry (not saved in Sled) from its components:
|
|
|
|
/// the key of an entry in the table, and the hash of the associated
|
|
|
|
/// serialized value
|
|
|
|
pub(crate) fn new(key: Vec<u8>, value_hash: Hash) -> Self {
|
|
|
|
Self {
|
2021-11-03 21:07:43 +00:00
|
|
|
tombstone_timestamp: now_msec(),
|
2021-10-28 10:49:37 +00:00
|
|
|
key,
|
|
|
|
value_hash,
|
|
|
|
value: None,
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
/// Parses a GcTodoEntry from a (k, v) pair stored in the gc_todo tree
|
2022-06-08 08:01:44 +00:00
|
|
|
pub(crate) fn parse(db_k: &[u8], db_v: &[u8]) -> Self {
|
2021-10-28 10:49:37 +00:00
|
|
|
Self {
|
2022-06-08 08:01:44 +00:00
|
|
|
tombstone_timestamp: u64::from_be_bytes(db_k[0..8].try_into().unwrap()),
|
|
|
|
key: db_k[8..].to_vec(),
|
|
|
|
value_hash: Hash::try_from(db_v).unwrap(),
|
2021-10-28 10:49:37 +00:00
|
|
|
value: None,
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
/// Saves the GcTodoEntry in the gc_todo tree
|
2022-06-08 08:01:44 +00:00
|
|
|
pub(crate) fn save(&self, gc_todo_tree: &CountedTree) -> Result<(), Error> {
|
2021-11-03 21:07:43 +00:00
|
|
|
gc_todo_tree.insert(self.todo_table_key(), self.value_hash.as_slice())?;
|
2021-10-28 10:49:37 +00:00
|
|
|
Ok(())
|
|
|
|
}
|
|
|
|
|
|
|
|
/// Removes the GcTodoEntry from the gc_todo tree if the
|
|
|
|
/// hash of the serialized value is the same here as in the tree.
|
|
|
|
/// This is usefull to remove a todo entry only under the condition
|
|
|
|
/// that it has not changed since the time it was read, i.e.
|
|
|
|
/// what we have to do is still the same
|
2022-06-08 08:01:44 +00:00
|
|
|
pub(crate) fn remove_if_equal(&self, gc_todo_tree: &CountedTree) -> Result<(), Error> {
|
|
|
|
gc_todo_tree.compare_and_swap::<_, _, &[u8]>(
|
|
|
|
&self.todo_table_key(),
|
2021-10-28 10:49:37 +00:00
|
|
|
Some(self.value_hash),
|
|
|
|
None,
|
|
|
|
)?;
|
|
|
|
Ok(())
|
|
|
|
}
|
2021-11-03 21:07:43 +00:00
|
|
|
|
|
|
|
fn todo_table_key(&self) -> Vec<u8> {
|
|
|
|
[
|
|
|
|
&u64::to_be_bytes(self.tombstone_timestamp)[..],
|
|
|
|
&self.key[..],
|
|
|
|
]
|
|
|
|
.concat()
|
|
|
|
}
|
|
|
|
|
|
|
|
fn deletion_time(&self) -> u64 {
|
|
|
|
self.tombstone_timestamp + TABLE_GC_DELAY.as_millis() as u64
|
|
|
|
}
|
2021-10-28 10:49:37 +00:00
|
|
|
}
|