aboutsummaryrefslogtreecommitdiff
path: root/src/table/table.rs
blob: dd3394bd1afe04a017a9d93a0ccfe2176aec8f6c (plain) (blame)
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
use std::collections::{BTreeMap, HashMap};
use std::sync::Arc;
use std::time::Duration;

use futures::stream::*;
use serde::{Deserialize, Serialize};
use serde_bytes::ByteBuf;

use garage_util::data::*;
use garage_util::error::Error;

use garage_rpc::membership::System;
use garage_rpc::rpc_client::*;
use garage_rpc::rpc_server::*;

use crate::crdt::CRDT;
use crate::data::*;
use crate::replication::*;
use crate::schema::*;
use crate::sync::*;

const TABLE_RPC_TIMEOUT: Duration = Duration::from_secs(10);

pub struct TableAux<R: TableReplication> {
	pub system: Arc<System>,
	pub replication: R,
}

pub struct Table<F: TableSchema, R: TableReplication> {
	pub data: Arc<TableData<F>>,
	pub aux: Arc<TableAux<R>>,
	pub syncer: Arc<TableSyncer<F, R>>,
	rpc_client: Arc<RpcClient<TableRPC<F>>>,
}

#[derive(Serialize, Deserialize)]
pub(crate) enum TableRPC<F: TableSchema> {
	Ok,

	ReadEntry(F::P, F::S),
	ReadEntryResponse(Option<ByteBuf>),

	// Read range: read all keys in partition P, possibly starting at a certain sort key offset
	ReadRange(F::P, Option<F::S>, Option<F::Filter>, usize),

	Update(Vec<Arc<ByteBuf>>),

	SyncRPC(SyncRPC),
}

impl<F: TableSchema> RpcMessage for TableRPC<F> {}

