aboutsummaryrefslogtreecommitdiff
path: root/src/rpc/layout/manager.rs
blob: 1746501905f85b992318df278436bde1a123c412 (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
use std::collections::HashMap;
use std::sync::{atomic::Ordering, Arc, Mutex, RwLock, RwLockReadGuard};
use std::time::Duration;

use tokio::sync::Notify;

use netapp::endpoint::Endpoint;
use netapp::peering::fullmesh::FullMeshPeeringStrategy;
use netapp::NodeID;

use garage_util::config::Config;
use garage_util::data::*;
use garage_util::error::*;
use garage_util::persister::Persister;

use super::*;
use crate::rpc_helper::*;
use crate::system::*;

pub struct LayoutManager {
	node_id: Uuid,
	replication_factor: usize,
	persist_cluster_layout: Persister<LayoutHistory>,

	layout: Arc<RwLock<LayoutHelper>>,
	pub(crate) change_notify: Arc<Notify>,

	table_sync_version: Mutex<HashMap<String, u64>>,

	pub(crate) rpc_helper: RpcHelper,
	system_endpoint: Arc<Endpoint<SystemRpc, System>>,
}

impl LayoutManager {
	pub fn new(
		config: &Config,
		node_id: NodeID,
		system_endpoint: Arc<Endpoint<SystemRpc, System>>,
		fullmesh: Arc<FullMeshPeeringStrategy>,
		replication_factor: usize,
	) -> Result<Arc<Self>, Error> {
		let persist_cluster_layout: Persister<LayoutHistory> =
			Persister::new(&config.metadata_dir, "cluster_layout");

		let cluster_layout = match persist_cluster_layout.load() {
			Ok(x) => {
				if x.current().replication_factor != replication_factor {
					return Err(Error::Message(format!(
						"Prevous cluster layout has replication factor {}, which is different than the one specified in the config file ({}). The previous cluster layout can be purged, if you know what you are doing, simply by deleting the `cluster_layout` file in your metadata directory.",
						x.current().replication_factor,
						replication_factor
					)));
				}
				x
			}
			Err(e) => {
				info!(
					"No valid previous cluster layout stored ({}), starting fresh.",
					e
				);
				LayoutHistory::new(replication_factor)
			}
		};

		let mut cluster_layout = LayoutHelper::new(cluster_layout, Default::default());
		cluster_layout.update_trackers(node_id.into());

		let layout = Arc::new(RwLock::new(cluster_layout));
		let change_notify = Arc::new(Notify::new());

		let rpc_helper = RpcHelper::new(
			node_id.into(),
			fullmesh,
			layout.clone(),
			config.rpc_timeout_msec.map(Duration::from_millis),
		);

		Ok(Arc::new(Self {
			node_id: node_id.into(),
			replication_factor,
			persist_cluster_layout,
			layout,
			change_notify,
			table_sync_version: Mutex::new(HashMap::new()),
			system_endpoint,
			rpc_helper,
		}))
	}

	// ---- PUBLIC INTERFACE ----

	pub fn layout(&self) -> RwLockReadGuard<'_, LayoutHelper> {
		self.layout.read().unwrap()
	}

	pub async fn update_cluster_layout(
		self: &Arc<Self>,
		layout: &LayoutHistory,
	) -> Result<(), Error> {
		self.handle_advertise_cluster_layout(layout).await?;
		Ok(())
	}

	pub fn add_table(&self, table_name: &'static str) {
		let first_version = self.layout().versions.first().unwrap().version;

		self.table_sync_version
			.lock()
			.unwrap()
			.insert(table_name.to_string(), first_version);
	}

	pub fn sync_table_until(self: &Arc<Self>, table_name: &'static str, version: u64) {
		let mut table_sync_version = self.table_sync_version.lock().unwrap();
		*table_sync_version.get_mut(table_name).unwrap() = version;
		let sync_until = table_sync_version.iter().map(|(_, v)| *v).min().unwrap();
		drop(table_sync_version);

		let mut layout = self.layout.write().unwrap();
		if layout.update(|l| l.update_trackers.sync_map.set_max(self.node_id, sync_until)) {
			info!("sync_until updated to {}", sync_until);
			self.broadcast_update(SystemRpc::AdvertiseClusterLayoutTrackers(
				layout.update_trackers.clone(),
			));
		}
	}

	fn ack_new_version(self: &Arc<Self>) {
		let mut layout = self.layout.write().unwrap();
		if layout.ack_max_free(self.node_id) {
			self.broadcast_update(SystemRpc::AdvertiseClusterLayoutTrackers(
				layout.update_trackers.clone(),
			));
		}
	}

	// ---- ACK LOCKING ----

	pub fn write_sets_of(self: &Arc<Self>, position: &Hash) -> WriteLock<Vec<Vec<Uuid>>> {
		let layout = self.layout();
		let version = layout.current().version;
		let nodes = layout.write_sets_of(position);
		layout
			.ack_lock
			.get(&version)
			.unwrap()
			.fetch_add(1, Ordering::Relaxed);
		WriteLock::new(version, self, nodes)
	}

	// ---- INTERNALS ---

	fn merge_layout(&self, adv: &LayoutHistory) -> Option<LayoutHistory> {
		let mut layout = self.layout.write().unwrap();
		let prev_digest = layout.digest();
		let prev_layout_check = layout.check().is_ok();

		if !prev_layout_check || adv.check().is_ok() {
			if layout.update(|l| l.merge(adv)) {
				layout.update_trackers(self.node_id);
				if prev_layout_check && layout.check().is_err() {
					panic!("Merged two correct layouts and got an incorrect layout.");
				}
				assert!(layout.digest() != prev_digest);
				return Some(layout.clone());
			}
		}

		None
	}

	fn merge_layout_trackers(&self, adv: &UpdateTrackers) -> Option<UpdateTrackers> {
		let mut layout = self.layout.write().unwrap();
		let prev_digest = layout.digest();

		if layout.update_trackers != *adv {
			if layout.update(|l| l.update_trackers.merge(adv)) {
				layout.update_trackers(self.node_id);
				assert!(layout.digest() != prev_digest);
				return Some(layout.update_trackers.clone());
			}
		}

		None
	}

	async fn pull_cluster_layout(self: &Arc<Self>, peer: Uuid) {
		let resp = self
			.rpc_helper
			.call(
				&self.system_endpoint,
				peer,
				SystemRpc::PullClusterLayout,
				RequestStrategy::with_priority(PRIO_HIGH),
			)
			.await;
		if let Ok(SystemRpc::AdvertiseClusterLayout(layout)) = resp {
			if let Err(e) = self.handle_advertise_cluster_layout(&layout).await {
				warn!("In pull_cluster_layout: {}", e);
			}
		}
	}

	async fn pull_cluster_layout_trackers(self: &Arc<Self>, peer: Uuid) {
		let resp = self
			.rpc_helper
			.call(
				&self.system_endpoint,
				peer,
				SystemRpc::PullClusterLayoutTrackers,
				RequestStrategy::with_priority(PRIO_HIGH),
			)
			.await;
		if let Ok(SystemRpc::AdvertiseClusterLayoutTrackers(trackers)) = resp {
			if let Err(e) = self
				.handle_advertise_cluster_layout_trackers(&trackers)
				.await
			{
				warn!("In pull_cluster_layout_trackers: {}", e);
			}
		}
	}

	/// Save cluster layout data to disk
	async fn save_cluster_layout(&self) -> Result<(), Error> {
		let layout = self.layout.read().unwrap().clone();
		self.persist_cluster_layout
			.save_async(&layout)
			.await
			.expect("Cannot save current cluster layout");
		Ok(())
	}

	fn broadcast_update(self: &Arc<Self>, rpc: SystemRpc) {
		tokio::spawn({
			let this = self.clone();
			async move {
				if let Err(e) = this
					.rpc_helper
					.broadcast(
						&this.system_endpoint,
						rpc,
						RequestStrategy::with_priority(PRIO_HIGH),
					)
					.await
				{
					warn!("Error while broadcasting new cluster layout: {}", e);
				}
			}
		});
	}

	// ---- RPC HANDLERS ----

	pub(crate) fn handle_advertise_status(self: &Arc<Self>, from: Uuid, remote: &LayoutDigest) {
		let local = self.layout().digest();
		if remote.current_version > local.current_version
			|| remote.active_versions != local.active_versions
			|| remote.staging_hash != local.staging_hash
		{
			tokio::spawn({
				let this = self.clone();
				async move { this.pull_cluster_layout(from).await }
			});
		} else if remote.trackers_hash != local.trackers_hash {
			tokio::spawn({
				let this = self.clone();
				async move { this.pull_cluster_layout_trackers(from).await }
			});
		}
	}

	pub(crate) fn handle_pull_cluster_layout(&self) -> SystemRpc {
		let layout = self.layout.read().unwrap().clone();
		SystemRpc::AdvertiseClusterLayout(layout)
	}

	pub(crate) fn handle_pull_cluster_layout_trackers(&self) -> SystemRpc {
		let layout = self.layout.read().unwrap();
		SystemRpc::AdvertiseClusterLayoutTrackers(layout.update_trackers.clone())
	}

	pub(crate) async fn handle_advertise_cluster_layout(
		self: &Arc<Self>,
		adv: &LayoutHistory,
	) -> Result<SystemRpc, Error> {
		debug!(
			"handle_advertise_cluster_layout: {} versions, last={}, trackers={:?}",
			adv.versions.len(),
			adv.current().version,
			adv.update_trackers
		);

		if adv.current().replication_factor != self.replication_factor {
			let msg = format!(
				"Received a cluster layout from another node with replication factor {}, which is different from what we have in our configuration ({}). Discarding the cluster layout we received.",
				adv.current().replication_factor,
				self.replication_factor
			);
			error!("{}", msg);
			return Err(Error::Message(msg));
		}

		if let Some(new_layout) = self.merge_layout(adv) {
			debug!("handle_advertise_cluster_layout: some changes were added to the current stuff");

			self.change_notify.notify_waiters();
			self.broadcast_update(SystemRpc::AdvertiseClusterLayout(new_layout));
			self.save_cluster_layout().await?;
		}

		Ok(SystemRpc::Ok)
	}

	pub(crate) async fn handle_advertise_cluster_layout_trackers(
		self: &Arc<Self>,
		trackers: &UpdateTrackers,
	) -> Result<SystemRpc, Error> {
		debug!("handle_advertise_cluster_layout_trackers: {:?}", trackers);

		if let Some(new_trackers) = self.merge_layout_trackers(trackers) {
			self.change_notify.notify_waiters();
			self.broadcast_update(SystemRpc::AdvertiseClusterLayoutTrackers(new_trackers));
			self.save_cluster_layout().await?;
		}

		Ok(SystemRpc::Ok)
	}
}

