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 874 875 876 877 878 879 880 881 882 883 884 885 886 887 888 889 890 891 892 893 894 895 896 897 898 899 900 901 902 903 904 905 906 907 908 909 910 911 912 913 914 915 916 917 918 919 920 921 922 923 924 925 926 927 928 929 930 931 932 933 934 935 936 937 938 939 940 941 942 943 944 945 946 947 948 949 950 951 952 953 954 955 956 957 958 959 960 961 962 963 964 965 966 967 968 969 970 971 972 973 974 975 976 977 978 979 980 981 982 983 984 985 986 987 988 989 990 991 992 993 994 995 996 997 998 999 1000 1001 1002 1003 1004 1005 1006 1007 1008 1009 1010 1011 1012 1013 1014 1015 1016 1017 1018 1019 1020 1021 1022 1023 1024 1025 1026 1027 1028 1029 1030 1031 1032 1033 1034 1035 1036 1037 1038 1039 1040 1041 1042 1043 1044 1045 1046 1047 1048 1049 1050 1051 1052 1053 1054 1055 1056 1057 1058 1059 1060 1061 1062 1063 1064 1065 1066 1067 1068 1069 1070 1071 1072 1073 1074 1075 1076 1077 1078 1079 1080 1081 1082 1083 1084 1085 1086 1087 1088 1089 1090 1091 1092 1093 1094 1095 1096 1097 1098 1099 1100 1101 1102 1103 1104 1105 1106 1107 1108 1109 1110 1111 1112 1113 1114 1115 1116 1117 1118 1119 1120 1121 1122 1123 1124 1125 1126 1127 1128 1129 1130 1131 1132 1133 1134 1135 1136 1137 1138 1139 1140 1141 1142 1143 1144 1145 1146 1147 1148 1149 1150 1151 1152 1153 1154 1155 1156 1157 1158 1159 1160 1161 1162 1163 1164 1165 1166 1167 1168 1169 1170 1171 1172 1173 1174 1175 1176 1177 1178 1179 1180 1181 1182 1183 1184 1185 1186 1187 1188 1189 1190 1191 1192 1193 1194 1195 1196 1197 1198 1199 1200 1201 1202 1203 1204 1205 1206 1207 1208 1209 1210 1211 1212 1213 1214 1215 1216 1217 1218 1219 1220 1221 1222 1223 1224 1225 1226 1227 1228 1229 1230 1231 1232 1233 1234 1235 1236 1237 1238 1239 1240 1241 1242 1243 1244 1245 1246 1247 1248 1249 1250 1251 1252 1253 1254 1255 1256 1257 1258 1259 1260 1261 1262 1263 1264 1265 1266 1267 1268 1269 1270 1271 1272 1273 1274 1275 1276 1277 1278 1279 1280 1281 1282 1283 1284 1285 1286 1287 1288 1289 1290 1291 1292 1293 1294 1295 1296 1297 1298 1299 1300 1301 1302 1303 1304 1305 1306 1307 1308 1309 1310 1311 1312 1313 1314 1315 1316 1317 1318 1319 1320 1321 1322 1323 1324 1325 1326 1327 1328 1329 1330 1331 1332 1333 1334 1335 1336 1337 1338 1339 1340 1341 1342 1343 1344 1345 1346 1347 1348 1349 1350 1351 1352 1353 1354 1355 1356 1357 1358 1359 1360 1361 1362 1363 1364 1365 1366 1367 1368 1369 1370 1371 1372 1373 1374 1375 1376 1377 1378 1379 1380 1381 1382 1383 1384 1385 1386 1387 1388 1389 1390 1391 1392 1393 1394 1395 1396 1397 1398 1399 1400 1401 1402 1403 1404 1405 1406 1407 1408 1409 1410 1411 1412 1413 1414 1415 1416 1417 1418 1419 1420 1421 1422 1423 1424 1425 1426 1427 1428 1429 1430 1431 1432 1433 1434 1435 1436 1437 1438 1439 1440 1441 1442 1443 1444 1445 1446 1447 1448 1449 1450 1451 1452 1453 1454 1455 1456
// 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.
//! A source that reads from an a persist shard.
use differential_dataflow::consolidation::ConsolidatingContainerBuilder;
use mz_dyncfg::ConfigSet;
use mz_persist_client::project::{error_free, ProjectionPushdown};
use std::convert::Infallible;
use std::fmt::Debug;
use std::future::Future;
use std::hash::Hash;
use std::pin::Pin;
use std::sync::Arc;
use std::time::Instant;
use differential_dataflow::lattice::Lattice;
use futures::{future::Either, StreamExt};
use mz_expr::{ColumnSpecs, Interpreter, MfpPlan, ResultSpec, UnmaterializableFunc};
use mz_ore::cast::CastFrom;
use mz_ore::collections::CollectionExt;
use mz_ore::vec::VecExt;
use mz_persist_client::cache::PersistClientCache;
use mz_persist_client::cfg::{PersistConfig, RetryParameters};
use mz_persist_client::fetch::{FetchedBlob, FetchedPart};
use mz_persist_client::fetch::{SerdeLeasedBatchPart, ShardSourcePart};
use mz_persist_client::operators::shard_source::{shard_source, SnapshotMode};
use mz_persist_types::codec_impls::UnitSchema;
use mz_persist_types::{Codec, Codec64};
use mz_repr::{Datum, DatumVec, Diff, GlobalId, RelationType, Row, RowArena, Timestamp};
use mz_storage_types::controller::{CollectionMetadata, TxnsCodecRow};
use mz_storage_types::errors::DataflowError;
use mz_storage_types::sources::SourceData;
use mz_storage_types::stats::RelationPartStats;
use mz_timely_util::builder_async::{
Event, OperatorBuilder as AsyncOperatorBuilder, PressOnDropButton,
};
use mz_timely_util::probe::ProbeNotify;
use mz_txn_wal::operator::{txns_progress, TxnsContext};
use serde::{Deserialize, Serialize};
use timely::communication::Push;
use timely::dataflow::channels::pact::Pipeline;
use timely::dataflow::channels::Bundle;
use timely::dataflow::operators::generic::builder_rc::OperatorBuilder;
use timely::dataflow::operators::generic::OutputHandleCore;
use timely::dataflow::operators::{Capability, Leave, OkErr};
use timely::dataflow::operators::{CapabilitySet, ConnectLoop, Feedback};
use timely::dataflow::scopes::Child;
use timely::dataflow::ScopeParent;
use timely::dataflow::{Scope, Stream};
use timely::order::TotalOrder;
use timely::progress::timestamp::PathSummary;
use timely::progress::Antichain;
use timely::progress::Timestamp as TimelyTimestamp;
use timely::scheduling::Activator;
use timely::PartialOrder;
use tokio::sync::mpsc::UnboundedSender;
use tracing::trace;
use crate::metrics::BackpressureMetrics;
/// This opaque token represents progress within a timestamp, allowing finer-grained frontier
/// progress than would otherwise be possible.
///
/// This is "opaque" since we'd like to reserve the right to change the definition in the future
/// without downstreams being able to rely on the precise representation. (At the moment, this
/// is a simple batch counter, though we may change it to eg. reflect progress through the keyspace
/// in the future.)
#[derive(
Copy, Clone, PartialEq, Default, Eq, PartialOrd, Ord, Debug, Serialize, Deserialize, Hash,
)]
pub struct Subtime(u64);
impl PartialOrder for Subtime {
fn less_equal(&self, other: &Self) -> bool {
self.0.less_equal(&other.0)
}
}
impl TotalOrder for Subtime {}
impl PathSummary<Subtime> for Subtime {
fn results_in(&self, src: &Subtime) -> Option<Subtime> {
self.0.results_in(&src.0).map(Subtime)
}
fn followed_by(&self, other: &Self) -> Option<Self> {
self.0.followed_by(&other.0).map(Subtime)
}
}
impl TimelyTimestamp for Subtime {
type Summary = Subtime;
fn minimum() -> Self {
Subtime(0)
}
}
impl Subtime {
/// The smallest non-zero summary for the opaque timestamp type.
pub const fn least_summary() -> Self {
Subtime(1)
}
}
/// Creates a new source that reads from a persist shard, distributing the work
/// of reading data to all timely workers.
///
/// All times emitted will have been [advanced by] the given `as_of` frontier.
/// All updates at times greater or equal to `until` will be suppressed.
/// The `map_filter_project` argument, if supplied, may be partially applied,
/// and any un-applied part of the argument will be left behind in the argument.
///
/// Users of this function have the ability to apply flow control to the output
/// to limit the in-flight data (measured in bytes) it can emit. The flow control
/// input is a timely stream that communicates the frontier at which the data
/// emitted from by this source have been dropped.
///
/// **Note:** Because this function is reading batches from `persist`, it is working
/// at batch granularity. In practice, the source will be overshooting the target
/// flow control upper by an amount that is related to the size of batches.
///
/// If no flow control is desired an empty stream whose frontier immediately advances
/// to the empty antichain can be used. An easy easy of creating such stream is by
/// using [`timely::dataflow::operators::generic::operator::empty`].
///
/// [advanced by]: differential_dataflow::lattice::Lattice::advance_by
pub fn persist_source<G>(
scope: &mut G,
source_id: GlobalId,
persist_clients: Arc<PersistClientCache>,
txns_ctx: &TxnsContext,
// In case we need to use a dyncfg to decide which operators to render in a
// dataflow.
worker_dyncfgs: &ConfigSet,
metadata: CollectionMetadata,
as_of: Option<Antichain<Timestamp>>,
snapshot_mode: SnapshotMode,
until: Antichain<Timestamp>,
map_filter_project: Option<&mut MfpPlan>,
max_inflight_bytes: Option<usize>,
start_signal: impl Future<Output = ()> + 'static,
error_handler: impl FnOnce(String) -> Pin<Box<dyn Future<Output = ()>>> + 'static,
) -> (
Stream<G, (Row, Timestamp, Diff)>,
Stream<G, (DataflowError, Timestamp, Diff)>,
Vec<PressOnDropButton>,
)
where
G: Scope<Timestamp = mz_repr::Timestamp>,
{
let shard_metrics = persist_clients.shard_metrics(&metadata.data_shard, &source_id.to_string());
let mut tokens = vec![];
let stream = scope.scoped(&format!("granular_backpressure({})", source_id), |scope| {
let (flow_control, flow_control_probe) = match max_inflight_bytes {
Some(max_inflight_bytes) => {
let backpressure_metrics = BackpressureMetrics {
emitted_bytes: Arc::clone(&shard_metrics.backpressure_emitted_bytes),
last_backpressured_bytes: Arc::clone(
&shard_metrics.backpressure_last_backpressured_bytes,
),
retired_bytes: Arc::clone(&shard_metrics.backpressure_retired_bytes),
};
let probe = mz_timely_util::probe::Handle::default();
let progress_stream = mz_timely_util::probe::source(
scope.clone(),
format!("decode_backpressure_probe({source_id})"),
probe.clone(),
);
let flow_control = FlowControl {
progress_stream,
max_inflight_bytes,
summary: (Default::default(), Subtime::least_summary()),
metrics: Some(backpressure_metrics),
};
(Some(flow_control), Some(probe))
}
None => (None, None),
};
// Our default listen sleeps are tuned for the case of a shard that is
// written once a second, but txn-wal allows these to be lazy.
// Override the tuning to reduce crdb load. The pubsub fallback
// responsibility is then replaced by manual "one state" wakeups in the
// txns_progress operator.
let cfg = Arc::clone(&persist_clients.cfg().configs);
let subscribe_sleep = match metadata.txns_shard {
Some(_) => Some(move || mz_txn_wal::operator::txns_data_shard_retry_params(&cfg)),
None => None,
};
let (stream, source_tokens) = persist_source_core(
scope,
source_id,
Arc::clone(&persist_clients),
metadata.clone(),
as_of.clone(),
snapshot_mode,
until.clone(),
map_filter_project,
flow_control,
subscribe_sleep,
start_signal,
error_handler,
);
tokens.extend(source_tokens);
let stream = match flow_control_probe {
Some(probe) => stream.probe_notify_with(vec![probe]),
None => stream,
};
stream.leave()
});
// If a txns_shard was provided, then this shard is in the txn-wal
// system. This means the "logical" upper may be ahead of the "physical"
// upper. Render a dataflow operator that passes through the input and
// translates the progress frontiers as necessary.
let (stream, txns_tokens) = match metadata.txns_shard {
Some(txns_shard) => txns_progress::<SourceData, (), Timestamp, i64, _, TxnsCodecRow, _, _>(
stream,
&source_id.to_string(),
txns_ctx,
worker_dyncfgs,
move || {
let (c, l) = (
Arc::clone(&persist_clients),
metadata.persist_location.clone(),
);
async move { c.open(l).await.expect("location is valid") }
},
txns_shard,
metadata.data_shard,
as_of
.expect("as_of is provided for table sources")
.into_option()
.expect("shard is not closed"),
until,
Arc::new(metadata.relation_desc),
Arc::new(UnitSchema),
),
None => (stream, vec![]),
};
tokens.extend(txns_tokens);
let (ok_stream, err_stream) = stream.ok_err(|(d, t, r)| match d {
Ok(row) => Ok((row, t.0, r)),
Err(err) => Err((err, t.0, r)),
});
(ok_stream, err_stream, tokens)
}
type RefinedScope<'g, G> = Child<'g, G, (<G as ScopeParent>::Timestamp, Subtime)>;
/// Creates a new source that reads from a persist shard, distributing the work
/// of reading data to all timely workers.
///
/// All times emitted will have been [advanced by] the given `as_of` frontier.
///
/// [advanced by]: differential_dataflow::lattice::Lattice::advance_by
#[allow(clippy::needless_borrow)]
pub fn persist_source_core<'g, G>(
scope: &RefinedScope<'g, G>,
source_id: GlobalId,
persist_clients: Arc<PersistClientCache>,
metadata: CollectionMetadata,
as_of: Option<Antichain<Timestamp>>,
snapshot_mode: SnapshotMode,
until: Antichain<Timestamp>,
map_filter_project: Option<&mut MfpPlan>,
flow_control: Option<FlowControl<RefinedScope<'g, G>>>,
// If Some, an override for the default listen sleep retry parameters.
listen_sleep: Option<impl Fn() -> RetryParameters + 'static>,
start_signal: impl Future<Output = ()> + 'static,
error_handler: impl FnOnce(String) -> Pin<Box<dyn Future<Output = ()>>> + 'static,
) -> (
Stream<
RefinedScope<'g, G>,
(
Result<Row, DataflowError>,
(mz_repr::Timestamp, Subtime),
Diff,
),
>,
Vec<PressOnDropButton>,
)
where
G: Scope<Timestamp = mz_repr::Timestamp>,
{
let cfg = persist_clients.cfg().clone();
let name = source_id.to_string();
let desc = metadata.relation_desc.clone();
let ignores_data = map_filter_project
.as_ref()
.map_or(false, |x| x.ignores_input());
let project = if ignores_data {
ProjectionPushdown::IgnoreAllNonErr {
err_col_name: "err",
key_bytes: SourceData(Ok(Row::default())).encode_to_vec(),
val_bytes: ().encode_to_vec(),
}
} else {
ProjectionPushdown::FetchAll
};
let filter_plan = map_filter_project.as_ref().map(|p| (*p).clone());
let desc_transformer = match flow_control {
Some(flow_control) => Some(move |mut scope: _, descs: &Stream<_, _>, chosen_worker| {
let (stream, token) = backpressure(
&mut scope,
&format!("backpressure({source_id})"),
descs,
flow_control,
chosen_worker,
None,
);
(stream, vec![token])
}),
None => None,
};
let metrics = Arc::clone(persist_clients.metrics());
let filter_name = name.clone();
// The `until` gives us an upper bound on the possible values of `mz_now` this query may see.
// Ranges are inclusive, so it's safe to use the maximum timestamp as the upper bound when
// `until ` is the empty antichain.
let upper = until.as_option().cloned().unwrap_or(Timestamp::MAX);
let (fetched, token) = shard_source(
&mut scope.clone(),
&name,
move || {
let (c, l) = (
Arc::clone(&persist_clients),
metadata.persist_location.clone(),
);
async move { c.open(l).await.unwrap() }
},
metadata.data_shard,
as_of,
snapshot_mode,
until.clone(),
desc_transformer,
Arc::new(metadata.relation_desc),
Arc::new(UnitSchema),
move |stats, frontier| {
let Some(lower) = frontier.as_option().copied() else {
// If the frontier has advanced to the empty antichain,
// we'll never emit any rows from any part.
return false;
};
if lower > upper {
// The frontier timestamp is larger than the until of the dataflow:
// anything from this part will necessarily be filtered out.
return false;
}
let time_range =
ResultSpec::value_between(Datum::MzTimestamp(lower), Datum::MzTimestamp(upper));
if let Some(plan) = &filter_plan {
let metrics = &metrics.pushdown.part_stats;
let stats = RelationPartStats::new(&filter_name, metrics, &desc, stats);
filter_may_match(desc.typ(), time_range, stats, plan)
} else {
true
}
},
listen_sleep,
start_signal,
error_handler,
project,
);
let rows = decode_and_mfp(cfg, &fetched, &name, until, map_filter_project);
(rows, token)
}
fn filter_may_match(
relation_type: &RelationType,
time_range: ResultSpec,
stats: RelationPartStats,
plan: &MfpPlan,
) -> bool {
let arena = RowArena::new();
let mut ranges = ColumnSpecs::new(relation_type, &arena);
ranges.push_unmaterializable(UnmaterializableFunc::MzNow, time_range);
if stats.err_count().into_iter().any(|count| count > 0) {
// If the error collection is nonempty, we always keep the part.
return true;
}
for (id, _) in relation_type.column_types.iter().enumerate() {
let result_spec = stats.col_stats(id, &arena);
ranges.push_column(id, result_spec);
}
let result = ranges.mfp_plan_filter(plan).range;
result.may_contain(Datum::True) || result.may_fail()
}
pub fn decode_and_mfp<G>(
cfg: PersistConfig,
fetched: &Stream<G, FetchedBlob<SourceData, (), Timestamp, Diff>>,
name: &str,
until: Antichain<Timestamp>,
mut map_filter_project: Option<&mut MfpPlan>,
) -> Stream<G, (Result<Row, DataflowError>, G::Timestamp, Diff)>
where
G: Scope<Timestamp = (mz_repr::Timestamp, Subtime)>,
{
let scope = fetched.scope();
let mut builder = OperatorBuilder::new(
format!("persist_source::decode_and_mfp({})", name),
scope.clone(),
);
let operator_info = builder.operator_info();
let mut fetched_input = builder.new_input(fetched, Pipeline);
let (mut updates_output, updates_stream) =
builder.new_output::<ConsolidatingContainerBuilder<_>>();
// Re-used state for processing and building rows.
let mut datum_vec = mz_repr::DatumVec::new();
let mut row_builder = Row::default();
// Extract the MFP if it exists; leave behind an identity MFP in that case.
let map_filter_project = map_filter_project.as_mut().map(|mfp| mfp.take());
builder.build(move |_caps| {
let name = name.to_owned();
// Acquire an activator to reschedule the operator when it has unfinished work.
let activations = scope.activations();
let activator = Activator::new(operator_info.address, activations);
// Maintain a list of work to do
let mut pending_work = std::collections::VecDeque::new();
move |_frontier| {
fetched_input.for_each(|time, data| {
let capability = time.retain();
for fetched_blob in data.drain(..) {
pending_work.push_back(PendingWork {
capability: capability.clone(),
part: PendingPart::Unparsed(fetched_blob),
})
}
});
// Get dyncfg values once per schedule to amortize the cost of
// loading the atomics.
let yield_fuel = cfg.storage_source_decode_fuel();
let yield_fn = |_, work| work >= yield_fuel;
let optimize_ignored_data_decode = cfg.optimize_ignored_data_decode();
let mut work = 0;
let start_time = Instant::now();
let mut output = updates_output.activate();
while !pending_work.is_empty() && !yield_fn(start_time, work) {
let done = pending_work.front_mut().unwrap().do_work(
&mut work,
&name,
optimize_ignored_data_decode,
start_time,
yield_fn,
&until,
map_filter_project.as_ref(),
&mut datum_vec,
&mut row_builder,
&mut output,
);
if done {
pending_work.pop_front();
}
}
if !pending_work.is_empty() {
activator.activate();
}
}
});
updates_stream
}
/// Pending work to read from fetched parts
struct PendingWork {
/// The time at which the work should happen.
capability: Capability<(mz_repr::Timestamp, Subtime)>,
/// Pending fetched part.
part: PendingPart,
}
enum PendingPart {
Unparsed(FetchedBlob<SourceData, (), Timestamp, Diff>),
Parsed {
part: ShardSourcePart<SourceData, (), Timestamp, Diff>,
error_free: bool,
},
}
impl PendingPart {
/// Returns the contained `FetchedPart`, first parsing it from a
/// `FetchedBlob` if necessary.
///
/// Also returns a bool, which is true if the part is known (from pushdown
/// stats) to be free of `SourceData(Err(_))`s. It will be false if the part
/// is known to contain errors or if it's unknown.
fn part_mut(&mut self) -> (&mut FetchedPart<SourceData, (), Timestamp, Diff>, bool) {
match self {
PendingPart::Unparsed(x) => {
let error_free = error_free(x.stats(), "err").unwrap_or(false);
*self = PendingPart::Parsed {
part: x.parse(),
error_free,
};
// Won't recurse any further.
self.part_mut()
}
PendingPart::Parsed { part, error_free } => (&mut part.part, *error_free),
}
}
}
impl PendingWork {
/// Perform work, reading from the fetched part, decoding, and sending outputs, while checking
/// `yield_fn` whether more fuel is available.
fn do_work<P, YFn>(
&mut self,
work: &mut usize,
name: &str,
optimize_ignored_data_decode: bool,
start_time: Instant,
yield_fn: YFn,
until: &Antichain<Timestamp>,
map_filter_project: Option<&MfpPlan>,
datum_vec: &mut DatumVec,
row_builder: &mut Row,
output: &mut OutputHandleCore<
'_,
(mz_repr::Timestamp, Subtime),
ConsolidatingContainerBuilder<
Vec<(
Result<Row, DataflowError>,
(mz_repr::Timestamp, Subtime),
Diff,
)>,
>,
P,
>,
) -> bool
where
P: Push<
Bundle<
(mz_repr::Timestamp, Subtime),
Vec<(
Result<Row, DataflowError>,
(mz_repr::Timestamp, Subtime),
Diff,
)>,
>,
>,
YFn: Fn(Instant, usize) -> bool,
{
let mut session = output.session_with_builder(&self.capability);
let (fetched_part, part_is_error_free) = self.part.part_mut();
let is_filter_pushdown_audit = fetched_part.is_filter_pushdown_audit();
let mut row_buf = None;
let row_override = map_filter_project
.as_ref()
.map(|p| optimize_ignored_data_decode && part_is_error_free && p.ignores_input())
.unwrap_or(false)
.then(|| (SourceData(Ok(Row::default())), ()));
while let Some(((key, val), time, diff)) =
fetched_part.next_with_storage(&mut row_buf, &mut None, row_override.clone())
{
if until.less_equal(&time) {
continue;
}
match (key, val) {
(Ok(SourceData(Ok(row))), Ok(())) => {
if let Some(mfp) = map_filter_project {
// We originally accounted work as the number of outputs, to give downstream
// operators a chance to reduce down anything we've emitted. This mfp call
// might have a restrictive filter, which would have been counted as no
// work. However, in practice, we've been decode_and_mfp be a source of
// interactivity loss during rehydration, so we now also count each mfp
// evaluation against our fuel.
*work += 1;
let arena = mz_repr::RowArena::new();
let mut datums_local = datum_vec.borrow_with(&row);
for result in mfp.evaluate(
&mut datums_local,
&arena,
time,
diff,
|time| !until.less_equal(time),
row_builder,
) {
if let Some(_stats) = &is_filter_pushdown_audit {
// NB: The tag added by this scope is used for alerting. The panic
// message may be changed arbitrarily, but the tag key and val must
// stay the same.
sentry::with_scope(
|scope| {
scope
.set_tag("alert_id", "persist_pushdown_audit_violation")
},
|| {
// TODO: include more (redacted) information here.
panic!(
"persist filter pushdown correctness violation! {}",
name
);
},
);
}
match result {
Ok((row, time, diff)) => {
// Additional `until` filtering due to temporal filters.
if !until.less_equal(&time) {
let mut emit_time = *self.capability.time();
emit_time.0 = time;
session.give((Ok(row), emit_time, diff));
*work += 1;
}
}
Err((err, time, diff)) => {
// Additional `until` filtering due to temporal filters.
if !until.less_equal(&time) {
let mut emit_time = *self.capability.time();
emit_time.0 = time;
session.give((Err(err), emit_time, diff));
*work += 1;
}
}
}
}
// At the moment, this is the only case where we can re-use the allocs for
// the `SourceData`/`Row` we decoded. This could be improved if this timely
// operator used a different container than `Vec<Row>`.
drop(datums_local);
row_buf.replace(SourceData(Ok(row)));
} else {
let mut emit_time = *self.capability.time();
emit_time.0 = time;
session.give((Ok(row), emit_time, diff));
*work += 1;
}
}
(Ok(SourceData(Err(err))), Ok(())) => {
let mut emit_time = *self.capability.time();
emit_time.0 = time;
session.give((Err(err), emit_time, diff));
*work += 1;
}
// TODO(petrosagg): error handling
(Err(_), Ok(_)) | (Ok(_), Err(_)) | (Err(_), Err(_)) => {
panic!("decoding failed")
}
}
if yield_fn(start_time, *work) {
return false;
}
}
true
}
}
/// A trait representing a type that can be used in `backpressure`.
pub trait Backpressureable: Clone + 'static {
/// Return the weight of the object, in bytes.
fn byte_size(&self) -> usize;
}
impl Backpressureable for (usize, SerdeLeasedBatchPart) {
fn byte_size(&self) -> usize {
self.1.encoded_size_bytes()
}
}
/// Flow control configuration.
#[derive(Debug)]
pub struct FlowControl<G: Scope> {
/// Stream providing in-flight frontier updates.
///
/// As implied by its type, this stream never emits data, only progress updates.
///
/// TODO: Replace `Infallible` with `!` once the latter is stabilized.
pub progress_stream: Stream<G, Infallible>,
/// Maximum number of in-flight bytes.
pub max_inflight_bytes: usize,
/// The minimum range of timestamps (be they granular or not) that must be emitted,
/// ignoring `max_inflight_bytes` to ensure forward progress is made.
pub summary: <G::Timestamp as TimelyTimestamp>::Summary,
/// Optional metrics for the `backpressure` operator to keep up-to-date.
pub metrics: Option<BackpressureMetrics>,
}
/// Apply flow control to the `data` input, based on the given `FlowControl`.
///
/// The `FlowControl` should have a `progress_stream` that is the pristine, unaltered
/// frontier of the downstream operator we want to backpressure from, a `max_inflight_bytes`,
/// and a `summary`. Note that the `data` input expects all the second part of the tuple
/// timestamp to be 0, and all data to be on the `chosen_worker` worker.
///
/// The `summary` represents the _minimum_ range of timestamps that needs to be emitted before
/// reasoning about `max_inflight_bytes`. In practice this means that we may overshoot
/// `max_inflight_bytes`.
///
/// The implementation of this operator is very subtle. Many inline comments have been added.
pub fn backpressure<T, G, O>(
scope: &mut G,
name: &str,
data: &Stream<G, O>,
flow_control: FlowControl<G>,
chosen_worker: usize,
// A probe used to inspect this operator during unit-testing
probe: Option<UnboundedSender<(Antichain<(T, Subtime)>, usize, usize)>>,
) -> (Stream<G, O>, PressOnDropButton)
where
T: TimelyTimestamp + Lattice + Codec64 + TotalOrder,
G: Scope<Timestamp = (T, Subtime)>,
O: Backpressureable + std::fmt::Debug,
{
let worker_index = scope.index();
let (flow_control_stream, flow_control_max_bytes, metrics) = (
flow_control.progress_stream,
flow_control.max_inflight_bytes,
flow_control.metrics,
);
// Both the `flow_control` input and the data input are disconnected from the output. We manually
// manage the output's frontier using a `CapabilitySet`. Note that we also adjust the
// `flow_control` progress stream using the `summary` here, using a `feedback` operator in a
// non-circular fashion.
let (handle, summaried_flow) = scope.feedback(flow_control.summary.clone());
flow_control_stream.connect_loop(handle);
let mut builder = AsyncOperatorBuilder::new(
format!("persist_source_backpressure({})", name),
scope.clone(),
);
let (data_output, data_stream) = builder.new_output();
let mut data_input = builder.new_disconnected_input(data, Pipeline);
let mut flow_control_input = builder.new_disconnected_input(&summaried_flow, Pipeline);
// Helper method used to synthesize current and next frontier for ordered times.
fn synthesize_frontiers<T: PartialOrder + Clone>(
mut frontier: Antichain<(T, Subtime)>,
mut time: (T, Subtime),
part_number: &mut u64,
) -> (
(T, Subtime),
Antichain<(T, Subtime)>,
Antichain<(T, Subtime)>,
) {
let mut next_frontier = frontier.clone();
time.1 = Subtime(*part_number);
frontier.insert(time.clone());
*part_number += 1;
let mut next_time = time.clone();
next_time.1 = Subtime(*part_number);
next_frontier.insert(next_time);
(time, frontier, next_frontier)
}
// _Refine_ the data stream by amending the second input with the part number. This also
// ensures that we order the parts by time.
let data_input = async_stream::stream!({
let mut part_number = 0;
let mut parts: Vec<((T, Subtime), O)> = Vec::new();
loop {
match data_input.next().await {
None => {
let empty = Antichain::new();
parts.sort_by_key(|val| val.0.clone());
for (part_time, d) in parts.drain(..) {
let (part_time, frontier, next_frontier) = synthesize_frontiers(
empty.clone(),
part_time.clone(),
&mut part_number,
);
yield Either::Right((part_time, d, frontier, next_frontier))
}
break;
}
Some(Event::Data(time, data)) => {
for d in data {
parts.push((time.clone(), d));
}
}
Some(Event::Progress(prog)) => {
let mut i = 0;
parts.sort_by_key(|val| val.0.clone());
// This can be replaced with `Vec::drain_filter` when it stabilizes.
// `drain_filter_swapping` doesn't work as it reorders the vec.
while i < parts.len() {
if !prog.less_equal(&parts[i].0) {
let (part_time, d) = parts.remove(i);
let (part_time, frontier, next_frontier) = synthesize_frontiers(
prog.clone(),
part_time.clone(),
&mut part_number,
);
yield Either::Right((part_time, d, frontier, next_frontier))
} else {
i += 1;
}
}
yield Either::Left(prog)
}
}
}
});
let shutdown_button = builder.build(move |caps| async move {
// The output capability.
let mut cap_set = CapabilitySet::from_elem(caps.into_element());
// The frontier of our output. This matches the `CapabilitySet` above.
let mut output_frontier = Antichain::from_elem(TimelyTimestamp::minimum());
// The frontier of the `flow_control` input.
let mut flow_control_frontier = Antichain::from_elem(TimelyTimestamp::minimum());
// Parts we have emitted, but have not yet retired (based on the `flow_control` edge).
let mut inflight_parts = Vec::new();
// Parts we have not yet emitted, but do participate in the `input_frontier`.
let mut pending_parts = std::collections::VecDeque::new();
// Only one worker is responsible for distributing parts
if worker_index != chosen_worker {
trace!(
"We are not the chosen worker ({}), exiting...",
chosen_worker
);
return;
}
tokio::pin!(data_input);
'emitting_parts: loop {
// At the beginning of our main loop, we determine the total size of
// inflight parts.
let inflight_bytes: usize = inflight_parts.iter().map(|(_, size)| size).sum();
// There are 2 main cases where we can continue to emit parts:
// - The total emitted bytes is less than `flow_control_max_bytes`.
// - The output frontier is not beyond the `flow_control_frontier`
//
// SUBTLE: in the latter case, we may arbitrarily go into the backpressure `else`
// block, as we wait for progress tracking to keep the `flow_control` frontier
// up-to-date. This is tested in unit-tests.
if inflight_bytes < flow_control_max_bytes
|| !PartialOrder::less_equal(&flow_control_frontier, &output_frontier)
{
let (time, part, next_frontier) =
if let Some((time, part, next_frontier)) = pending_parts.pop_front() {
(time, part, next_frontier)
} else {
match data_input.next().await {
Some(Either::Right((time, part, frontier, next_frontier))) => {
// Downgrade the output frontier to this part's time. This is useful
// "close" timestamp's from previous parts, even if we don't yet
// emit this part. Note that this is safe because `data_input` ensures
// time-ordering.
output_frontier = frontier;
cap_set.downgrade(output_frontier.iter());
// If the most recent value's time is _beyond_ the
// `flow_control` frontier (which takes into account the `summary`), we
// have emitted an entire `summary` worth of data, and can store this
// value for later.
if inflight_bytes >= flow_control_max_bytes
&& !PartialOrder::less_than(
&output_frontier,
&flow_control_frontier,
)
{
pending_parts.push_back((time, part, next_frontier));
continue 'emitting_parts;
}
(time, part, next_frontier)
}
Some(Either::Left(prog)) => {
output_frontier = prog;
cap_set.downgrade(output_frontier.iter());
continue 'emitting_parts;
}
None => {
if pending_parts.is_empty() {
break 'emitting_parts;
} else {
continue 'emitting_parts;
}
}
}
};
let byte_size = part.byte_size();
// Store the value with the _frontier_ the `flow_control_input` must reach
// to retire it. Note that if this `results_in` is `None`, then we
// are at `T::MAX`, and give up on flow_control entirely.
//
// SUBTLE: If we stop storing these parts, we will likely never check the
// `flow_control_input` ever again. This won't pile up data as that input
// only has frontier updates. There may be spurious activations from it though.
//
// Also note that we don't attempt to handle overflowing the `u64` part counter.
if let Some(emission_ts) = flow_control.summary.results_in(&time) {
inflight_parts.push((emission_ts, byte_size));
}
// Emit the data at the given time, and update the frontier and capabilities
// to just beyond the part.
data_output.give(&cap_set.delayed(&time), part);
if let Some(metrics) = &metrics {
metrics.emitted_bytes.inc_by(u64::cast_from(byte_size))
}
output_frontier = next_frontier;
cap_set.downgrade(output_frontier.iter())
} else {
if let Some(metrics) = &metrics {
metrics
.last_backpressured_bytes
.set(u64::cast_from(inflight_bytes))
}
let parts_count = inflight_parts.len();
// We've exhausted our budget, listen for updates to the flow_control
// input's frontier until we free up new budget. If we don't interact with
// with this side of the if statement, because the stream has no data, we
// don't cause unbounded buffering in timely.
let new_flow_control_frontier = match flow_control_input.next().await {
Some(Event::Progress(frontier)) => frontier,
Some(Event::Data(_, _)) => {
unreachable!("flow_control_input should not contain data")
}
None => Antichain::new(),
};
// Update the `flow_control_frontier` if its advanced.
flow_control_frontier.clone_from(&new_flow_control_frontier);
// Retire parts that are processed downstream.
let retired_parts = inflight_parts
.drain_filter_swapping(|(ts, _size)| !flow_control_frontier.less_equal(ts));
let (retired_size, retired_count): (usize, usize) = retired_parts
.fold((0, 0), |(accum_size, accum_count), (_ts, size)| {
(accum_size + size, accum_count + 1)
});
trace!(
"returning {} parts with {} bytes, frontier: {:?}",
retired_count,
retired_size,
flow_control_frontier,
);
if let Some(metrics) = &metrics {
metrics.retired_bytes.inc_by(u64::cast_from(retired_size))
}
// Optionally emit some information for tests to examine.
if let Some(probe) = probe.as_ref() {
let _ = probe.send((new_flow_control_frontier, parts_count, retired_count));
}
}
}
});
(data_stream, shutdown_button.press_on_drop())
}
#[cfg(test)]
mod tests {
use timely::container::CapacityContainerBuilder;
use timely::dataflow::operators::{Enter, Probe};
use tokio::sync::mpsc::unbounded_channel;
use tokio::sync::oneshot;
use super::*;
#[mz_ore::test]
fn test_backpressure_non_granular() {
use Step::*;
backpressure_runner(
vec![(50, Part(101)), (50, Part(102)), (100, Part(1))],
100,
(1, Subtime(0)),
vec![
// Assert we backpressure only after we have emitted
// the entire timestamp.
AssertOutputFrontier((50, Subtime(2))),
AssertBackpressured {
frontier: (1, Subtime(0)),
inflight_parts: 1,
retired_parts: 0,
},
AssertBackpressured {
frontier: (51, Subtime(0)),
inflight_parts: 1,
retired_parts: 0,
},
ProcessXParts(2),
AssertBackpressured {
frontier: (101, Subtime(0)),
inflight_parts: 2,
retired_parts: 2,
},
// Assert we make later progress once processing
// the parts.
AssertOutputFrontier((100, Subtime(3))),
],
true,
);
backpressure_runner(
vec![
(50, Part(10)),
(50, Part(10)),
(51, Part(100)),
(52, Part(1000)),
],
50,
(1, Subtime(0)),
vec![
// Assert we backpressure only after we emitted enough bytes
AssertOutputFrontier((51, Subtime(3))),
AssertBackpressured {
frontier: (1, Subtime(0)),
inflight_parts: 3,
retired_parts: 0,
},
ProcessXParts(3),
AssertBackpressured {
frontier: (52, Subtime(0)),
inflight_parts: 3,
retired_parts: 2,
},
AssertBackpressured {
frontier: (53, Subtime(0)),
inflight_parts: 1,
retired_parts: 1,
},
// Assert we make later progress once processing
// the parts.
AssertOutputFrontier((52, Subtime(4))),
],
true,
);
backpressure_runner(
vec![
(50, Part(98)),
(50, Part(1)),
(51, Part(10)),
(52, Part(100)),
// Additional parts at the same timestamp
(52, Part(10)),
(52, Part(10)),
(52, Part(10)),
(52, Part(100)),
// A later part with a later ts.
(100, Part(100)),
],
100,
(1, Subtime(0)),
vec![
AssertOutputFrontier((51, Subtime(3))),
// Assert we backpressure after we have emitted enough bytes.
// We assert twice here because we get updates as
// `flow_control` progresses from `(0, 0)`->`(0, 1)`-> a real frontier.
AssertBackpressured {
frontier: (1, Subtime(0)),
inflight_parts: 3,
retired_parts: 0,
},
AssertBackpressured {
frontier: (51, Subtime(0)),
inflight_parts: 3,
retired_parts: 0,
},
ProcessXParts(1),
// Our output frontier doesn't move, as the downstream frontier hasn't moved past
// 50.
AssertOutputFrontier((51, Subtime(3))),
// After we process all of `50`, we can start emitting data at `52`, but only until
// we exhaust out budget. We don't need to emit all of `52` because we have emitted
// all of `51`.
ProcessXParts(1),
AssertOutputFrontier((52, Subtime(4))),
AssertBackpressured {
frontier: (52, Subtime(0)),
inflight_parts: 3,
retired_parts: 2,
},
// After processing `50` and `51`, the minimum time is `52`, so we ensure that,
// regardless of byte count, we emit the entire time (but do NOT emit the part at
// time `100`.
ProcessXParts(1),
// Clear the previous `51` part, and start filling up `inflight_parts` with other
// parts at `52`
// This is an intermediate state.
AssertBackpressured {
frontier: (53, Subtime(0)),
inflight_parts: 2,
retired_parts: 1,
},
// After we process all of `52`, we can continue to the next time.
ProcessXParts(5),
AssertBackpressured {
frontier: (101, Subtime(0)),
inflight_parts: 5,
retired_parts: 5,
},
AssertOutputFrontier((100, Subtime(9))),
],
true,
);
}
#[mz_ore::test]
fn test_backpressure_granular() {
use Step::*;
backpressure_runner(
vec![(50, Part(101)), (50, Part(101))],
100,
(0, Subtime(1)),
vec![
// Advance our frontier to outputting a single part.
AssertOutputFrontier((50, Subtime(1))),
// Receive backpressure updates until our frontier is up-to-date but
// not beyond the parts (while considering the summary).
AssertBackpressured {
frontier: (0, Subtime(1)),
inflight_parts: 1,
retired_parts: 0,
},
AssertBackpressured {
frontier: (50, Subtime(1)),
inflight_parts: 1,
retired_parts: 0,
},
// Process that part.
ProcessXParts(1),
// Assert that we clear the backpressure status
AssertBackpressured {
frontier: (50, Subtime(2)),
inflight_parts: 1,
retired_parts: 1,
},
// Ensure we make progress to the next part.
AssertOutputFrontier((50, Subtime(2))),
],
false,
);
backpressure_runner(
vec![
(50, Part(10)),
(50, Part(10)),
(51, Part(35)),
(52, Part(100)),
],
50,
(0, Subtime(1)),
vec![
// we can emit 3 parts before we hit the backpressure limit
AssertOutputFrontier((51, Subtime(3))),
AssertBackpressured {
frontier: (0, Subtime(1)),
inflight_parts: 3,
retired_parts: 0,
},
AssertBackpressured {
frontier: (50, Subtime(1)),
inflight_parts: 3,
retired_parts: 0,
},
// Retire the single part.
ProcessXParts(1),
AssertBackpressured {
frontier: (50, Subtime(2)),
inflight_parts: 3,
retired_parts: 1,
},
// Ensure we make progress, and then
// can retire the next 2 parts.
AssertOutputFrontier((52, Subtime(4))),
ProcessXParts(2),
AssertBackpressured {
frontier: (52, Subtime(4)),
inflight_parts: 3,
retired_parts: 2,
},
],
false,
);
}
type Time = (u64, Subtime);
#[derive(Clone, Debug)]
struct Part(usize);
impl Backpressureable for Part {
fn byte_size(&self) -> usize {
self.0
}
}
/// Actions taken by `backpressure_runner`.
enum Step {
/// Assert that the output frontier of the `backpressure` operator has AT LEAST made it
/// this far. This is a single time because we assume
AssertOutputFrontier(Time),
/// Assert that we have entered the backpressure flow in the `backpressure` operator. This
/// allows us to assert what feedback frontier we got to, and how many inflight parts we
/// retired.
AssertBackpressured {
frontier: Time,
inflight_parts: usize,
retired_parts: usize,
},
/// Process X parts in the downstream operator. This affects the feedback frontier.
ProcessXParts(usize),
}
/// A function that runs the `steps` to ensure that `backpressure` works as expected.
fn backpressure_runner(
// The input data to the `backpressure` operator
input: Vec<(u64, Part)>,
// The maximum inflight bytes the `backpressure` operator allows through.
max_inflight_bytes: usize,
// The feedback summary used by the `backpressure` operator.
summary: Time,
// List of steps to run through.
steps: Vec<Step>,
// Whether or not to consume records in the non-granular scope. This is useful when the
// `summary` is something like `(1, 0)`.
non_granular_consumer: bool,
) {
timely::execute::execute_directly(move |worker| {
let (backpressure_probe, consumer_tx, mut backpressure_status_rx, finalizer_tx, _token) =
// Set up the top-level non-granular scope.
worker.dataflow::<u64, _, _>(|scope| {
let (non_granular_feedback_handle, non_granular_feedback) =
if non_granular_consumer {
let (h, f) = scope.feedback(Default::default());
(Some(h), Some(f))
} else {
(None, None)
};
let (
backpressure_probe,
consumer_tx,
backpressure_status_rx,
token,
backpressured,
finalizer_tx,
) = scope.scoped::<(u64, Subtime), _, _>("hybrid", |scope| {
let (input, finalizer_tx) =
iterator_operator(scope.clone(), input.into_iter());
let (flow_control, granular_feedback_handle) = if non_granular_consumer {
(
FlowControl {
progress_stream: non_granular_feedback.unwrap().enter(scope),
max_inflight_bytes,
summary,
metrics: None
},
None,
)
} else {
let (granular_feedback_handle, granular_feedback) =
scope.feedback(Default::default());
(
FlowControl {
progress_stream: granular_feedback,
max_inflight_bytes,
summary,
metrics: None,
},
Some(granular_feedback_handle),
)
};
let (backpressure_status_tx, backpressure_status_rx) = unbounded_channel();
let (backpressured, token) = backpressure(
scope,
"test",
&input,
flow_control,
0,
Some(backpressure_status_tx),
);
// If we want to granularly consume the output, we setup the consumer here.
let tx = if !non_granular_consumer {
Some(consumer_operator(
scope.clone(),
&backpressured,
granular_feedback_handle.unwrap(),
))
} else {
None
};
(
backpressured.probe(),
tx,
backpressure_status_rx,
token,
backpressured.leave(),
finalizer_tx,
)
});
// If we want to non-granularly consume the output, we setup the consumer here.
let consumer_tx = if non_granular_consumer {
consumer_operator(
scope.clone(),
&backpressured,
non_granular_feedback_handle.unwrap(),
)
} else {
consumer_tx.unwrap()
};
(
backpressure_probe,
consumer_tx,
backpressure_status_rx,
finalizer_tx,
token,
)
});
use Step::*;
for step in steps {
match step {
AssertOutputFrontier(time) => {
eprintln!("checking advance to {time:?}");
backpressure_probe.with_frontier(|front| {
eprintln!("current backpressure output frontier: {front:?}");
});
while backpressure_probe.less_than(&time) {
worker.step();
backpressure_probe.with_frontier(|front| {
eprintln!("current backpressure output frontier: {front:?}");
});
std::thread::sleep(std::time::Duration::from_millis(25));
}
}
ProcessXParts(parts) => {
eprintln!("processing {parts:?} parts");
for _ in 0..parts {
consumer_tx.send(()).unwrap();
}
}
AssertBackpressured {
frontier,
inflight_parts,
retired_parts,
} => {
let frontier = Antichain::from_elem(frontier);
eprintln!(
"asserting backpressured at {frontier:?}, with {inflight_parts:?} inflight parts \
and {retired_parts:?} retired"
);
let (new_frontier, new_count, new_retired_count) = loop {
if let Ok(val) = backpressure_status_rx.try_recv() {
break val;
}
worker.step();
std::thread::sleep(std::time::Duration::from_millis(25));
};
assert_eq!(
(frontier, inflight_parts, retired_parts),
(new_frontier, new_count, new_retired_count)
);
}
}
}
// Send the input to the empty frontier.
let _ = finalizer_tx.send(());
});
}
/// An operator that emits `Part`'s at the specified timestamps. Does not
/// drop its capability until it gets a signal from the `Sender` it returns.
fn iterator_operator<
G: Scope<Timestamp = (u64, Subtime)>,
I: Iterator<Item = (u64, Part)> + 'static,
>(
scope: G,
mut input: I,
) -> (Stream<G, Part>, oneshot::Sender<()>) {
let (finalizer_tx, finalizer_rx) = oneshot::channel();
let mut iterator = AsyncOperatorBuilder::new("iterator".to_string(), scope);
let (output_handle, output) = iterator.new_output::<CapacityContainerBuilder<Vec<Part>>>();
iterator.build(|mut caps| async move {
let mut capability = Some(caps.pop().unwrap());
let mut last = None;
while let Some(element) = input.next() {
let time = element.0.clone();
let part = element.1;
last = Some((time, Subtime(0)));
output_handle.give(&capability.as_ref().unwrap().delayed(&last.unwrap()), part);
}
if let Some(last) = last {
capability
.as_mut()
.unwrap()
.downgrade(&(last.0 + 1, last.1));
}
let _ = finalizer_rx.await;
capability.take();
});
(output, finalizer_tx)
}
/// An operator that consumes its input ONLY when given a signal to do from
/// the `UnboundedSender` it returns. Each `send` corresponds with 1 `Data` event
/// being processed. Also connects the `feedback` handle to its output.
fn consumer_operator<G: Scope, O: Backpressureable + std::fmt::Debug>(
scope: G,
input: &Stream<G, O>,
feedback: timely::dataflow::operators::feedback::Handle<G, Vec<std::convert::Infallible>>,
) -> UnboundedSender<()> {
let (tx, mut rx) = unbounded_channel::<()>();
let mut consumer = AsyncOperatorBuilder::new("consumer".to_string(), scope);
let (output_handle, output) =
consumer.new_output::<CapacityContainerBuilder<Vec<std::convert::Infallible>>>();
let mut input = consumer.new_input_for(input, Pipeline, &output_handle);
consumer.build(|_caps| async move {
while let Some(()) = rx.recv().await {
// Consume exactly one messages (unless the input is exhausted).
while let Some(Event::Progress(_)) = input.next().await {}
}
});
output.connect_loop(feedback);
tx
}
}