impl<F, R> Table<F, R>
where
	F: TableSchema + 'static,
	R: TableReplication + 'static,
{
	// =============== PUBLIC INTERFACE FUNCTIONS (new, insert, get, etc) ===============

	pub fn new(
		instance: F,
		replication: R,
		system: Arc<System>,
		db: &sled::Db,
		name: String,
		rpc_server: &mut RpcServer,
	) -> Arc<Self> {
		let rpc_path = format!("table_{}", name);
		let rpc_client = system.rpc_client::<TableRPC<F>>(&rpc_path);

		let data = TableData::new(name, instance, db, system.background.clone());

		let aux = Arc::new(TableAux {
			system,
			replication,
		});

		let syncer = TableSyncer::launch(data.clone(), aux.clone(), rpc_server);

		let table = Arc::new(Self {
			data,
			aux,
			syncer,
			rpc_client,
		});

		table.clone().register_handler(rpc_server, rpc_path);

		table
	}

	pub async fn insert(&self, e: &F::E) -> Result<(), Error> {
		let hash = e.partition_key().hash();
		let who = self.aux.replication.write_nodes(&hash, &self.aux.system);
		//eprintln!("insert who: {:?}", who);

		let e_enc = Arc::new(ByteBuf::from(rmp_to_vec_all_named(e)?));
		let rpc = TableRPC::<F>::Update(vec![e_enc]);

		self.rpc_client
			.try_call_many(
				&who[..],
				rpc,
				RequestStrategy::with_quorum(self.aux.replication.write_quorum(&self.aux.system))
					.with_timeout(TABLE_RPC_TIMEOUT),
			)
			.await?;
		Ok(())
	}

	pub async fn insert_many(&self, entries: &[F::E]) -> Result<(), Error> {
		let mut call_list = HashMap::new();

		for entry in entries.iter() {
			let hash = entry.partition_key().hash();
			let who = self.aux.replication.write_nodes(&hash, &self.aux.system);
			let e_enc = Arc::new(ByteBuf::from(rmp_to_vec_all_named(entry)?));
			for node in who {
				if !call_list.contains_key(&node) {
					call_list.insert(node, vec![]);
				}
				call_list.get_mut(&node).unwrap().push(e_enc.clone());
			}
		}

		let call_futures = call_list.drain().map(|(node, entries)| async move {
			let rpc = TableRPC::<F>::Update(entries);

			let resp = self.rpc_client.call(node, rpc, TABLE_RPC_TIMEOUT).await?;
			Ok::<_, Error>((node, resp))
		});
		let mut resps = call_futures.collect::<FuturesUnordered<_>>();
		let mut errors = vec![];

		while let Some(resp) = resps.next().await {
			if let Err(e) = resp {
				errors.push(e);
			}
		}
		if errors.len() > self.aux.replication.max_write_errors() {
			Err(Error::Message("Too many errors".into()))
		} else {
			Ok(())
		}
	}

	pub async fn get(
		self: &Arc<Self>,
		partition_key: &F::P,
		sort_key: &F::S,
	) -> Result<Option<F::E>, Error> {
		let hash = partition_key.hash();
		let who = self.aux.replication.read_nodes(&hash, &self.aux.system);
		//eprintln!("get who: {:?}", who);

		let rpc = TableRPC::<F>::ReadEntry(partition_key.clone(), sort_key.clone());
		let resps = self
			.rpc_client
			.try_call_many(
				&who[..],
				rpc,
				RequestStrategy::with_quorum(self.aux.replication.read_quorum())
					.with_timeout(TABLE_RPC_TIMEOUT)
					.interrupt_after_quorum(true),
			)
			.await?;

		let mut ret = None;
		let mut not_all_same = false;
		for resp in resps {
			if let TableRPC::ReadEntryResponse(value) = resp {
				if let Some(v_bytes) = value {
					let v = self.data.decode_entry(v_bytes.as_slice())?;
					ret = match ret {
						None => Some(v),
						Some(mut x) => {
							if x != v {
								not_all_same = true;
								x.merge(&v);
							}
							Some(x)
						}
					}
				}
			} else {
				return Err(Error::Message(format!("Invalid return value to read")));
			}
		}
		if let Some(ret_entry) = &ret {
			if not_all_same {
				let self2 = self.clone();
				let ent2 = ret_entry.clone();
				self.aux
					.system
					.background
					.spawn_cancellable(async move { self2.repair_on_read(&who[..], ent2).await });
			}
		}
		Ok(ret)
	}

	pub async fn get_range(
		self: &Arc<Self>,
		partition_key: &F::P,
		begin_sort_key: Option<F::S>,
		filter: Option<F::Filter>,
		limit: usize,
	) -> Result<Vec<F::E>, Error> {
		let hash = partition_key.hash();
		let who = self.aux.replication.read_nodes(&hash, &self.aux.system);

		let rpc = TableRPC::<F>::ReadRange(partition_key.clone(), begin_sort_key, filter, limit);

		let resps = self
			.rpc_client
			.try_call_many(
				&who[..],
				rpc,
				RequestStrategy::with_quorum(self.aux.replication.read_quorum())
					.with_timeout(TABLE_RPC_TIMEOUT)
					.interrupt_after_quorum(true),
			)
			.await?;

		let mut ret = BTreeMap::new();
		let mut to_repair = BTreeMap::new();
		for resp in resps {
			if let TableRPC::Update(entries) = resp {
				for entry_bytes in entries.iter() {
					let entry = self.data.decode_entry(entry_bytes.as_slice())?;
					let entry_key = self.data.tree_key(entry.partition_key(), entry.sort_key());
					match ret.remove(&entry_key) {
						None => {
							ret.insert(entry_key, Some(entry));
						}
						Some(Some(mut prev)) => {
							let must_repair = prev != entry;
							prev.merge(&entry);
							if must_repair {
								to_repair.insert(entry_key.clone(), Some(prev.clone()));
							}
							ret.insert(entry_key, Some(prev));
						}
						Some(None) => unreachable!(),
					}
				}
			}
		}
		if !to_repair.is_empty() {
			let self2 = self.clone();
			self.aux.system.background.spawn_cancellable(async move {
				for (_, v) in to_repair.iter_mut() {
					self2.repair_on_read(&who[..], v.take().unwrap()).await?;
				}
				Ok(())
			});
		}
		let ret_vec = ret
			.iter_mut()
			.take(limit)
			.map(|(_k, v)| v.take().unwrap())
			.collect::<Vec<_>>();
		Ok(ret_vec)
	}

	// =============== UTILITY FUNCTION FOR CLIENT OPERATIONS ===============

	async fn repair_on_read(&self, who: &[UUID], what: F::E) -> Result<(), Error> {
		let what_enc = Arc::new(ByteBuf::from(rmp_to_vec_all_named(&what)?));
		self.rpc_client
			.try_call_many(
				&who[..],
				TableRPC::<F>::Update(vec![what_enc]),
				RequestStrategy::with_quorum(who.len()).with_timeout(TABLE_RPC_TIMEOUT),
			)
			.await?;
		Ok(())
	}

	// =============== HANDLERS FOR RPC OPERATIONS (SERVER SIDE) ==============

	fn register_handler(self: Arc<Self>, rpc_server: &mut RpcServer, path: String) {
		let self2 = self.clone();
		rpc_server.add_handler::<TableRPC<F>, _, _>(path, move |msg, _addr| {
			let self2 = self2.clone();
			async move { self2.handle(&msg).await }
		});

		let self2 = self.clone();
		self.rpc_client
			.set_local_handler(self.aux.system.id, move |msg| {
				let self2 = self2.clone();
				async move { self2.handle(&msg).await }
			});
	}

	async fn handle(self: &Arc<Self>, msg: &TableRPC<F>) -> Result<TableRPC<F>, Error> {
		match msg {
			TableRPC::ReadEntry(key, sort_key) => {
				let value = self.data.read_entry(key, sort_key)?;
				Ok(TableRPC::ReadEntryResponse(value))
			}
			TableRPC::ReadRange(key, begin_sort_key, filter, limit) => {
				let values = self.data.read_range(key, begin_sort_key, filter, *limit)?;
				Ok(TableRPC::Update(values))
			}
			TableRPC::Update(pairs) => {
				self.data.update_many(pairs)?;
				Ok(TableRPC::Ok)
			}
			TableRPC::SyncRPC(rpc) => {
				let response = self.syncer.handle_rpc(rpc).await?;
				Ok(TableRPC::SyncRPC(response))
			}
			_ => Err(Error::BadRPC(format!("Unexpected table RPC"))),
		}
	}
}