// ---- ack lock ----

pub struct WriteLock<T> {
	layout_version: u64,
	layout_manager: Arc<LayoutManager>,
	value: T,
}

impl<T> WriteLock<T> {
	fn new(version: u64, layout_manager: &Arc<LayoutManager>, value: T) -> Self {
		Self {
			layout_version: version,
			layout_manager: layout_manager.clone(),
			value,
		}
	}
}

impl<T> AsRef<T> for WriteLock<T> {
	fn as_ref(&self) -> &T {
		&self.value
	}
}

impl<T> AsMut<T> for WriteLock<T> {
	fn as_mut(&mut self) -> &mut T {
		&mut self.value
	}
}

impl<T> Drop for WriteLock<T> {
	fn drop(&mut self) {
		let layout = self.layout_manager.layout(); // acquire read lock
		if let Some(counter) = layout.ack_lock.get(&self.layout_version) {
			let prev_lock = counter.fetch_sub(1, Ordering::Relaxed);
			if prev_lock == 1 && layout.current().version > self.layout_version {
				drop(layout); // release read lock, write lock will be acquired
				self.layout_manager.ack_new_version();
			}
		} else {
			error!("Could not find ack lock counter for layout version {}. This probably indicates a bug in Garage.", self.layout_version);
		}
	}
}