aboutsummaryrefslogtreecommitdiff
path: root/src/model/k2v/rpc.rs
blob: f5d8ffc28ee9cc0390ee51057252301b777d0080 (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
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
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
//! Module that implements RPCs specific to K2V.
//! This is necessary for insertions into the K2V store,
//! as they have to be transmitted to one of the nodes responsible
//! for storing the entry to be processed (the API entry
//! node does not process the entry directly, as this would
//! mean the vector clock gets much larger than needed).

use std::collections::{BTreeMap, HashMap};
use std::convert::TryInto;
use std::sync::{Arc, Mutex, MutexGuard};
use std::time::Duration;

use async_trait::async_trait;
use futures::stream::FuturesUnordered;
use futures::StreamExt;
use serde::{Deserialize, Serialize};
use tokio::select;

use garage_db as db;

use garage_util::crdt::*;
use garage_util::data::*;
use garage_util::error::*;
use garage_util::time::now_msec;

use garage_rpc::system::System;
use garage_rpc::*;

use garage_table::replication::{TableReplication, TableShardedReplication};
use garage_table::{PartitionKey, Table};

use crate::helper::error::Error as HelperError;
use crate::k2v::causality::*;
use crate::k2v::item_table::*;
use crate::k2v::seen::*;
use crate::k2v::sub::*;

const TIMESTAMP_KEY: &'static [u8] = b"timestamp";

/// RPC messages for K2V
#[derive(Debug, Serialize, Deserialize)]
enum K2VRpc {
	Ok,
	InsertItem(InsertedItem),
	InsertManyItems(Vec<InsertedItem>),
	PollItem {
		key: PollKey,
		causal_context: CausalContext,
		timeout_msec: u64,
	},
	PollRange {
		range: PollRange,
		seen_str: Option<String>,
		timeout_msec: u64,
	},
	PollItemResponse(Option<K2VItem>),
	PollRangeResponse(Uuid, Vec<K2VItem>),
}

#[derive(Debug, Serialize, Deserialize)]
struct InsertedItem {
	partition: K2VItemPartition,
	sort_key: String,
	causal_context: Option<CausalContext>,
	value: DvvsValue,
}

impl Rpc for K2VRpc {
	type Response = Result<K2VRpc, Error>;
}

/// The block manager, handling block exchange between nodes, and block storage on local node
pub struct K2VRpcHandler {
	system: Arc<System>,
	item_table: Arc<Table<K2VItemTable, TableShardedReplication>>,

	// Using a mutex on the local_timestamp_tree is not strictly necessary,
	// but it helps to not try to do several inserts at the same time,
	// which would create transaction conflicts and force many useless retries.
	local_timestamp_tree: Mutex<db::Tree>,

	endpoint: Arc<Endpoint<K2VRpc, Self>>,
	subscriptions: Arc<SubscriptionManager>,
}

impl K2VRpcHandler {
	pub fn new(
		system: Arc<System>,
		db: &db::Db,
		item_table: Arc<Table<K2VItemTable, TableShardedReplication>>,
		subscriptions: Arc<SubscriptionManager>,
	) -> Arc<Self> {
		let local_timestamp_tree = db
			.open_tree("k2v_local_timestamp")
			.expect("Unable to open DB tree for k2v local timestamp");
		let endpoint = system.netapp.endpoint("garage_model/k2v/Rpc".to_string());

		let rpc_handler = Arc::new(Self {
			system,
			item_table,
			local_timestamp_tree: Mutex::new(local_timestamp_tree),
			endpoint,
			subscriptions,
		});
		rpc_handler.endpoint.set_handler(rpc_handler.clone());

		rpc_handler
	}

	// ---- public interface ----

	pub async fn insert(
		&self,
		bucket_id: Uuid,
		partition_key: String,
		sort_key: String,
		causal_context: Option<CausalContext>,
		value: DvvsValue,
	) -> Result<(), Error> {
		let partition = K2VItemPartition {
			bucket_id,
			partition_key,
		};
		let mut who = self
			.item_table
			.data
			.replication
			.write_nodes(&partition.hash());
		who.sort();

		self.system
			.rpc
			.try_call_many(
				&self.endpoint,
				&who[..],
				K2VRpc::InsertItem(InsertedItem {
					partition,
					sort_key,
					causal_context,
					value,
				}),
				RequestStrategy::with_priority(PRIO_NORMAL)
					.with_quorum(1)
					.interrupt_after_quorum(true),
			)
			.await?;

		Ok(())
	}

	pub async fn insert_batch(
		&self,
		bucket_id: Uuid,
		items: Vec<(String, String, Option<CausalContext>, DvvsValue)>,
	) -> Result<(), Error> {
		let n_items = items.len();

		let mut call_list: HashMap<_, Vec<_>> = HashMap::new();

		for (partition_key, sort_key, causal_context, value) in items {
			let partition = K2VItemPartition {
				bucket_id,
				partition_key,
			};
			let mut who = self
				.item_table
				.data
				.replication
				.write_nodes(&partition.hash());
			who.sort();

			call_list.entry(who).or_default().push(InsertedItem {
				partition,
				sort_key,
				causal_context,
				value,
			});
		}

		debug!(
			"K2V insert_batch: {} requests to insert {} items",
			call_list.len(),
			n_items
		);
		let call_futures = call_list.into_iter().map(|(nodes, items)| async move {
			let resp = self
				.system
				.rpc
				.try_call_many(
					&self.endpoint,
					&nodes[..],
					K2VRpc::InsertManyItems(items),
					RequestStrategy::with_priority(PRIO_NORMAL)
						.with_quorum(1)
						.interrupt_after_quorum(true),
				)
				.await?;
			Ok::<_, Error>((nodes, resp))
		});

		let mut resps = call_futures.collect::<FuturesUnordered<_>>();
		while let Some(resp) = resps.next().await {
			resp?;
		}

		Ok(())
	}

	pub async fn poll_item(
		&self,
		bucket_id: Uuid,
		partition_key: String,
		sort_key: String,
		causal_context: CausalContext,
		timeout_msec: u64,
	) -> Result<Option<K2VItem>, Error> {
		let poll_key = PollKey {
			partition: K2VItemPartition {
				bucket_id,
				partition_key,
			},
			sort_key,
		};
		let nodes = self
			.item_table
			.data
			.replication
			.write_nodes(&poll_key.partition.hash());

		let rpc = self.system.rpc.try_call_many(
			&self.endpoint,
			&nodes[..],
			K2VRpc::PollItem {
				key: poll_key,
				causal_context,
				timeout_msec,
			},
			RequestStrategy::with_priority(PRIO_NORMAL)
				.with_quorum(self.item_table.data.replication.read_quorum())
				.without_timeout(),
		);
		let timeout_duration = Duration::from_millis(timeout_msec) + self.system.rpc.rpc_timeout();
		let resps = select! {
			r = rpc => r?,
			_ = tokio::time::sleep(timeout_duration) => return Ok(None),
		};

		let mut resp: Option<K2VItem> = None;
		for v in resps {
			match v {
				K2VRpc::PollItemResponse(Some(x)) => {
					if let Some(y) = &mut resp {
						y.merge(&x);
					} else {
						resp = Some(x);
					}
				}
				K2VRpc::PollItemResponse(None) => (),
				v => return Err(Error::unexpected_rpc_message(v)),
			}
		}

		Ok(resp)
	}

	pub async fn poll_range(
		&self,
		range: PollRange,
		seen_str: Option<String>,
		timeout_msec: u64,
	) -> Result<Option<(BTreeMap<String, K2VItem>, String)>, HelperError> {
		let has_seen_marker = seen_str.is_some();

		let mut seen = seen_str
			.as_deref()
			.map(RangeSeenMarker::decode_helper)
			.transpose()?
			.unwrap_or_default();
		seen.restrict(&range);

		let nodes = self
			.item_table
			.data
			.replication
			.write_nodes(&range.partition.hash());

		let rpc = self.system.rpc.try_call_many(
			&self.endpoint,
			&nodes[..],
			K2VRpc::PollRange {
				range,
				seen_str,
				timeout_msec,
			},
			RequestStrategy::with_priority(PRIO_NORMAL)
				.with_quorum(self.item_table.data.replication.read_quorum())
				.without_timeout(),
		);
		let timeout_duration = Duration::from_millis(timeout_msec) + self.system.rpc.rpc_timeout();
		let resps = select! {
			r = rpc => r?,
			_ = tokio::time::sleep(timeout_duration) => return Ok(None),
		};

		let mut new_items = BTreeMap::<String, K2VItem>::new();
		for v in resps {
			if let K2VRpc::PollRangeResponse(node, items) = v {
				seen.mark_seen_node_items(node, items.iter());
				for item in items.into_iter() {
					match new_items.get_mut(&item.sort_key) {
						Some(ent) => {
							ent.merge(&item);
						}
						None => {
							new_items.insert(item.sort_key.clone(), item);
						}
					}
				}
			} else {
				return Err(Error::unexpected_rpc_message(v).into());
			}
		}

		if new_items.is_empty() && has_seen_marker {
			Ok(None)
		} else {
			Ok(Some((new_items, seen.encode()?)))
		}
	}

	// ---- internal handlers ----

	async fn handle_insert(&self, item: &InsertedItem) -> Result<K2VRpc, Error> {
		let new = {
			let local_timestamp_tree = self.local_timestamp_tree.lock().unwrap();
			self.local_insert(&local_timestamp_tree, item)?
		};

		// Propagate to rest of network
		if let Some(updated) = new {
			self.item_table.insert(&updated).await?;
		}

		Ok(K2VRpc::Ok)
	}

	async fn handle_insert_many(&self, items: &[InsertedItem]) -> Result<K2VRpc, Error> {
		let mut updated_vec = vec![];

		{
			let local_timestamp_tree = self.local_timestamp_tree.lock().unwrap();
			for item in items {
				let new = self.local_insert(&local_timestamp_tree, item)?;

				if let Some(updated) = new {
					updated_vec.push(updated);
				}
			}
		}

		// Propagate to rest of network
		if !updated_vec.is_empty() {
			self.item_table.insert_many(&updated_vec).await?;
		}

		Ok(K2VRpc::Ok)
	}

	fn local_insert(
		&self,
		local_timestamp_tree: &MutexGuard<'_, db::Tree>,
		item: &InsertedItem,
	) -> Result<Option<K2VItem>, Error> {
		let now = now_msec();

		self.item_table
			.data
			.update_entry_with(&item.partition, &item.sort_key, |tx, ent| {
				let old_local_timestamp = tx
					.get(&local_timestamp_tree, TIMESTAMP_KEY)?
					.and_then(|x| x.try_into().ok())
					.map(u64::from_be_bytes)
					.unwrap_or_default();

				let mut ent = ent.unwrap_or_else(|| {
					K2VItem::new(
						item.partition.bucket_id,
						item.partition.partition_key.clone(),
						item.sort_key.clone(),
					)
				});
				let new_local_timestamp = ent.update(
					self.system.id,
					&item.causal_context,
					item.value.clone(),
					std::cmp::max(old_local_timestamp, now),
				);

				tx.insert(
					&local_timestamp_tree,
					TIMESTAMP_KEY,
					u64::to_be_bytes(new_local_timestamp),
				)?;

				Ok(ent)
			})
	}

	async fn handle_poll_item(&self, key: &PollKey, ct: &CausalContext) -> Result<K2VItem, Error> {
		let mut chan = self.subscriptions.subscribe_item(key);

		let mut value = self
			.item_table
			.data
			.read_entry(&key.partition, &key.sort_key)?
			.map(|bytes| self.item_table.data.decode_entry(&bytes[..]))
			.transpose()?
			.unwrap_or_else(|| {
				K2VItem::new(
					key.partition.bucket_id,
					key.partition.partition_key.clone(),
					key.sort_key.clone(),
				)
			});

		while !value.causal_context().is_newer_than(ct) {
			value = chan.recv().await?;
		}

		Ok(value)
	}

	async fn handle_poll_range(
		&self,
		range: &PollRange,
		seen_str: &Option<String>,
	) -> Result<Vec<K2VItem>, Error> {
		if let Some(seen_str) = seen_str {
			let seen = RangeSeenMarker::decode(seen_str).ok_or_message("Invalid seenMarker")?;

			// Subscribe now to all changes on that partition,
			// so that new items that are inserted while we are reading the range
			// will be seen in the loop below
			let mut chan = self.subscriptions.subscribe_partition(&range.partition);

			// Check for the presence of any new items already stored in the item table
			let mut new_items = self.poll_range_read_range(range, &seen)?;

			// If we found no new items, wait for a matching item to arrive
			// on the channel
			while new_items.is_empty() {
				let item = chan.recv().await?;
				if range.matches(&item) && seen.is_new_item(&item) {
					new_items.push(item);
				}
			}

			Ok(new_items)
		} else {
			// If no seen marker was specified, we do not poll for anything.
			// We return immediately with the set of known items (even if
			// it is empty), which will give the client an inital view of
			// the dataset and an initial seen marker for further
			// PollRange calls.
			self.poll_range_read_range(range, &RangeSeenMarker::default())
		}
	}

	fn poll_range_read_range(
		&self,
		range: &PollRange,
		seen: &RangeSeenMarker,
	) -> Result<Vec<K2VItem>, Error> {
		let mut new_items = vec![];

		let partition_hash = range.partition.hash();
		let first_key = match &range.start {
			None => partition_hash.to_vec(),
			Some(sk) => self.item_table.data.tree_key(&range.partition, sk),
		};
		for item in self.item_table.data.store.range(first_key..)? {
			let (key, value) = item?;
			if &key[..32] != partition_hash.as_slice() {
				break;
			}
			let item = self.item_table.data.decode_entry(&value)?;
			if !range.matches(&item) {
				break;
			}
			if seen.is_new_item(&item) {
				new_items.push(item);
			}
		}

		Ok(new_items)
	}
}

#[async_trait]
impl EndpointHandler<K2VRpc> for K2VRpcHandler {
	async fn handle(self: &Arc<Self>, message: &K2VRpc, _from: NodeID) -> Result<K2VRpc, Error> {
		match message {
			K2VRpc::InsertItem(item) => self.handle_insert(item).await,
			K2VRpc::InsertManyItems(items) => self.handle_insert_many(&items[..]).await,
			K2VRpc::PollItem {
				key,
				causal_context,
				timeout_msec,
			} => {
				let delay = tokio::time::sleep(Duration::from_millis(*timeout_msec));
				select! {
					ret = self.handle_poll_item(key, causal_context) => ret.map(Some).map(K2VRpc::PollItemResponse),
					_ = delay => Ok(K2VRpc::PollItemResponse(None)),
				}
			}
			K2VRpc::PollRange {
				range,
				seen_str,
				timeout_msec,
			} => {
				let delay = tokio::time::sleep(Duration::from_millis(*timeout_msec));
				select! {
					ret = self.handle_poll_range(range, seen_str) => ret.map(|items| K2VRpc::PollRangeResponse(self.system.id, items)),
					_ = delay => Ok(K2VRpc::PollRangeResponse(self.system.id, vec![])),
				}
			}
			m => Err(Error::unexpected_rpc_message(m)),
		}
	}
}