aboutsummaryrefslogtreecommitdiff
path: root/src/rpc/rpc_helper.rs
blob: 7e1387ed211e9425d6910027a9b620cd66458d27 (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
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
//! Contain structs related to making RPCs
use std::collections::HashMap;
use std::sync::{Arc, RwLock};
use std::time::Duration;

use futures::future::join_all;
use futures::stream::futures_unordered::FuturesUnordered;
use futures::stream::StreamExt;
use tokio::select;

use opentelemetry::KeyValue;
use opentelemetry::{
	trace::{FutureExt as OtelFutureExt, Span, TraceContextExt, Tracer},
	Context,
};

pub use netapp::endpoint::{Endpoint, EndpointHandler, StreamingEndpointHandler};
pub use netapp::message::{
	IntoReq, Message as Rpc, OrderTag, Req, RequestPriority, Resp, PRIO_BACKGROUND, PRIO_HIGH,
	PRIO_NORMAL, PRIO_SECONDARY,
};
use netapp::peering::fullmesh::FullMeshPeeringStrategy;
pub use netapp::{self, NetApp, NodeID};

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

use crate::layout::{LayoutHelper, LayoutHistory};
use crate::metrics::RpcMetrics;

// Default RPC timeout = 5 minutes
const DEFAULT_TIMEOUT: Duration = Duration::from_secs(300);

/// Strategy to apply when making RPC
#[derive(Copy, Clone)]
pub struct RequestStrategy {
	/// Min number of response to consider the request successful
	rs_quorum: Option<usize>,
	/// Send all requests at once
	rs_send_all_at_once: Option<bool>,
	/// Request priority
	rs_priority: RequestPriority,
	/// Custom timeout for this request
	rs_timeout: Timeout,
}

#[derive(Copy, Clone)]
enum Timeout {
	None,
	Default,
	Custom(Duration),
}

impl RequestStrategy {
	/// Create a RequestStrategy with default timeout and not interrupting when quorum reached
	pub fn with_priority(prio: RequestPriority) -> Self {
		RequestStrategy {
			rs_quorum: None,
			rs_send_all_at_once: None,
			rs_priority: prio,
			rs_timeout: Timeout::Default,
		}
	}
	/// Set quorum to be reached for request
	pub fn with_quorum(mut self, quorum: usize) -> Self {
		self.rs_quorum = Some(quorum);
		self
	}
	/// Set quorum to be reached for request
	pub fn send_all_at_once(mut self, value: bool) -> Self {
		self.rs_send_all_at_once = Some(value);
		self
	}
	/// Deactivate timeout for this request
	pub fn without_timeout(mut self) -> Self {
		self.rs_timeout = Timeout::None;
		self
	}
	/// Set custom timeout for this request
	pub fn with_custom_timeout(mut self, timeout: Duration) -> Self {
		self.rs_timeout = Timeout::Custom(timeout);
		self
	}
}

#[derive(Clone)]
pub struct RpcHelper(Arc<RpcHelperInner>);

struct RpcHelperInner {
	our_node_id: Uuid,
	fullmesh: Arc<FullMeshPeeringStrategy>,
	layout: Arc<RwLock<LayoutHelper>>,
	metrics: RpcMetrics,
	rpc_timeout: Duration,
}

impl RpcHelper {
	pub(crate) fn new(
		our_node_id: Uuid,
		fullmesh: Arc<FullMeshPeeringStrategy>,
		layout: Arc<RwLock<LayoutHelper>>,
		rpc_timeout: Option<Duration>,
	) -> Self {
		let metrics = RpcMetrics::new();

		Self(Arc::new(RpcHelperInner {
			our_node_id,
			fullmesh,
			layout,
			metrics,
			rpc_timeout: rpc_timeout.unwrap_or(DEFAULT_TIMEOUT),
		}))
	}

	pub fn rpc_timeout(&self) -> Duration {
		self.0.rpc_timeout
	}

	pub async fn call<M, N, H, S>(
		&self,
		endpoint: &Endpoint<M, H>,
		to: Uuid,
		msg: N,
		strat: RequestStrategy,
	) -> Result<S, Error>
	where
		M: Rpc<Response = Result<S, Error>>,
		N: IntoReq<M> + Send,
		H: StreamingEndpointHandler<M>,
	{
		let tracer = opentelemetry::global::tracer("garage");
		let span_name = format!("RPC [{}] to {:?}", endpoint.path(), to);
		let mut span = tracer.start(span_name);
		span.set_attribute(KeyValue::new("from", format!("{:?}", self.0.our_node_id)));
		span.set_attribute(KeyValue::new("to", format!("{:?}", to)));

		let metric_tags = [
			KeyValue::new("rpc_endpoint", endpoint.path().to_string()),
			KeyValue::new("from", format!("{:?}", self.0.our_node_id)),
			KeyValue::new("to", format!("{:?}", to)),
		];

		self.0.metrics.rpc_counter.add(1, &metric_tags);

		let node_id = to.into();
		let rpc_call = endpoint
			.call_streaming(&node_id, msg, strat.rs_priority)
			.with_context(Context::current_with_span(span))
			.record_duration(&self.0.metrics.rpc_duration, &metric_tags);

		let timeout = async {
			match strat.rs_timeout {
				Timeout::None => futures::future::pending().await,
				Timeout::Default => tokio::time::sleep(self.0.rpc_timeout).await,
				Timeout::Custom(t) => tokio::time::sleep(t).await,
			}
		};

		select! {
			res = rpc_call => {
				if res.is_err() {
					self.0.metrics.rpc_netapp_error_counter.add(1, &metric_tags);
				}
				let res = res?.into_msg();

				if res.is_err() {
					self.0.metrics.rpc_garage_error_counter.add(1, &metric_tags);
				}

				Ok(res?)
			}
			() = timeout => {
				self.0.metrics.rpc_timeout_counter.add(1, &metric_tags);
				Err(Error::Timeout)
			}
		}
	}

	pub async fn call_many<M, N, H, S>(
		&self,
		endpoint: &Endpoint<M, H>,
		to: &[Uuid],
		msg: N,
		strat: RequestStrategy,
	) -> Result<Vec<(Uuid, Result<S, Error>)>, Error>
	where
		M: Rpc<Response = Result<S, Error>>,
		N: IntoReq<M>,
		H: StreamingEndpointHandler<M>,
	{
		let tracer = opentelemetry::global::tracer("garage");
		let span_name = format!("RPC [{}] call_many {} nodes", endpoint.path(), to.len());
		let span = tracer.start(span_name);

		let msg = msg.into_req().map_err(netapp::error::Error::from)?;

		let resps = join_all(
			to.iter()
				.map(|to| self.call(endpoint, *to, msg.clone(), strat)),
		)
		.with_context(Context::current_with_span(span))
		.await;
		Ok(to
			.iter()
			.cloned()
			.zip(resps.into_iter())
			.collect::<Vec<_>>())
	}

	pub async fn broadcast<M, N, H, S>(
		&self,
		endpoint: &Endpoint<M, H>,
		msg: N,
		strat: RequestStrategy,
	) -> Result<Vec<(Uuid, Result<S, Error>)>, Error>
	where
		M: Rpc<Response = Result<S, Error>>,
		N: IntoReq<M>,
		H: StreamingEndpointHandler<M>,
	{
		let to = self
			.0
			.fullmesh
			.get_peer_list()
			.iter()
			.map(|p| p.id.into())
			.collect::<Vec<_>>();
		self.call_many(endpoint, &to[..], msg, strat).await
	}

	/// Make a RPC call to multiple servers, returning either a Vec of responses,
	/// or an error if quorum could not be reached due to too many errors
	///
	/// If RequestStrategy has send_all_at_once set, then all requests will be
	/// sent at once, and `try_call_many` will return as soon as a quorum of
	/// responses is achieved, dropping and cancelling the remaining requests.
	///
	/// Otherwise, `quorum` requests will be sent at the same time, and if an
	/// error response is received, a new request will be sent to replace it.
	/// The ordering of nodes to which requests are sent is determined by
	/// the `RpcHelper::request_order` function, which takes into account
	/// parameters such as node zones and measured ping values.
	///
	/// In both cases, the basic contract of this function is that even in the
	/// absence of failures, the RPC call might not be driven to completion
	/// on all of the specified nodes. It is therefore unfit for broadcast
	/// write operations where we expect all nodes to successfully store
	/// the written date.
	pub async fn try_call_many<M, N, H, S>(
		&self,
		endpoint: &Arc<Endpoint<M, H>>,
		to: &[Uuid],
		msg: N,
		strategy: RequestStrategy,
	) -> Result<Vec<S>, Error>
	where
		M: Rpc<Response = Result<S, Error>> + 'static,
		N: IntoReq<M>,
		H: StreamingEndpointHandler<M> + 'static,
		S: Send + 'static,
	{
		let quorum = strategy.rs_quorum.unwrap_or(to.len());

		let tracer = opentelemetry::global::tracer("garage");
		let span_name = format!(
			"RPC [{}] try_call_many (quorum {}/{})",
			endpoint.path(),
			quorum,
			to.len()
		);

		let mut span = tracer.start(span_name);
		span.set_attribute(KeyValue::new("from", format!("{:?}", self.0.our_node_id)));
		span.set_attribute(KeyValue::new("to", format!("{:?}", to)));
		span.set_attribute(KeyValue::new("quorum", quorum as i64));

		self.try_call_many_inner(endpoint, to, msg, strategy, quorum)
			.with_context(Context::current_with_span(span))
			.await
	}

	async fn try_call_many_inner<M, N, H, S>(
		&self,
		endpoint: &Arc<Endpoint<M, H>>,
		to: &[Uuid],
		msg: N,
		strategy: RequestStrategy,
		quorum: usize,
	) -> Result<Vec<S>, Error>
	where
		M: Rpc<Response = Result<S, Error>> + 'static,
		N: IntoReq<M>,
		H: StreamingEndpointHandler<M> + 'static,
		S: Send + 'static,
	{
		// Once quorum is reached, other requests don't matter.
		// What we do here is only send the required number of requests
		// to reach a quorum, priorizing nodes with the lowest latency.
		// When there are errors, we start new requests to compensate.

		// TODO: this could be made more aggressive, e.g. if after 2x the
		// average ping of a given request, the response is not yet received,
		// preemptively send an additional request to any remaining nodes.

		// Reorder requests to priorize closeness / low latency
		let request_order = self.request_order(&self.0.layout.read().unwrap(), to.iter().copied());
		let send_all_at_once = strategy.rs_send_all_at_once.unwrap_or(false);

		// Build future for each request
		// They are not started now: they are added below in a FuturesUnordered
		// object that will take care of polling them (see below)
		let msg = msg.into_req().map_err(netapp::error::Error::from)?;
		let mut requests = request_order.into_iter().map(|to| {
			let self2 = self.clone();
			let msg = msg.clone();
			let endpoint2 = endpoint.clone();
			async move { self2.call(&endpoint2, to, msg, strategy).await }
		});

		// Vectors in which success results and errors will be collected
		let mut successes = vec![];
		let mut errors = vec![];

		// resp_stream will contain all of the requests that are currently in flight.
		// (for the moment none, they will be added in the loop below)
		let mut resp_stream = FuturesUnordered::new();

		// Do some requests and collect results
		while successes.len() < quorum {
			// If the current set of requests that are running is not enough to possibly
			// reach quorum, start some new requests.
			while send_all_at_once || successes.len() + resp_stream.len() < quorum {
				if let Some(fut) = requests.next() {
					resp_stream.push(fut)
				} else {
					break;
				}
			}

			if successes.len() + resp_stream.len() < quorum {
				// We know we won't ever reach quorum
				break;
			}

			// Wait for one request to terminate
			match resp_stream.next().await.unwrap() {
				Ok(msg) => {
					successes.push(msg);
				}
				Err(e) => {
					errors.push(e);
				}
			}
		}

		if successes.len() >= quorum {
			Ok(successes)
		} else {
			let errors = errors.iter().map(|e| format!("{}", e)).collect::<Vec<_>>();
			Err(Error::Quorum(
				quorum,
				None,
				successes.len(),
				to.len(),
				errors,
			))
		}
	}

	/// Make a RPC call to multiple servers, returning either a Vec of responses,
	/// or an error if quorum could not be reached due to too many errors
	///
	/// Contrary to try_call_many, this fuction is especially made for broadcast
	/// write operations. In particular:
	///
	/// - The request are sent to all specified nodes as soon as `try_write_many_sets`
	///   is invoked.
	///
	/// - When `try_write_many_sets` returns, all remaining requests that haven't
	///   completed move to a background task so that they have a chance to
	///   complete successfully if there are no failures.
	///
	/// In addition, the nodes to which requests should be sent are divided in
	/// "quorum sets", and `try_write_many_sets` only returns once a quorum
	/// has been validated in each set. This is used in the case of cluster layout
	/// changes, where data has to be written both in the old layout and in the
	/// new one as long as all nodes have not successfully tranisitionned and
	/// moved all data to the new layout.
	pub async fn try_write_many_sets<M, N, H, S>(
		&self,
		endpoint: &Arc<Endpoint<M, H>>,
		to_sets: &[Vec<Uuid>],
		msg: N,
		strategy: RequestStrategy,
	) -> Result<Vec<S>, Error>
	where
		M: Rpc<Response = Result<S, Error>> + 'static,
		N: IntoReq<M>,
		H: StreamingEndpointHandler<M> + 'static,
		S: Send + 'static,
	{
		let quorum = strategy
			.rs_quorum
			.expect("internal error: missing quorum value in try_write_many_sets");

		let tracer = opentelemetry::global::tracer("garage");
		let span_name = format!(
			"RPC [{}] try_write_many_sets (quorum {} in {} sets)",
			endpoint.path(),
			quorum,
			to_sets.len()
		);

		let mut span = tracer.start(span_name);
		span.set_attribute(KeyValue::new("from", format!("{:?}", self.0.our_node_id)));
		span.set_attribute(KeyValue::new("to", format!("{:?}", to_sets)));
		span.set_attribute(KeyValue::new("quorum", quorum as i64));

		self.try_write_many_sets_inner(endpoint, to_sets, msg, strategy, quorum)
			.with_context(Context::current_with_span(span))
			.await
	}

	async fn try_write_many_sets_inner<M, N, H, S>(
		&self,
		endpoint: &Arc<Endpoint<M, H>>,
		to_sets: &[Vec<Uuid>],
		msg: N,
		strategy: RequestStrategy,
		quorum: usize,
	) -> Result<Vec<S>, Error>
	where
		M: Rpc<Response = Result<S, Error>> + 'static,
		N: IntoReq<M>,
		H: StreamingEndpointHandler<M> + 'static,
		S: Send + 'static,
	{
		let msg = msg.into_req().map_err(netapp::error::Error::from)?;

		// Peers may appear in many quorum sets. Here, build a list of peers,
		// mapping to the index of the quorum sets in which they appear.
		let mut result_tracker = QuorumSetResultTracker::new(to_sets, quorum);

		// Send one request to each peer of the quorum sets
		let requests = result_tracker.nodes.iter().map(|(peer, _)| {
			let self2 = self.clone();
			let msg = msg.clone();
			let endpoint2 = endpoint.clone();
			let to = *peer;
			async move { (to, self2.call(&endpoint2, to, msg, strategy).await) }
		});
		let mut resp_stream = requests.collect::<FuturesUnordered<_>>();

		// Drive requests to completion
		while let Some((node, resp)) = resp_stream.next().await {
			// Store the response in the correct vector and increment the
			// appropriate counters
			result_tracker.register_result(node, resp);

			// If we have a quorum of ok in all quorum sets, then it's a success!
			if result_tracker.all_quorums_ok() {
				// Continue all other requets in background
				tokio::spawn(async move {
					resp_stream.collect::<Vec<(Uuid, Result<_, _>)>>().await;
				});

				return Ok(result_tracker.success_values());
			}

			// If there is a quorum set for which too many errors were received,
			// we know it's impossible to get a quorum, so return immediately.
			if result_tracker.too_many_failures() {
				break;
			}
		}

		// At this point, there is no quorum and we know that a quorum
		// will never be achieved. Currently, we drop all remaining requests.
		// Should we still move them to background so that they can continue
		// for non-failed nodes? Not doing so has no impact on correctness,
		// but it means that more cancellation messages will be sent. Idk.
		// (When an in-progress request future is dropped, Netapp automatically
		// sends a cancellation message to the remote node to inform it that
		// the result is no longer needed. In turn, if the remote node receives
		// the cancellation message in time, it interrupts the task of the
		// running request handler.)

		// Failure, could not get quorum
		Err(result_tracker.quorum_error())
	}

	// ---- functions not related to MAKING RPCs, but just determining to what nodes
	//      they should be made and in which order ----

	pub fn block_read_nodes_of(&self, position: &Hash, rpc_helper: &RpcHelper) -> Vec<Uuid> {
		let layout = self.0.layout.read().unwrap();

		let mut ret = Vec::with_capacity(12);
		let ver_iter = layout
			.versions
			.iter()
			.rev()
			.chain(layout.old_versions.iter().rev());
		for ver in ver_iter {
			if ver.version > layout.all_sync() {
				continue;
			}
			let nodes = ver.nodes_of(position, ver.replication_factor);
			for node in rpc_helper.request_order(&layout, nodes) {
				if !ret.contains(&node) {
					ret.push(node);
				}
			}
		}
		ret
	}

	fn request_order(
		&self,
		layout: &LayoutHistory,
		nodes: impl Iterator<Item = Uuid>,
	) -> Vec<Uuid> {
		// Retrieve some status variables that we will use to sort requests
		let peer_list = self.0.fullmesh.get_peer_list();
		let our_zone = match layout.current().node_role(&self.0.our_node_id) {
			Some(pc) => &pc.zone,
			None => "",
		};

		// Augment requests with some information used to sort them.
		// The tuples are as follows:
		//         (is another node?, is another zone?, latency, node ID, request future)
		// We store all of these tuples in a vec that we can sort.
		// By sorting this vec, we priorize ourself, then nodes in the same zone,
		// and within a same zone we priorize nodes with the lowest latency.
		let mut nodes = nodes
			.map(|to| {
				let peer_zone = match layout.current().node_role(&to) {
					Some(pc) => &pc.zone,
					None => "",
				};
				let peer_avg_ping = peer_list
					.iter()
					.find(|x| x.id.as_ref() == to.as_slice())
					.and_then(|pi| pi.avg_ping)
					.unwrap_or_else(|| Duration::from_secs(10));
				(
					to != self.0.our_node_id,
					peer_zone != our_zone,
					peer_avg_ping,
					to,
				)
			})
			.collect::<Vec<_>>();

		// Sort requests by (priorize ourself, priorize same zone, priorize low latency)
		nodes.sort_by_key(|(diffnode, diffzone, ping, _to)| (*diffnode, *diffzone, *ping));

		nodes
			.into_iter()
			.map(|(_, _, _, to)| to)
			.collect::<Vec<_>>()
	}
}

// ------- utility for tracking successes/errors among write sets --------

pub struct QuorumSetResultTracker<S, E> {
	// The set of nodes and the quorum sets they belong to
	pub nodes: HashMap<Uuid, Vec<usize>>,
	pub quorum: usize,

	// The success and error responses received
	pub successes: Vec<(Uuid, S)>,
	pub failures: Vec<(Uuid, E)>,

	// The counters for successes and failures in each set
	pub success_counters: Box<[usize]>,
	pub failure_counters: Box<[usize]>,
	pub set_lens: Box<[usize]>,
}

impl<S, E: std::fmt::Display> QuorumSetResultTracker<S, E> {
	pub fn new<A>(sets: &[A], quorum: usize) -> Self
	where
		A: AsRef<[Uuid]>,
	{
		let mut nodes = HashMap::<Uuid, Vec<usize>>::new();
		for (i, set) in sets.iter().enumerate() {
			for node in set.as_ref().iter() {
				nodes.entry(*node).or_default().push(i);
			}
		}

		let num_nodes = nodes.len();
		Self {
			nodes,
			quorum,
			successes: Vec::with_capacity(num_nodes),
			failures: vec![],
			success_counters: vec![0; sets.len()].into_boxed_slice(),
			failure_counters: vec![0; sets.len()].into_boxed_slice(),
			set_lens: sets
				.iter()
				.map(|x| x.as_ref().len())
				.collect::<Vec<_>>()
				.into_boxed_slice(),
		}
	}

	pub fn register_result(&mut self, node: Uuid, result: Result<S, E>) {
		match result {
			Ok(s) => {
				self.successes.push((node, s));
				for set in self.nodes.get(&node).unwrap().iter() {
					self.success_counters[*set] += 1;
				}
			}
			Err(e) => {
				self.failures.push((node, e));
				for set in self.nodes.get(&node).unwrap().iter() {
					self.failure_counters[*set] += 1;
				}
			}
		}
	}

	pub fn all_quorums_ok(&self) -> bool {
		self.success_counters
			.iter()
			.all(|ok_cnt| *ok_cnt >= self.quorum)
	}

	pub fn too_many_failures(&self) -> bool {
		self.failure_counters
			.iter()
			.zip(self.set_lens.iter())
			.any(|(err_cnt, set_len)| *err_cnt + self.quorum > *set_len)
	}

	pub fn success_values(self) -> Vec<S> {
		self.successes
			.into_iter()
			.map(|(_, x)| x)
			.collect::<Vec<_>>()
	}

	pub fn quorum_error(self) -> Error {
		let errors = self
			.failures
			.iter()
			.map(|(n, e)| format!("{:?}: {}", n, e))
			.collect::<Vec<_>>();
		Error::Quorum(
			self.quorum,
			Some(self.set_lens.len()),
			self.successes.len(),
			self.nodes.len(),
			errors,
		)
	}
}