persistcli/maelstrom/txn_list_append_single.rs
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 664 665 666 667 668 669 670 671 672 673 674 675 676 677 678 679 680 681 682 683 684 685 686 687 688 689 690 691 692 693 694 695 696 697 698 699 700 701 702 703 704 705 706 707 708 709 710 711 712 713 714 715 716 717 718 719 720 721 722 723 724 725 726 727 728 729 730 731 732 733 734 735 736 737 738 739 740 741 742 743 744 745 746 747 748 749 750 751 752 753 754 755 756 757 758 759 760 761 762 763 764 765 766 767 768 769 770 771 772 773 774 775 776 777 778 779 780 781 782 783 784 785 786 787 788 789 790 791 792 793 794 795 796 797 798 799 800 801 802 803 804 805 806 807 808 809 810 811 812 813 814 815 816 817 818 819 820 821 822 823 824 825 826 827 828 829 830 831 832 833 834 835 836 837 838 839 840 841 842 843 844 845 846 847 848 849 850 851 852 853 854 855 856 857 858 859 860 861 862 863 864 865 866 867 868 869 870 871 872 873
// Copyright Materialize, Inc. and contributors. All rights reserved.
//
// Use of this software is governed by the Business Source License
// included in the LICENSE file.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0.
//! An implementation of the Maelstrom txn-list-append workload using a single
//! persist shard.
use std::collections::BTreeMap;
use std::sync::Arc;
use std::time::{Duration, SystemTime, UNIX_EPOCH};
use async_trait::async_trait;
use differential_dataflow::consolidation::consolidate_updates;
use differential_dataflow::lattice::Lattice;
use mz_ore::metrics::MetricsRegistry;
use mz_ore::now::SYSTEM_TIME;
use mz_persist::cfg::{BlobConfig, ConsensusConfig};
use mz_persist::location::{Blob, Consensus, ExternalError};
use mz_persist::unreliable::{UnreliableBlob, UnreliableConsensus, UnreliableHandle};
use mz_persist_client::async_runtime::IsolatedRuntime;
use mz_persist_client::cache::StateCache;
use mz_persist_client::cfg::PersistConfig;
use mz_persist_client::critical::SinceHandle;
use mz_persist_client::metrics::Metrics;
use mz_persist_client::read::{Listen, ListenEvent};
use mz_persist_client::rpc::PubSubClientConnection;
use mz_persist_client::write::WriteHandle;
use mz_persist_client::{Diagnostics, PersistClient, ShardId};
use timely::order::TotalOrder;
use timely::progress::{Antichain, Timestamp};
use timely::PartialOrder;
use tokio::sync::Mutex;
use tracing::{debug, info, trace};
use crate::maelstrom::api::{Body, ErrorCode, MaelstromError, NodeId, ReqTxnOp, ResTxnOp};
use crate::maelstrom::node::{Handle, Service};
use crate::maelstrom::services::{CachingBlob, MaelstromBlob, MaelstromConsensus};
use crate::maelstrom::txn_list_append_single::codec_impls::{
MaelstromKeySchema, MaelstromValSchema,
};
use crate::maelstrom::Args;
/// Key of the persist shard used by [Transactor]
#[derive(Debug, Clone, Default, PartialEq, Eq, PartialOrd, Ord, Hash)]
pub struct MaelstromKey(u64);
/// Val of the persist shard used by [Transactor]
#[derive(Debug, Clone, Default, PartialEq, Eq, PartialOrd, Ord)]
pub struct MaelstromVal(Vec<u64>);
/// An implementation of read-write transactions on top of persist
///
/// This executes Maelstrom [txn-list-append] transactions. The timestamp
/// selection is modeled after Materialize's SQL implementation.
///
/// [txn-list-append]: https://github.com/jepsen-io/maelstrom/blob/v0.2.1/doc/workloads.md#workload-txn-list-append
///
/// A persist shard is maintained that directly represents a key-value map as it
/// evolves over time. (Our real SQL implementation would likely instead
/// maintain a WAL of updates to multiple tables.) Each transaction has
/// associated timestamps:
///
/// - `read_ts`: at which the contents of the map are read (inclusive)
/// - `write_ts`: at which txn mutations (if any) are written
/// - `expected_upper`: the upper bound of the previous txn
/// - `new_upper`: the upper bound after this transaction commits, if it does
///
/// To keep things simple, `write_ts` is always `read_ts+1`, `expected_upper` is
/// `antichain[write_ts]`, and `new_upper` is `antichain[write_ts+1]`.
///
/// Transactions are "committed" by using `[WriteHandle::compare_and_append]`,
/// which atomically:
/// - verifies that `expected_upper` matches
/// - writes the updates
/// - advances the upper to `new_upper`.
///
/// This guarantees that all transactions are linearized with each txn's
/// `read_ts` equal to the previous txn's `write_ts`. If two transactions race
/// by reading at the same `read_ts` and writing at the same `write_ts`, the
/// `compare_and_append` guarantees that only one of them succeeds and the other
/// must retry with new timestamps.
///
/// Upon first use, the persist shard is initialized by advancing the upper to
/// `antichain[T::minimum() + 1]`. This allows the first txn to use 0 as the
/// `read_ts` and 1 as the `write_ts`.
///
/// Similarly modeling Materialize, the since of the shard is kept following the
/// upper. To keep things simple, this is done by a fixed offset. This exercises
/// persist compaction.
///
/// To ensure that both [mz_persist_client::read::ReadHandle::snapshot] and
/// [mz_persist_client::read::ReadHandle::listen] are exercised, when a txn
/// reads the state at `read_ts`, it artificially picks an `as_of` timestamp in
/// `[since, read_ts]` and splits up the read data between snapshot and listen
/// along this timestamp.
#[derive(Debug)]
pub struct Transactor {
cads_token: u64,
shard_id: ShardId,
client: PersistClient,
since: SinceHandle<MaelstromKey, MaelstromVal, u64, i64, u64>,
write: WriteHandle<MaelstromKey, MaelstromVal, u64, i64>,
read_ts: u64,
// Keep a long-lived listen, which is incrementally read as we go. Then
// assert that it has the same data as the short-lived snapshot+listen in
// `read`. This hopefully stresses slightly different parts of the system.
long_lived_updates: Vec<(
(Result<MaelstromKey, String>, Result<MaelstromVal, String>),
u64,
i64,
)>,
long_lived_listen: Listen<MaelstromKey, MaelstromVal, u64, i64>,
}
impl Transactor {
pub async fn new(
client: &PersistClient,
node_id: NodeId,
shard_id: ShardId,
) -> Result<Self, MaelstromError> {
let cads_token = node_id
.0
.trim_start_matches('n')
.parse::<u64>()
.expect("maelstrom node_id should be n followed by an integer");
let (mut write, mut read) = client
.open(
shard_id,
Arc::new(MaelstromKeySchema),
Arc::new(MaelstromValSchema),
Diagnostics::from_purpose("maelstrom long-lived"),
true,
)
.await?;
// Use the CONTROLLER_CRITICAL_SINCE id for all nodes so we get coverage
// of contending traffic.
let since = client
.open_critical_since(
shard_id,
PersistClient::CONTROLLER_CRITICAL_SINCE,
Diagnostics::from_purpose("maelstrom since"),
)
.await?;
let read_ts = Self::maybe_init_shard(&mut write).await?;
let mut long_lived_updates = Vec::new();
let as_of = since.since().clone();
let mut updates = read
.snapshot_and_fetch(as_of.clone())
.await
.expect("as_of unexpectedly unavailable");
long_lived_updates.append(&mut updates);
let long_lived_listen = read
.listen(as_of.clone())
.await
.expect("as_of unexpectedly unavailable");
Ok(Transactor {
client: client.clone(),
cads_token,
shard_id,
since,
write,
read_ts,
long_lived_updates,
long_lived_listen,
})
}
/// Initializes the shard, if it hasn't been already, and returns the read
/// timestamp.
async fn maybe_init_shard(
write: &mut WriteHandle<MaelstromKey, MaelstromVal, u64, i64>,
) -> Result<u64, MaelstromError> {
debug!("Transactor::maybe_init");
const EMPTY_UPDATES: &[((MaelstromKey, MaelstromVal), u64, i64)] = &[];
let ts_min = u64::minimum();
let initial_upper = Antichain::from_elem(ts_min);
let new_upper = Antichain::from_elem(ts_min + 1);
let cas_res = write
.compare_and_append(EMPTY_UPDATES, initial_upper.clone(), new_upper.clone())
.await;
let read_ts = match cas_res? {
Ok(()) => 0,
Err(mismatch) => Self::extract_ts(&mismatch.current)? - 1,
};
Ok(read_ts)
}
pub async fn transact(
&mut self,
req_ops: &[ReqTxnOp],
) -> Result<Vec<ResTxnOp>, MaelstromError> {
loop {
trace!("transact req={:?}", req_ops);
let state = self.read().await?;
debug!("transact req={:?} state={:?}", req_ops, state);
let (writes, res_ops) = Self::eval_txn(&state, req_ops);
// NB: We do the CaS even if writes is empty, so that read-only txns
// are also linearizable.
let write_ts = self.read_ts + 1;
let updates = writes
.into_iter()
.map(|(k, v, diff)| ((k, v), write_ts, diff));
let expected_upper = Antichain::from_elem(write_ts);
let new_upper = Antichain::from_elem(write_ts + 1);
let cas_res = self
.write
.compare_and_append(updates, expected_upper.clone(), new_upper)
.await?;
match cas_res {
Ok(()) => {
self.read_ts = write_ts;
self.advance_since().await?;
return Ok(res_ops);
}
// We lost the CaS race, try again.
Err(mismatch) => {
info!(
"transact lost the CaS race, retrying: {:?} vs {:?}",
mismatch.expected, mismatch.current
);
self.read_ts = Self::extract_ts(&mismatch.current)? - 1;
continue;
}
}
}
}
async fn read_short_lived(
&mut self,
) -> Result<
(
Vec<(
(Result<MaelstromKey, String>, Result<MaelstromVal, String>),
u64,
i64,
)>,
Antichain<u64>,
),
MaelstromError,
> {
loop {
// We're reading as of read_ts, but we can split the read between the
// snapshot and listen at any ts in `[since_ts, read_ts]`. Intentionally
// pick one that uses a combination of both to get coverage.
let as_of = Antichain::from_elem(self.read_ts);
let since_ts = Self::extract_ts(self.since.since())?;
assert!(self.read_ts >= since_ts, "{} vs {}", self.read_ts, since_ts);
let snap_ts = since_ts + (self.read_ts - since_ts) / 2;
let snap_as_of = Antichain::from_elem(snap_ts);
// Intentionally create this from scratch so we get a brand new copy of
// state and exercise some more code paths.
let mut read = self
.client
.open_leased_reader(
self.shard_id,
Arc::new(MaelstromKeySchema),
Arc::new(MaelstromValSchema),
Diagnostics::from_purpose("maelstrom short-lived"),
true,
)
.await
.expect("codecs should match");
let updates_res = read.snapshot_and_fetch(snap_as_of.clone()).await;
let mut updates = match updates_res {
Ok(x) => x,
Err(since) => {
let recent_upper = self.write.fetch_recent_upper().await;
// Because we artificially share the same CriticalReaderId
// between nodes, it doesn't quite act like a capability.
// Prod doesn't have this issue, because a new one will
// fence out old ones, but it's done here to stress edge
// cases.
//
// If we did succeed in this read, we'd anyway just find out
// from compare_and_append that our read_ts was out of date,
// so proceed by fetching a new one and trying again. We
// also re-register the since to get an updated since value
// for the snap_ts calculation above.
info!(
"snapshot cannot serve requested as_of {} since is {:?}, fetching a new read_ts and trying again",
snap_ts,
since.0.as_option()
);
self.read_ts = Self::extract_ts(recent_upper)? - 1;
self.since = self
.client
.open_critical_since(
self.shard_id,
PersistClient::CONTROLLER_CRITICAL_SINCE,
Diagnostics::from_purpose("maelstrom since"),
)
.await?;
continue;
}
};
let listen_res = read.listen(snap_as_of).await;
let listen = match listen_res {
Ok(x) => x,
Err(since) => {
let recent_upper = self.write.fetch_recent_upper().await;
// Because we artificially share the same CriticalReaderId
// between nodes, it doesn't quite act like a capability.
// Prod doesn't have this issue, because a new one will
// fence out old ones, but it's done here to stress edge
// cases.
//
// If we did succeed in this read, we'd anyway just find out
// from compare_and_append that our read_ts was out of date,
// so proceed by fetching a new one and trying again. We
// also re-register the since to get an updated since value
// for the snap_ts calculation above.
info!(
"listen cannot serve requested as_of {} since is {:?}, fetching a new read_ts and trying again",
snap_ts,
since.0.as_option(),
);
self.read_ts = Self::extract_ts(recent_upper)? - 1;
self.since = self
.client
.open_critical_since(
self.shard_id,
PersistClient::CONTROLLER_CRITICAL_SINCE,
Diagnostics::from_purpose("maelstrom since"),
)
.await?;
continue;
}
};
trace!(
"read updates from snapshot as_of {}: {:?}",
snap_ts,
updates
);
let listen_updates = Self::listen_through(listen, &as_of).await?;
trace!(
"read updates from listener as_of {} through {}: {:?}",
snap_ts,
self.read_ts,
listen_updates
);
updates.extend(listen_updates);
// Compute the contents of the collection as of `as_of`.
for (_, t, _) in updates.iter_mut() {
t.advance_by(as_of.borrow());
}
consolidate_updates(&mut updates);
return Ok((updates, as_of));
}
}
async fn read(&mut self) -> Result<BTreeMap<MaelstromKey, MaelstromVal>, MaelstromError> {
let (updates, as_of) = self.read_short_lived().await?;
let long_lived = self.read_long_lived(&as_of).await;
assert_eq!(&updates, &long_lived);
Self::extract_state_map(self.read_ts, updates)
}
async fn listen_through(
mut listen: Listen<MaelstromKey, MaelstromVal, u64, i64>,
frontier: &Antichain<u64>,
) -> Result<
Vec<(
(Result<MaelstromKey, String>, Result<MaelstromVal, String>),
u64,
i64,
)>,
ExternalError,
> {
let mut ret = Vec::new();
loop {
for event in listen.fetch_next().await {
match event {
ListenEvent::Progress(x) => {
// NB: Unlike the snapshot as_of, a listener frontier is
// not inclusive, so we have to wait until it's > our
// as_of to be sure we have everything.
if PartialOrder::less_than(frontier, &x) {
return Ok(ret);
}
}
ListenEvent::Updates(x) => {
// We want the collection at as_of, so skip anything
// past that.
ret.extend(x.into_iter().filter(|(_, ts, _)| !frontier.less_than(ts)));
}
}
}
}
}
async fn read_long_lived(
&mut self,
as_of: &Antichain<u64>,
) -> Vec<(
(Result<MaelstromKey, String>, Result<MaelstromVal, String>),
u64,
i64,
)> {
while PartialOrder::less_equal(self.long_lived_listen.frontier(), as_of) {
for event in self.long_lived_listen.fetch_next().await {
match event {
ListenEvent::Updates(mut updates) => {
self.long_lived_updates.append(&mut updates)
}
ListenEvent::Progress(_) => {} // No-op.
}
}
}
for (_, t, _) in self.long_lived_updates.iter_mut() {
t.advance_by(as_of.borrow());
}
consolidate_updates(&mut self.long_lived_updates);
// If as_of is less_than the frontier, we may have ended up with updates
// that we didn't want yet. We can't remove them from
// `self.long_lived_updates` because the long lived listener will only
// emit them once and we'll want them later. If performance was
// important, we could sort them to the end and return a subset, but
// it's not, so do the easy thing and copy the Vec.
self.long_lived_updates
.iter()
.filter(|(_, t, _)| !as_of.less_than(t))
.cloned()
.collect()
}
fn extract_state_map(
read_ts: u64,
updates: Vec<(
(Result<MaelstromKey, String>, Result<MaelstromVal, String>),
u64,
i64,
)>,
) -> Result<BTreeMap<MaelstromKey, MaelstromVal>, MaelstromError> {
let mut ret = BTreeMap::new();
for ((k, v), _, d) in updates {
if d != 1 {
return Err(MaelstromError {
code: ErrorCode::Crash,
text: format!("invalid read at time {}", read_ts),
});
}
let k = k.map_err(|err| MaelstromError {
code: ErrorCode::Crash,
text: format!("invalid key {}", err),
})?;
let v = v.map_err(|err| MaelstromError {
code: ErrorCode::Crash,
text: format!("invalid val {}", err),
})?;
if ret.contains_key(&k) {
return Err(MaelstromError {
code: ErrorCode::Crash,
text: format!("unexpected duplicate key {:?}", k),
});
}
ret.insert(k, v);
}
Ok(ret)
}
fn eval_txn(
state: &BTreeMap<MaelstromKey, MaelstromVal>,
req_ops: &[ReqTxnOp],
) -> (Vec<(MaelstromKey, MaelstromVal, i64)>, Vec<ResTxnOp>) {
let mut res_ops = Vec::new();
let mut updates = Vec::new();
let mut txn_state = BTreeMap::new();
for req_op in req_ops.iter() {
match req_op {
ReqTxnOp::Read { key } => {
let current = txn_state
.get(&MaelstromKey(*key))
.or_else(|| state.get(&MaelstromKey(*key)));
let val = current.cloned().unwrap_or_default().0;
res_ops.push(ResTxnOp::Read { key: *key, val })
}
ReqTxnOp::Append { key, val } => {
let current = txn_state
.get(&MaelstromKey(*key))
.or_else(|| state.get(&MaelstromKey(*key)));
let mut vals = match current {
Some(val) => {
// Retract the value we're about to overwrite.
updates.push((MaelstromKey(*key), val.clone(), -1));
val.clone()
}
None => MaelstromVal::default(),
};
vals.0.push(*val);
txn_state.insert(MaelstromKey(*key), vals.clone());
updates.push((MaelstromKey(*key), vals, 1));
res_ops.push(ResTxnOp::Append {
key: key.clone(),
val: *val,
})
}
}
}
debug!(
"eval_txn\n req={:?}\n res={:?}\n updates={:?}\n state={:?}\n txn_state={:?}",
req_ops, res_ops, updates, state, txn_state
);
(updates, res_ops)
}
async fn advance_since(&mut self) -> Result<(), MaelstromError> {
// To keep things interesting, advance the since.
const SINCE_LAG: u64 = 10;
let new_since = Antichain::from_elem(self.read_ts.saturating_sub(SINCE_LAG));
let mut expected_token = self.cads_token;
loop {
let res = self
.since
.maybe_compare_and_downgrade_since(&expected_token, (&self.cads_token, &new_since))
.await;
match res {
Some(Ok(latest_since)) => {
// Success! If we weren't the last one to update since, but
// only then, it might have advanced past our read_ts, so
// forward read_ts to since_ts.
if expected_token != self.cads_token {
let since_ts = Self::extract_ts(&latest_since)?;
if since_ts > self.read_ts {
info!(
"since was last updated by {}, forwarding our read_ts from {} to {}",
expected_token, self.read_ts, since_ts
);
self.read_ts = since_ts;
}
}
return Ok(());
}
Some(Err(actual_token)) => {
debug!(
"actual downgrade_since token {} didn't match expected {}, retrying",
actual_token, expected_token,
);
expected_token = actual_token;
}
None => {
panic!("should not no-op `maybe_compare_and_downgrade_since` during testing");
}
}
}
}
fn extract_ts<T: TotalOrder + Copy>(frontier: &Antichain<T>) -> Result<T, MaelstromError> {
frontier.as_option().copied().ok_or_else(|| MaelstromError {
code: ErrorCode::Crash,
text: "shard unexpectedly closed".into(),
})
}
}
/// An adaptor to implement [Service] using [Transactor]
#[derive(Debug)]
pub struct TransactorService(pub Arc<Mutex<Transactor>>);
#[async_trait]
impl Service for TransactorService {
async fn init(args: &Args, handle: &Handle) -> Result<Self, MaelstromError> {
// Use the Maelstrom services to initialize a new random ShardId (so we
// can repeatedly run tests against the same Blob and Consensus without
// conflicting) and communicate it between processes.
let shard_id = handle.maybe_init_shard_id().await?;
// Make sure the seed is recomputed each time through the retry
// closure, so we don't retry the same deterministic timeouts.
let seed: u64 = SystemTime::now()
.duration_since(UNIX_EPOCH)
.unwrap_or_default()
.subsec_nanos()
.into();
// It doesn't particularly matter what we set should_happen to, so we do
// this to have a convenient single tunable param.
let should_happen = 1.0 - args.unreliability;
// For consensus, set should_timeout to `args.unreliability` so that once we split
// ExternalErrors into determinate vs indeterminate, then
// `args.unreliability` will also be the fraction of txns that it's
// not save for Maelstrom to retry (b/c indeterminate error in
// Consensus CaS).
let should_timeout = args.unreliability;
// It doesn't particularly matter what we set should_happen and
// should_timeout to for blobs, so use the same handle for both.
let unreliable = UnreliableHandle::new(seed, should_happen, should_timeout);
let mut config =
PersistConfig::new_default_configs(&mz_persist_client::BUILD_INFO, SYSTEM_TIME.clone());
let metrics = Arc::new(Metrics::new(&config, &MetricsRegistry::new()));
// Construct requested Blob.
let blob = match &args.blob_uri {
Some(blob_uri) => {
let cfg = BlobConfig::try_from(
blob_uri,
Box::new(config.clone()),
metrics.s3_blob.clone(),
Arc::clone(&config.configs),
)
.await
.expect("blob_uri should be valid");
loop {
match cfg.clone().open().await {
Ok(x) => break x,
Err(err) => {
info!("failed to open blob, trying again: {}", err);
}
}
}
}
None => MaelstromBlob::new(handle.clone()),
};
let blob: Arc<dyn Blob> = Arc::new(UnreliableBlob::new(blob, unreliable.clone()));
// Normal production persist usage (even including a real SQL txn impl)
// isn't particularly benefitted by a cache, so we don't have one baked
// into persist. In contrast, our Maelstrom transaction model
// intentionally exercises both a new snapshot and new listener on each
// txn. As a result, without a cache, things would be terribly slow,
// unreliable would cause more retries than are interesting, and the
// Lamport diagrams that Maelstrom generates would be noisy.
let blob = CachingBlob::new(blob);
// to simplify some downstream logic (+ a bit more stress testing),
// always downgrade the since of critical handles when asked
config.critical_downgrade_interval = Duration::from_secs(0);
// set a live diff scan limit such that we'll explore both the fast and slow paths
config.set_state_versions_recent_live_diffs_limit(5);
let consensus = match &args.consensus_uri {
Some(consensus_uri) => {
let cfg = ConsensusConfig::try_from(
consensus_uri,
Box::new(config.clone()),
metrics.postgres_consensus.clone(),
)
.expect("consensus_uri should be valid");
loop {
match cfg.clone().open().await {
Ok(x) => break x,
Err(err) => {
info!("failed to open consensus, trying again: {}", err);
}
}
}
}
None => MaelstromConsensus::new(handle.clone()),
};
let consensus: Arc<dyn Consensus> =
Arc::new(UnreliableConsensus::new(consensus, unreliable));
// Wire up the TransactorService.
let isolated_runtime = Arc::new(IsolatedRuntime::default());
let pubsub_sender = PubSubClientConnection::noop().sender;
let shared_states = Arc::new(StateCache::new(
&config,
Arc::clone(&metrics),
Arc::clone(&pubsub_sender),
));
let client = PersistClient::new(
config,
blob,
consensus,
metrics,
isolated_runtime,
shared_states,
pubsub_sender,
)?;
let transactor = Transactor::new(&client, handle.node_id(), shard_id).await?;
let service = TransactorService(Arc::new(Mutex::new(transactor)));
Ok(service)
}
async fn eval(&self, handle: Handle, src: NodeId, req: Body) {
match req {
Body::ReqTxn { msg_id, txn } => {
let in_reply_to = msg_id;
match self.0.lock().await.transact(&txn).await {
Ok(txn) => handle.send_res(src, |msg_id| Body::ResTxn {
msg_id,
in_reply_to,
txn,
}),
Err(MaelstromError { code, text }) => {
handle.send_res(src, |msg_id| Body::Error {
msg_id: Some(msg_id),
in_reply_to,
code,
text,
})
}
}
}
req => unimplemented!("unsupported req: {:?}", req),
}
}
}
mod codec_impls {
use arrow::array::{BinaryArray, BinaryBuilder, UInt64Array, UInt64Builder};
use arrow::datatypes::ToByteSlice;
use bytes::Bytes;
use mz_persist_types::codec_impls::{
SimpleColumnarData, SimpleColumnarDecoder, SimpleColumnarEncoder,
};
use mz_persist_types::columnar::Schema2;
use mz_persist_types::stats::NoneStats;
use mz_persist_types::Codec;
use crate::maelstrom::txn_list_append_single::{MaelstromKey, MaelstromVal};
impl Codec for MaelstromKey {
type Storage = ();
type Schema = MaelstromKeySchema;
fn codec_name() -> String {
"MaelstromKey".into()
}
fn encode<B>(&self, buf: &mut B)
where
B: bytes::BufMut,
{
let bytes = serde_json::to_vec(&self.0).expect("failed to encode key");
buf.put(bytes.as_slice());
}
fn decode<'a>(buf: &'a [u8], _schema: &MaelstromKeySchema) -> Result<Self, String> {
Ok(MaelstromKey(
serde_json::from_slice(buf).map_err(|err| err.to_string())?,
))
}
fn encode_schema(_schema: &Self::Schema) -> Bytes {
Bytes::new()
}
fn decode_schema(buf: &Bytes) -> Self::Schema {
assert_eq!(*buf, Bytes::new());
MaelstromKeySchema
}
}
impl SimpleColumnarData for MaelstromKey {
type ArrowBuilder = UInt64Builder;
type ArrowColumn = UInt64Array;
fn goodbytes(builder: &Self::ArrowBuilder) -> usize {
builder.values_slice().to_byte_slice().len()
}
fn push(&self, builder: &mut Self::ArrowBuilder) {
builder.append_value(self.0);
}
fn push_null(builder: &mut Self::ArrowBuilder) {
builder.append_null();
}
fn read(&mut self, idx: usize, column: &Self::ArrowColumn) {
*self = MaelstromKey(column.value(idx));
}
}
#[derive(Debug, PartialEq)]
pub struct MaelstromKeySchema;
impl Schema2<MaelstromKey> for MaelstromKeySchema {
type ArrowColumn = UInt64Array;
type Statistics = NoneStats;
type Decoder = SimpleColumnarDecoder<MaelstromKey>;
type Encoder = SimpleColumnarEncoder<MaelstromKey>;
fn encoder(&self) -> Result<Self::Encoder, anyhow::Error> {
Ok(SimpleColumnarEncoder::default())
}
fn decoder(&self, col: Self::ArrowColumn) -> Result<Self::Decoder, anyhow::Error> {
Ok(SimpleColumnarDecoder::new(col))
}
}
impl Codec for MaelstromVal {
type Storage = ();
type Schema = MaelstromValSchema;
fn codec_name() -> String {
"MaelstromVal".into()
}
fn encode<B>(&self, buf: &mut B)
where
B: bytes::BufMut,
{
let bytes = serde_json::to_vec(&self.0).expect("failed to encode val");
buf.put(bytes.as_slice());
}
fn decode<'a>(buf: &'a [u8], _schema: &MaelstromValSchema) -> Result<Self, String> {
Ok(MaelstromVal(
serde_json::from_slice(buf).map_err(|err| err.to_string())?,
))
}
fn encode_schema(_schema: &Self::Schema) -> Bytes {
Bytes::new()
}
fn decode_schema(buf: &Bytes) -> Self::Schema {
assert_eq!(*buf, Bytes::new());
MaelstromValSchema
}
}
impl SimpleColumnarData for MaelstromVal {
type ArrowBuilder = BinaryBuilder;
type ArrowColumn = BinaryArray;
fn goodbytes(builder: &Self::ArrowBuilder) -> usize {
builder.values_slice().to_byte_slice().len()
}
fn push(&self, builder: &mut Self::ArrowBuilder) {
builder.append_value(&self.encode_to_vec());
}
fn push_null(builder: &mut Self::ArrowBuilder) {
builder.append_null()
}
fn read(&mut self, idx: usize, column: &Self::ArrowColumn) {
*self = MaelstromVal::decode(column.value(idx), &MaelstromValSchema)
.expect("should be valid MaelstromVal");
}
}
#[derive(Debug, PartialEq)]
pub struct MaelstromValSchema;
impl Schema2<MaelstromVal> for MaelstromValSchema {
type ArrowColumn = BinaryArray;
type Statistics = NoneStats;
type Decoder = SimpleColumnarDecoder<MaelstromVal>;
type Encoder = SimpleColumnarEncoder<MaelstromVal>;
fn encoder(&self) -> Result<Self::Encoder, anyhow::Error> {
Ok(SimpleColumnarEncoder::default())
}
fn decoder(&self, col: Self::ArrowColumn) -> Result<Self::Decoder, anyhow::Error> {
Ok(SimpleColumnarDecoder::new(col))
}
}
}