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 1457 1458 1459 1460 1461 1462 1463 1464 1465 1466 1467 1468 1469 1470 1471 1472 1473 1474 1475 1476 1477 1478 1479 1480 1481 1482 1483 1484 1485 1486 1487 1488 1489 1490 1491 1492 1493 1494 1495 1496 1497 1498 1499 1500 1501 1502 1503 1504 1505 1506 1507 1508 1509 1510 1511 1512 1513 1514 1515 1516 1517 1518 1519 1520 1521 1522 1523 1524 1525 1526 1527 1528 1529 1530 1531 1532 1533 1534 1535 1536 1537 1538 1539 1540 1541 1542 1543 1544 1545 1546 1547 1548 1549 1550 1551 1552 1553 1554 1555 1556 1557 1558 1559 1560 1561 1562 1563 1564 1565 1566 1567 1568 1569 1570 1571 1572 1573 1574 1575 1576 1577 1578 1579 1580 1581 1582 1583 1584 1585 1586 1587 1588 1589 1590 1591 1592 1593 1594 1595 1596 1597 1598 1599 1600 1601 1602 1603 1604 1605 1606 1607 1608 1609 1610 1611 1612 1613 1614 1615 1616 1617 1618 1619 1620 1621 1622 1623 1624 1625 1626 1627 1628 1629 1630 1631 1632 1633 1634 1635 1636 1637 1638 1639 1640 1641 1642 1643 1644 1645 1646 1647 1648 1649 1650 1651 1652 1653 1654 1655 1656 1657 1658 1659 1660 1661 1662 1663 1664 1665 1666 1667 1668 1669 1670 1671 1672 1673 1674 1675 1676 1677 1678 1679 1680 1681 1682 1683 1684 1685 1686 1687 1688 1689 1690 1691 1692 1693 1694 1695 1696 1697 1698 1699 1700 1701 1702 1703 1704 1705 1706 1707 1708 1709 1710 1711 1712 1713 1714 1715 1716 1717 1718 1719 1720 1721 1722 1723 1724 1725 1726 1727 1728 1729 1730 1731 1732 1733 1734 1735 1736 1737 1738 1739 1740 1741 1742 1743 1744 1745 1746 1747 1748 1749 1750 1751 1752 1753 1754 1755 1756 1757 1758 1759 1760 1761 1762 1763 1764 1765 1766 1767 1768 1769 1770 1771 1772 1773 1774 1775 1776 1777 1778 1779 1780 1781 1782 1783 1784 1785 1786 1787 1788 1789 1790 1791 1792 1793 1794 1795 1796 1797 1798 1799 1800 1801 1802 1803 1804 1805 1806 1807 1808 1809 1810 1811 1812 1813 1814 1815 1816 1817 1818 1819 1820 1821 1822 1823 1824 1825 1826 1827 1828 1829 1830 1831 1832 1833 1834 1835 1836 1837 1838 1839 1840 1841 1842 1843 1844 1845 1846 1847 1848 1849 1850 1851 1852 1853 1854 1855 1856 1857 1858 1859 1860 1861 1862 1863 1864 1865 1866 1867 1868 1869 1870 1871 1872 1873 1874 1875 1876 1877 1878 1879 1880 1881 1882 1883 1884 1885 1886 1887 1888 1889 1890 1891 1892 1893 1894 1895 1896 1897 1898 1899 1900 1901 1902 1903 1904 1905 1906 1907 1908 1909 1910 1911 1912 1913 1914 1915 1916 1917 1918 1919 1920 1921 1922 1923 1924 1925 1926 1927 1928 1929 1930 1931 1932 1933 1934 1935 1936 1937 1938 1939 1940 1941 1942 1943 1944 1945 1946 1947 1948 1949 1950 1951 1952 1953 1954 1955 1956 1957 1958 1959 1960 1961 1962 1963 1964 1965 1966 1967 1968 1969 1970 1971 1972 1973 1974 1975 1976 1977 1978 1979 1980 1981 1982 1983 1984 1985 1986 1987 1988 1989 1990 1991 1992 1993 1994 1995 1996 1997 1998 1999 2000 2001 2002 2003 2004 2005 2006 2007 2008 2009
// 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.
//! Tokio tasks (and support machinery) for maintaining storage-managed
//! collections.
//!
//! We differentiate between append-only collections and differential
//! collections. The intent is that knowing the type allows being more
//! intentional about what state we keep in memory and how we work when in
//! read-only mode / during zero-downtime upgrades.
//!
//! ## Append-only collections
//!
//! Writers only append blind writes. Those writes never fail. It does not
//! matter at what timestamp they happen (to a degree, but ...).
//!
//! While in read-only mode, the append-only write task can immediately write
//! updates out as batches, but only append them when going out of read-only
//! mode.
//!
//! ## Differential collections
//!
//! These are very similar to the self-correcting persist_sink. We have an
//! in-memory desired state and continually make it so that persist matches
//! desired. As described below (in the task implementation), we could do this
//! in a memory efficient way by keeping open a persist read handle and
//! continually updating/consolidating our desired collection. This way, we
//! would be memory-efficient even in read-only mode.
//!
//! This is an evolution of the current design where, on startup, we bring the
//! persist collection into a known state (mostly by retracting everything) and
//! then assume that this `envd` is the only writer. We panic when that is ever
//! not the case, which we notice when the upper of a collection changes
//! unexpectedly. With this new design we can instead continually update our
//! view of the persist shard and emit updates when needed, when desired
//! changed.
//!
//! NOTE: As it is, we always keep all of desired in memory. Only when told to
//! go out of read-only mode would we start attempting to write.
//!
//! ## Read-only mode
//!
//! When [`CollectionManager`] is in read-only mode it cannot write out to
//! persist. It will, however, maintain the `desired` state of differential
//! collections so that we can immediately start writing out updates when going
//! out of read-only mode.
//!
//! For append-only collections we either panic, in the case of
//! [`CollectionManager::blind_write`], or report back a
//! [`StorageError::ReadOnly`] when trying to append through a
//! [`MonotonicAppender`] returned from
//! [`CollectionManager::monotonic_appender`].
use std::any::Any;
use std::cmp::Reverse;
use std::collections::{BTreeMap, BinaryHeap};
use std::fmt::Debug;
use std::ops::ControlFlow;
use std::pin::Pin;
use std::str::FromStr;
use std::sync::atomic::{AtomicU64, Ordering};
use std::sync::{Arc, Mutex};
use anyhow::{anyhow, bail};
use chrono::{DateTime, Utc};
use differential_dataflow::consolidation;
use differential_dataflow::lattice::Lattice;
use futures::future::BoxFuture;
use futures::stream::StreamExt;
use futures::{Future, FutureExt};
use mz_dyncfg::ConfigSet;
use mz_ore::now::{EpochMillis, NowFn};
use mz_ore::retry::Retry;
use mz_ore::soft_panic_or_log;
use mz_ore::task::AbortOnDropHandle;
use mz_ore::vec::VecExt;
use mz_persist_client::cache::PersistClientCache;
use mz_persist_client::read::ReadHandle;
use mz_persist_client::write::WriteHandle;
use mz_persist_types::Codec64;
use mz_repr::adt::timestamp::CheckedTimestamp;
use mz_repr::{ColumnName, Diff, GlobalId, Row, TimestampManipulation};
use mz_storage_client::client::{AppendOnlyUpdate, Status, TimestamplessUpdate};
use mz_storage_client::controller::{IntrospectionType, MonotonicAppender, StorageWriteOp};
use mz_storage_client::healthcheck::{
MZ_SINK_STATUS_HISTORY_DESC, MZ_SOURCE_STATUS_HISTORY_DESC, REPLICA_METRICS_HISTORY_DESC,
WALLCLOCK_LAG_HISTORY_DESC,
};
use mz_storage_client::metrics::StorageControllerMetrics;
use mz_storage_client::statistics::{SinkStatisticsUpdate, SourceStatisticsUpdate};
use mz_storage_client::storage_collections::StorageCollections;
use mz_storage_types::controller::InvalidUpper;
use mz_storage_types::dyncfgs::{
REPLICA_METRICS_HISTORY_RETENTION_INTERVAL, WALLCLOCK_LAG_HISTORY_RETENTION_INTERVAL,
};
use mz_storage_types::parameters::{
StorageParameters, STORAGE_MANAGED_COLLECTIONS_BATCH_DURATION_DEFAULT,
};
use mz_storage_types::sources::SourceData;
use mz_txn_wal::txn_read::TxnsRead;
use timely::progress::{Antichain, Timestamp};
use tokio::sync::{mpsc, oneshot, watch};
use tokio::time::{Duration, Instant};
use tracing::{debug, error, info};
use crate::{
collection_mgmt, privatelink_status_history_desc, replica_status_history_desc,
sink_status_history_desc, snapshot, snapshot_statistics, source_status_history_desc,
statistics, StatusHistoryDesc, StatusHistoryRetentionPolicy, StorageError,
};
// Default rate at which we advance the uppers of managed collections.
const DEFAULT_TICK_MS: u64 = 1_000;
/// A channel for sending writes to a differential collection.
type DifferentialWriteChannel<T> =
mpsc::UnboundedSender<(StorageWriteOp, oneshot::Sender<Result<(), StorageError<T>>>)>;
/// A channel for sending writes to an append-only collection.
type AppendOnlyWriteChannel<T> = mpsc::UnboundedSender<(
Vec<AppendOnlyUpdate>,
oneshot::Sender<Result<(), StorageError<T>>>,
)>;
type WriteTask = AbortOnDropHandle<()>;
type ShutdownSender = oneshot::Sender<()>;
/// Types of storage-managed/introspection collections:
///
/// Append-only: Only accepts blind writes, writes that can be applied at any
/// timestamp and don’t depend on current collection contents.
///
/// Pseudo append-only: We treat them largely as append-only collections but
/// periodically (currently on bootstrap) retract old updates from them.
///
/// Differential: at any given time `t` , collection contents mirrors some
/// (small cardinality) state. The cardinality of the collection stays constant
/// if the thing that is mirrored doesn’t change in cardinality. At steady
/// state, updates always come in pairs of retractions/additions.
pub enum CollectionManagerKind {
AppendOnly,
Differential,
}
#[derive(Debug, Clone)]
pub struct CollectionManager<T>
where
T: Timestamp + Lattice + Codec64 + TimestampManipulation,
{
/// When a [`CollectionManager`] is in read-only mode it must not affect any
/// changes to external state.
read_only: bool,
// WIP: Name TBD! I thought about `managed_collections`, `ivm_collections`,
// `self_correcting_collections`.
/// These are collections that we write to by adding/removing updates to an
/// internal _desired_ collection. The `CollectionManager` continually makes
/// sure that collection contents (in persist) match the desired state.
differential_collections:
Arc<Mutex<BTreeMap<GlobalId, (DifferentialWriteChannel<T>, WriteTask, ShutdownSender)>>>,
/// Collections that we only append to using blind-writes.
///
/// Every write succeeds at _some_ timestamp, and we never check what the
/// actual contents of the collection (in persist) are.
append_only_collections:
Arc<Mutex<BTreeMap<GlobalId, (AppendOnlyWriteChannel<T>, WriteTask, ShutdownSender)>>>,
/// Amount of time we'll wait before sending a batch of inserts to Persist, for user
/// collections.
user_batch_duration_ms: Arc<AtomicU64>,
now: NowFn,
}
/// The `CollectionManager` provides two complementary functions:
/// - Providing an API to append values to a registered set of collections.
/// For this usecase:
/// - The `CollectionManager` expects to be the only writer.
/// - Appending to a closed collection panics
/// - Automatically advancing the timestamp of managed collections every
/// second. For this usecase:
/// - The `CollectionManager` handles contention by permitting and ignoring errors.
/// - Closed collections will not panic if they continue receiving these requests.
impl<T> CollectionManager<T>
where
T: Timestamp + Lattice + Codec64 + From<EpochMillis> + TimestampManipulation,
{
pub(super) fn new(read_only: bool, now: NowFn) -> CollectionManager<T> {
let batch_duration_ms: u64 = STORAGE_MANAGED_COLLECTIONS_BATCH_DURATION_DEFAULT
.as_millis()
.try_into()
.expect("known to fit");
CollectionManager {
read_only,
differential_collections: Arc::new(Mutex::new(BTreeMap::new())),
append_only_collections: Arc::new(Mutex::new(BTreeMap::new())),
user_batch_duration_ms: Arc::new(AtomicU64::new(batch_duration_ms)),
now,
}
}
/// Updates the duration we'll wait to batch events for user owned collections.
pub fn update_user_batch_duration(&self, duration: Duration) {
tracing::info!(?duration, "updating user batch duration");
let millis: u64 = duration.as_millis().try_into().unwrap_or(u64::MAX);
self.user_batch_duration_ms.store(millis, Ordering::Relaxed);
}
/// Registers a new _differential collection_.
///
/// The [CollectionManager] will automatically advance the upper of every
/// registered collection every second.
///
/// Update the `desired` state of a differential collection using
/// [Self::differential_write].
pub(super) fn register_differential_collection<R>(
&self,
id: GlobalId,
write_handle: WriteHandle<SourceData, (), T, Diff>,
read_handle_fn: R,
force_writable: bool,
introspection_config: DifferentialIntrospectionConfig<T>,
) where
R: FnMut() -> Pin<Box<dyn Future<Output = ReadHandle<SourceData, (), T, Diff>> + Send>>
+ Send
+ Sync
+ 'static,
{
let mut guard = self
.differential_collections
.lock()
.expect("collection_mgmt panicked");
// Check if this collection is already registered.
if let Some((_writer, task, _shutdown_tx)) = guard.get(&id) {
// The collection is already registered and the task is still running so nothing to do.
if !task.is_finished() {
// TODO(parkmycar): Panic here if we never see this error in production.
tracing::error!("Registered a collection twice! {id:?}");
return;
}
}
let read_only = self.get_read_only(id, force_writable);
// Spawns a new task so we can write to this collection.
let writer_and_handle = DifferentialWriteTask::spawn(
id,
write_handle,
read_handle_fn,
read_only,
self.now.clone(),
introspection_config,
);
let prev = guard.insert(id, writer_and_handle);
// Double check the previous task was actually finished.
if let Some((_, prev_task, _)) = prev {
assert!(
prev_task.is_finished(),
"should only spawn a new task if the previous is finished"
);
}
}
/// Registers a new _append-only collection_.
///
/// The [CollectionManager] will automatically advance the upper of every
/// registered collection every second.
pub(super) fn register_append_only_collection(
&self,
id: GlobalId,
write_handle: WriteHandle<SourceData, (), T, Diff>,
force_writable: bool,
introspection_config: Option<AppendOnlyIntrospectionConfig<T>>,
) {
let mut guard = self
.append_only_collections
.lock()
.expect("collection_mgmt panicked");
// Check if this collection is already registered.
if let Some((_writer, task, _shutdown_tx)) = guard.get(&id) {
// The collection is already registered and the task is still running so nothing to do.
if !task.is_finished() {
// TODO(parkmycar): Panic here if we never see this error in production.
tracing::error!("Registered a collection twice! {id:?}");
return;
}
}
let read_only = self.get_read_only(id, force_writable);
// Spawns a new task so we can write to this collection.
let writer_and_handle = AppendOnlyWriteTask::spawn(
id,
write_handle,
read_only,
self.now.clone(),
Arc::clone(&self.user_batch_duration_ms),
introspection_config,
);
let prev = guard.insert(id, writer_and_handle);
// Double check the previous task was actually finished.
if let Some((_, prev_task, _)) = prev {
assert!(
prev_task.is_finished(),
"should only spawn a new task if the previous is finished"
);
}
}
/// Unregisters the given collection.
///
/// Also waits until the `CollectionManager` has completed all outstanding work to ensure that
/// it has stopped referencing the provided `id`.
#[mz_ore::instrument(level = "debug")]
pub(super) fn unregister_collection(&self, id: GlobalId) -> BoxFuture<'static, ()> {
let prev = self
.differential_collections
.lock()
.expect("CollectionManager panicked")
.remove(&id);
// Wait for the task to complete before reporting as unregisted.
if let Some((_prev_writer, prev_task, shutdown_tx)) = prev {
// Notify the task it needs to shutdown.
//
// We can ignore errors here because they indicate the task is already done.
let _ = shutdown_tx.send(());
return Box::pin(prev_task.map(|_| ()));
}
let prev = self
.append_only_collections
.lock()
.expect("CollectionManager panicked")
.remove(&id);
// Wait for the task to complete before reporting as unregisted.
if let Some((_prev_writer, prev_task, shutdown_tx)) = prev {
// Notify the task it needs to shutdown.
//
// We can ignore errors here because they indicate the task is already done.
let _ = shutdown_tx.send(());
return Box::pin(prev_task.map(|_| ()));
}
Box::pin(futures::future::ready(()))
}
/// Appends `updates` to the append-only collection identified by `id`, at
/// _some_ timestamp. Does not wait for the append to complete.
///
/// # Panics
/// - If `id` does not belong to an append-only collections.
/// - If this [`CollectionManager`] is in read-only mode.
/// - If the collection closed.
pub(super) fn blind_write(&self, id: GlobalId, updates: Vec<AppendOnlyUpdate>) {
if self.read_only {
panic!("attempting blind write to {} while in read-only mode", id);
}
if !updates.is_empty() {
// Get the update channel in a block to make sure the Mutex lock is scoped.
let update_tx = {
let guard = self
.append_only_collections
.lock()
.expect("CollectionManager panicked");
let (update_tx, _, _) = guard.get(&id).expect("missing append-only collection");
update_tx.clone()
};
let (tx, _rx) = oneshot::channel();
update_tx.send((updates, tx)).expect("rx hung up");
}
}
/// Updates the desired collection state of the differential collection identified by
/// `id`. The underlying persist shard will reflect this change at
/// _some_point. Does not wait for the change to complete.
///
/// # Panics
/// - If `id` does not belong to a differential collection.
/// - If the collection closed.
pub(super) fn differential_write(&self, id: GlobalId, op: StorageWriteOp) {
if !op.is_empty_append() {
// Get the update channel in a block to make sure the Mutex lock is scoped.
let update_tx = {
let guard = self
.differential_collections
.lock()
.expect("CollectionManager panicked");
let (update_tx, _, _) = guard.get(&id).expect("missing differential collection");
update_tx.clone()
};
let (tx, _rx) = oneshot::channel();
update_tx.send((op, tx)).expect("rx hung up");
}
}
/// Appends the given `updates` to the differential collection identified by `id`.
///
/// # Panics
/// - If `id` does not belong to a differential collection.
/// - If the collection closed.
pub(super) fn differential_append(&self, id: GlobalId, updates: Vec<(Row, Diff)>) {
self.differential_write(id, StorageWriteOp::Append { updates })
}
/// Returns a [`MonotonicAppender`] that can be used to monotonically append updates to the
/// collection correlated with `id`.
pub(super) fn monotonic_appender(
&self,
id: GlobalId,
) -> Result<MonotonicAppender<T>, StorageError<T>> {
let guard = self
.append_only_collections
.lock()
.expect("CollectionManager panicked");
let tx = guard
.get(&id)
.map(|(tx, _, _)| tx.clone())
.ok_or(StorageError::IdentifierMissing(id))?;
Ok(MonotonicAppender::new(tx))
}
fn get_read_only(&self, id: GlobalId, force_writable: bool) -> bool {
if force_writable {
assert!(id.is_system(), "unexpected non-system global id: {id:?}");
false
} else {
self.read_only
}
}
}
pub(crate) struct DifferentialIntrospectionConfig<T>
where
T: Lattice + Codec64 + From<EpochMillis> + TimestampManipulation,
{
pub(crate) recent_upper: Antichain<T>,
pub(crate) introspection_type: IntrospectionType,
pub(crate) storage_collections: Arc<dyn StorageCollections<Timestamp = T> + Send + Sync>,
pub(crate) txns_read: TxnsRead<T>,
pub(crate) persist: Arc<PersistClientCache>,
pub(crate) collection_manager: collection_mgmt::CollectionManager<T>,
pub(crate) source_statistics: Arc<Mutex<statistics::SourceStatistics>>,
pub(crate) sink_statistics:
Arc<Mutex<BTreeMap<GlobalId, statistics::StatsState<SinkStatisticsUpdate>>>>,
pub(crate) statistics_interval: Duration,
pub(crate) statistics_interval_receiver: watch::Receiver<Duration>,
pub(crate) metrics: StorageControllerMetrics,
pub(crate) introspection_tokens: Arc<Mutex<BTreeMap<GlobalId, Box<dyn Any + Send + Sync>>>>,
}
/// A task that will make it so that the state in persist matches the desired
/// state and continuously bump the upper for the specified collection.
///
/// NOTE: This implementation is a bit clunky, and could be optimized by not keeping
/// all of desired in memory (see commend below). It is meant to showcase the
/// general approach.
struct DifferentialWriteTask<T, R>
where
T: Timestamp + Lattice + Codec64 + From<EpochMillis> + TimestampManipulation,
R: FnMut() -> Pin<Box<dyn Future<Output = ReadHandle<SourceData, (), T, Diff>> + Send>>
+ Send
+ 'static,
{
/// The collection that we are writing to.
id: GlobalId,
write_handle: WriteHandle<SourceData, (), T, Diff>,
/// For getting a [`ReadHandle`] to sync our state to persist contents.
read_handle_fn: R,
read_only: bool,
now: NowFn,
/// In the absence of updates, we regularly bump the upper to "now", on this
/// interval. This makes it so the collection remains readable at recent
/// timestamps.
upper_tick_interval: tokio::time::Interval,
/// Receiver for write commands. These change our desired state.
cmd_rx: mpsc::UnboundedReceiver<(StorageWriteOp, oneshot::Sender<Result<(), StorageError<T>>>)>,
/// We have to shut down when receiving from this.
shutdown_rx: oneshot::Receiver<()>,
/// The contents of the collection as it should be according to whoever is
/// driving us around.
// This is memory inefficient: we always keep a full copy of
// desired, so that we can re-derive a to_write if/when someone else
// writes to persist and we notice because of an upper conflict.
// This is optimized for the case where we rarely have more than one
// writer.
//
// We can optimize for a multi-writer case by keeping an open
// ReadHandle and continually reading updates from persist, updating
// a desired in place. Similar to the self-correcting persist_sink.
desired: Vec<(Row, i64)>,
/// Updates that we have to write when next writing to persist. This is
/// determined by looking at what is desired and what is in persist.
to_write: Vec<(Row, i64)>,
/// Current upper of the persist shard. We keep track of this so that we
/// realize when someone else writes to the shard, in which case we have to
/// update our state of the world, that is update our `to_write` based on
/// `desired` and the contents of the persist shard.
current_upper: T,
}
impl<T, R> DifferentialWriteTask<T, R>
where
T: Timestamp + Lattice + Codec64 + From<EpochMillis> + TimestampManipulation,
R: FnMut() -> Pin<Box<dyn Future<Output = ReadHandle<SourceData, (), T, Diff>> + Send>>
+ Send
+ Sync
+ 'static,
{
/// Spawns a [`DifferentialWriteTask`] in an [`mz_ore::task`] and returns
/// handles for interacting with it.
fn spawn(
id: GlobalId,
write_handle: WriteHandle<SourceData, (), T, Diff>,
read_handle_fn: R,
read_only: bool,
now: NowFn,
introspection_config: DifferentialIntrospectionConfig<T>,
) -> (DifferentialWriteChannel<T>, WriteTask, ShutdownSender) {
let (tx, rx) = mpsc::unbounded_channel();
let (shutdown_tx, shutdown_rx) = oneshot::channel();
let upper_tick_interval = tokio::time::interval(Duration::from_millis(DEFAULT_TICK_MS));
let current_upper = T::minimum();
let task = Self {
id,
write_handle,
read_handle_fn,
read_only,
now,
upper_tick_interval,
cmd_rx: rx,
shutdown_rx,
desired: Vec::new(),
to_write: Vec::new(),
current_upper,
};
let handle = mz_ore::task::spawn(
|| format!("CollectionManager-differential_write_task-{id}"),
async move {
if !task.read_only {
task.prepare(introspection_config).await;
}
let res = task.run().await;
match res {
ControlFlow::Break(reason) => {
info!("write_task-{} ending: {}", id, reason);
}
c => {
unreachable!(
"cannot break out of the loop with a Continue, but got: {:?}",
c
);
}
}
},
);
(tx, handle.abort_on_drop(), shutdown_tx)
}
/// Does any work that is required before this background task starts
/// writing to the given introspection collection.
///
/// This might include consolidation, deleting older entries or seeding
/// in-memory state of, say, scrapers, with current collection contents.
async fn prepare(&self, introspection_config: DifferentialIntrospectionConfig<T>) {
tracing::info!(%self.id, ?introspection_config.introspection_type, "preparing differential introspection collection for writes");
match introspection_config.introspection_type {
IntrospectionType::ShardMapping => {
// Done by the `append_shard_mappings` call.
}
IntrospectionType::Frontiers | IntrospectionType::ReplicaFrontiers => {
// Differential collections start with an empty
// desired state. No need to manually reset.
}
IntrospectionType::StorageSourceStatistics => {
let prev = snapshot_statistics(
self.id,
introspection_config.recent_upper,
&introspection_config.storage_collections,
&introspection_config.txns_read,
&introspection_config.persist,
)
.await;
let scraper_token = statistics::spawn_statistics_scraper::<
statistics::SourceStatistics,
SourceStatisticsUpdate,
_,
>(
self.id.clone(),
// These do a shallow copy.
introspection_config.collection_manager,
Arc::clone(&introspection_config.source_statistics),
prev,
introspection_config.statistics_interval.clone(),
introspection_config.statistics_interval_receiver.clone(),
introspection_config.metrics,
);
let web_token = statistics::spawn_webhook_statistics_scraper(
introspection_config.source_statistics,
introspection_config.statistics_interval,
introspection_config.statistics_interval_receiver,
);
// Make sure these are dropped when the controller is
// dropped, so that the internal task will stop.
introspection_config
.introspection_tokens
.lock()
.expect("poisoned")
.insert(self.id, Box::new((scraper_token, web_token)));
}
IntrospectionType::StorageSinkStatistics => {
let prev = snapshot_statistics(
self.id,
introspection_config.recent_upper,
&introspection_config.storage_collections,
&introspection_config.txns_read,
&introspection_config.persist,
)
.await;
let scraper_token =
statistics::spawn_statistics_scraper::<_, SinkStatisticsUpdate, _>(
self.id.clone(),
introspection_config.collection_manager,
Arc::clone(&introspection_config.sink_statistics),
prev,
introspection_config.statistics_interval,
introspection_config.statistics_interval_receiver,
introspection_config.metrics,
);
// Make sure this is dropped when the controller is
// dropped, so that the internal task will stop.
introspection_config
.introspection_tokens
.lock()
.expect("poisoned")
.insert(self.id, scraper_token);
}
IntrospectionType::ComputeDependencies
| IntrospectionType::ComputeOperatorHydrationStatus
| IntrospectionType::ComputeMaterializedViewRefreshes
| IntrospectionType::ComputeErrorCounts
| IntrospectionType::ComputeHydrationTimes => {
// Differential collections start with an empty
// desired state. No need to manually reset.
}
introspection_type @ IntrospectionType::ReplicaMetricsHistory
| introspection_type @ IntrospectionType::WallclockLagHistory
| introspection_type @ IntrospectionType::PreparedStatementHistory
| introspection_type @ IntrospectionType::StatementExecutionHistory
| introspection_type @ IntrospectionType::SessionHistory
| introspection_type @ IntrospectionType::StatementLifecycleHistory
| introspection_type @ IntrospectionType::SqlText
| introspection_type @ IntrospectionType::SourceStatusHistory
| introspection_type @ IntrospectionType::SinkStatusHistory
| introspection_type @ IntrospectionType::PrivatelinkConnectionStatusHistory
| introspection_type @ IntrospectionType::ReplicaStatusHistory => {
unreachable!("not differential collection: {introspection_type:?}")
}
}
}
async fn run(mut self) -> ControlFlow<String> {
const BATCH_SIZE: usize = 4096;
let mut updates = Vec::with_capacity(BATCH_SIZE);
loop {
tokio::select! {
// Prefer sending actual updates over just bumping the upper,
// because sending updates also bump the upper.
biased;
// Listen for a shutdown signal so we can gracefully cleanup.
_ = &mut self.shutdown_rx => {
self.handle_shutdown();
return ControlFlow::Break("graceful shutdown".to_string());
}
// Pull a chunk of queued updates off the channel.
count = self.cmd_rx.recv_many(&mut updates, BATCH_SIZE) => {
if count > 0 {
let _ = self.handle_updates(&mut updates).await?;
} else {
// Sender has been dropped, which means the collection
// should have been unregistered, break out of the run
// loop if we weren't already aborted.
return ControlFlow::Break("sender has been dropped".to_string());
}
}
// If we haven't received any updates, then we'll move the upper forward.
_ = self.upper_tick_interval.tick() => {
if self.read_only {
// Not bumping uppers while in read-only mode.
continue;
}
let _ = self.tick_upper().await?;
},
}
}
}
async fn tick_upper(&mut self) -> ControlFlow<String> {
let now = T::from((self.now)());
if now <= self.current_upper {
// Upper is already further along than current wall-clock time, no
// need to bump it.
return ControlFlow::Continue(());
}
assert!(!self.read_only);
let res = self
.write_handle
.compare_and_append_batch(
&mut [],
Antichain::from_elem(self.current_upper.clone()),
Antichain::from_elem(now.clone()),
)
.await
.expect("valid usage");
match res {
// All good!
Ok(()) => {
tracing::debug!(%self.id, "bumped upper of differential collection");
self.current_upper = now;
}
Err(err) => {
// Someone else wrote to the collection or bumped the upper. We
// need to sync to latest persist state and potentially patch up
// our `to_write`, based on what we learn and `desired`.
let actual_upper = if let Some(ts) = err.current.as_option() {
ts.clone()
} else {
return ControlFlow::Break("upper is the empty antichain".to_string());
};
tracing::info!(%self.id, ?actual_upper, expected_upper = ?self.current_upper, "upper mismatch while bumping upper, syncing to persist state");
self.current_upper = actual_upper;
self.sync_to_persist().await;
}
}
ControlFlow::Continue(())
}
fn handle_shutdown(&mut self) {
let mut senders = Vec::new();
// Prevent new messages from being sent.
self.cmd_rx.close();
// Get as many waiting senders as possible.
while let Ok((_batch, sender)) = self.cmd_rx.try_recv() {
senders.push(sender);
}
// Notify them that this collection is closed.
//
// Note: if a task is shutting down, that indicates the source has been
// dropped, at which point the identifier is invalid. Returning this
// error provides a better user experience.
notify_listeners(senders, || Err(StorageError::IdentifierInvalid(self.id)));
}
async fn handle_updates(
&mut self,
batch: &mut Vec<(StorageWriteOp, oneshot::Sender<Result<(), StorageError<T>>>)>,
) -> ControlFlow<String> {
// Put in place _some_ rate limiting.
let batch_duration_ms = STORAGE_MANAGED_COLLECTIONS_BATCH_DURATION_DEFAULT;
let use_batch_now = Instant::now();
let min_time_to_complete = use_batch_now + batch_duration_ms;
tracing::debug!(
?use_batch_now,
?batch_duration_ms,
?min_time_to_complete,
"batch duration",
);
let mut responders = Vec::with_capacity(batch.len());
for (op, tx) in batch.drain(..) {
self.apply_write_op(op);
responders.push(tx);
}
// TODO: Maybe don't do it every time?
consolidation::consolidate(&mut self.desired);
consolidation::consolidate(&mut self.to_write);
// Reset the interval which is used to periodically bump the uppers
// because the uppers will get bumped with the following update.
// This makes it such that we will write at most once every
// `interval`.
//
// For example, let's say our `DEFAULT_TICK` interval is 10, so at
// `t + 10`, `t + 20`, ... we'll bump the uppers. If we receive an
// update at `t + 3` we want to shift this window so we bump the
// uppers at `t + 13`, `t + 23`, ... which resetting the interval
// accomplishes.
self.upper_tick_interval.reset();
self.write_to_persist(responders).await?;
// Wait until our artificial latency has completed.
//
// Note: if writing to persist took longer than `DEFAULT_TICK` this
// await will resolve immediately.
tokio::time::sleep_until(min_time_to_complete).await;
ControlFlow::Continue(())
}
/// Apply the given write operation to the `desired`/`to_write` state.
fn apply_write_op(&mut self, op: StorageWriteOp) {
match op {
StorageWriteOp::Append { updates } => {
self.desired.extend_from_slice(&updates);
self.to_write.extend(updates);
}
StorageWriteOp::Delete { filter } => {
let to_delete = self.desired.drain_filter_swapping(|(row, _)| filter(row));
let retractions = to_delete.map(|(row, diff)| (row, -diff));
self.to_write.extend(retractions);
}
}
}
/// Attempt to write what is currently in [Self::to_write] to persist,
/// retrying and re-syncing to persist when necessary, that is when the
/// upper was not what we expected.
async fn write_to_persist(
&mut self,
responders: Vec<oneshot::Sender<Result<(), StorageError<T>>>>,
) -> ControlFlow<String> {
if self.read_only {
tracing::debug!(%self.id, "not writing to differential collection: read-only");
// Not attempting to write while in read-only mode.
return ControlFlow::Continue(());
}
// We'll try really hard to succeed, but eventually stop.
//
// Note: it's very rare we should ever need to retry, and if we need to
// retry it should only take 1 or 2 attempts. We set `max_tries` to be
// high though because if we hit some edge case we want to try hard to
// commit the data.
let retries = Retry::default()
.initial_backoff(Duration::from_secs(1))
.clamp_backoff(Duration::from_secs(3))
.factor(1.25)
.max_tries(20)
.into_retry_stream();
let mut retries = Box::pin(retries);
loop {
// Append updates to persist!
let now = T::from((self.now)());
let new_upper = std::cmp::max(
now,
TimestampManipulation::step_forward(&self.current_upper),
);
let updates_to_write = self
.to_write
.iter()
.map(|(row, diff)| {
(
(SourceData(Ok(row.clone())), ()),
self.current_upper.clone(),
diff.clone(),
)
})
.collect::<Vec<_>>();
assert!(!self.read_only);
let res = self
.write_handle
.compare_and_append(
updates_to_write,
Antichain::from_elem(self.current_upper.clone()),
Antichain::from_elem(new_upper.clone()),
)
.await
.expect("valid usage");
match res {
// Everything was successful!
Ok(()) => {
// Notify all of our listeners.
notify_listeners(responders, || Ok(()));
self.current_upper = new_upper;
// Very important! This is empty at steady state, while
// desired keeps an in-memory copy of desired state.
self.to_write.clear();
tracing::debug!(%self.id, "appended to differential collection");
// Break out of the retry loop so we can wait for more data.
break;
}
// Failed to write to some collections,
Err(err) => {
// Someone else wrote to the collection. We need to read
// from persist and update to_write based on that and the
// desired state.
let actual_upper = if let Some(ts) = err.current.as_option() {
ts.clone()
} else {
return ControlFlow::Break("upper is the empty antichain".to_string());
};
tracing::info!(%self.id, ?actual_upper, expected_upper = ?self.current_upper, "retrying append for differential collection");
// We've exhausted all of our retries, notify listeners and
// break out of the retry loop so we can wait for more data.
if retries.next().await.is_none() {
let invalid_upper = InvalidUpper {
id: self.id,
current_upper: err.current,
};
notify_listeners(responders, || {
Err(StorageError::InvalidUppers(vec![invalid_upper.clone()]))
});
error!(
"exhausted retries when appending to managed collection {}",
self.id
);
break;
}
self.current_upper = actual_upper;
self.sync_to_persist().await;
debug!("Retrying invalid-uppers error while appending to differential collection {}", self.id);
}
}
}
ControlFlow::Continue(())
}
/// Re-derives [Self::to_write] by looking at [Self::desired] and the
/// current state in persist. We want to insert everything in desired and
/// retract everything in persist. But ideally most of that cancels out in
/// consolidation.
///
/// To be called when a `compare_and_append` failed because the upper didn't
/// match what we expected.
async fn sync_to_persist(&mut self) {
let mut read_handle = (self.read_handle_fn)().await;
let as_of = self
.current_upper
.step_back()
.unwrap_or_else(|| T::minimum());
let as_of = Antichain::from_elem(as_of);
let snapshot = read_handle.snapshot_and_fetch(as_of).await;
let mut negated_oks = match snapshot {
Ok(contents) => {
let mut snapshot = Vec::with_capacity(contents.len());
for ((data, _), _, diff) in contents {
let row = data.expect("invalid protobuf data").0.unwrap();
snapshot.push((row, -diff));
}
snapshot
}
Err(_) => panic!("read before since"),
};
self.to_write.clear();
self.to_write.extend(self.desired.iter().cloned());
self.to_write.append(&mut negated_oks);
consolidation::consolidate(&mut self.to_write);
}
}
pub(crate) struct AppendOnlyIntrospectionConfig<T>
where
T: Lattice + Codec64 + From<EpochMillis> + TimestampManipulation,
{
pub(crate) introspection_type: IntrospectionType,
pub(crate) config_set: Arc<ConfigSet>,
pub(crate) parameters: StorageParameters,
pub(crate) storage_collections: Arc<dyn StorageCollections<Timestamp = T> + Send + Sync>,
pub(crate) txns_read: TxnsRead<T>,
pub(crate) persist: Arc<PersistClientCache>,
}
/// A task that writes to an append only collection and continuously bumps the upper for the specified
/// collection.
///
/// For status history collections, this task can deduplicate redundant [`Statuses`](Status).
struct AppendOnlyWriteTask<T>
where
T: Lattice + Codec64 + From<EpochMillis> + TimestampManipulation,
{
/// The collection that we are writing to.
id: GlobalId,
write_handle: WriteHandle<SourceData, (), T, Diff>,
read_only: bool,
now: NowFn,
user_batch_duration_ms: Arc<AtomicU64>,
/// Receiver for write commands.
rx: mpsc::UnboundedReceiver<(
Vec<AppendOnlyUpdate>,
oneshot::Sender<Result<(), StorageError<T>>>,
)>,
/// We have to shut down when receiving from this.
shutdown_rx: oneshot::Receiver<()>,
/// If this collection deduplicates statuses, this map is used to track the previous status.
previous_statuses: Option<BTreeMap<GlobalId, Status>>,
}
impl<T> AppendOnlyWriteTask<T>
where
T: Lattice + Codec64 + From<EpochMillis> + TimestampManipulation,
{
/// Spawns an [`AppendOnlyWriteTask`] in an [`mz_ore::task`] that will continuously bump the
/// upper for the specified collection,
/// and append data that is sent via the provided [`mpsc::UnboundedSender`].
///
/// TODO(parkmycar): One day if we want to customize the tick interval for each collection, that
/// should be done here.
/// TODO(parkmycar): Maybe add prometheus metrics for each collection?
fn spawn(
id: GlobalId,
write_handle: WriteHandle<SourceData, (), T, Diff>,
read_only: bool,
now: NowFn,
user_batch_duration_ms: Arc<AtomicU64>,
introspection_config: Option<AppendOnlyIntrospectionConfig<T>>,
) -> (AppendOnlyWriteChannel<T>, WriteTask, ShutdownSender) {
let (tx, rx) = mpsc::unbounded_channel();
let (shutdown_tx, shutdown_rx) = oneshot::channel();
let previous_statuses: Option<BTreeMap<GlobalId, Status>> = match introspection_config
.as_ref()
.map(|config| config.introspection_type)
{
Some(IntrospectionType::SourceStatusHistory)
| Some(IntrospectionType::SinkStatusHistory) => Some(BTreeMap::new()),
Some(IntrospectionType::ReplicaMetricsHistory)
| Some(IntrospectionType::WallclockLagHistory)
| Some(IntrospectionType::PrivatelinkConnectionStatusHistory)
| Some(IntrospectionType::ReplicaStatusHistory)
| Some(IntrospectionType::PreparedStatementHistory)
| Some(IntrospectionType::StatementExecutionHistory)
| Some(IntrospectionType::SessionHistory)
| Some(IntrospectionType::StatementLifecycleHistory)
| Some(IntrospectionType::SqlText)
| None => None,
Some(introspection_type @ IntrospectionType::ShardMapping)
| Some(introspection_type @ IntrospectionType::Frontiers)
| Some(introspection_type @ IntrospectionType::ReplicaFrontiers)
| Some(introspection_type @ IntrospectionType::StorageSourceStatistics)
| Some(introspection_type @ IntrospectionType::StorageSinkStatistics)
| Some(introspection_type @ IntrospectionType::ComputeDependencies)
| Some(introspection_type @ IntrospectionType::ComputeOperatorHydrationStatus)
| Some(introspection_type @ IntrospectionType::ComputeMaterializedViewRefreshes)
| Some(introspection_type @ IntrospectionType::ComputeErrorCounts)
| Some(introspection_type @ IntrospectionType::ComputeHydrationTimes) => {
unreachable!("not append-only collection: {introspection_type:?}")
}
};
let mut task = Self {
id,
write_handle,
rx,
shutdown_rx,
read_only,
now,
user_batch_duration_ms,
previous_statuses,
};
let handle = mz_ore::task::spawn(
|| format!("CollectionManager-append_only_write_task-{id}"),
async move {
if !task.read_only {
task.prepare(introspection_config).await;
}
task.run().await;
},
);
(tx, handle.abort_on_drop(), shutdown_tx)
}
/// Does any work that is required before the background task starts
/// writing to the given append only introspection collection.
///
/// This might include consolidation or deleting older entries.
async fn prepare(&mut self, introspection_config: Option<AppendOnlyIntrospectionConfig<T>>) {
let Some(AppendOnlyIntrospectionConfig {
introspection_type,
config_set,
parameters,
storage_collections,
txns_read,
persist,
}) = introspection_config
else {
return;
};
let initial_statuses = match introspection_type {
IntrospectionType::ReplicaMetricsHistory | IntrospectionType::WallclockLagHistory => {
let result = partially_truncate_metrics_history(
self.id,
introspection_type,
&mut self.write_handle,
config_set,
self.now.clone(),
storage_collections,
txns_read,
persist,
)
.await;
if let Err(error) = result {
soft_panic_or_log!(
"error truncating metrics history: {error} (type={introspection_type:?})"
);
}
Vec::new()
}
IntrospectionType::PrivatelinkConnectionStatusHistory => {
partially_truncate_status_history(
self.id,
IntrospectionType::PrivatelinkConnectionStatusHistory,
&mut self.write_handle,
privatelink_status_history_desc(¶meters),
self.now.clone(),
&storage_collections,
&txns_read,
&persist,
)
.await;
Vec::new()
}
IntrospectionType::ReplicaStatusHistory => {
partially_truncate_status_history(
self.id,
IntrospectionType::ReplicaStatusHistory,
&mut self.write_handle,
replica_status_history_desc(¶meters),
self.now.clone(),
&storage_collections,
&txns_read,
&persist,
)
.await;
Vec::new()
}
// Note [btv] - we don't truncate these, because that uses
// a huge amount of memory on environmentd startup.
IntrospectionType::PreparedStatementHistory
| IntrospectionType::StatementExecutionHistory
| IntrospectionType::SessionHistory
| IntrospectionType::StatementLifecycleHistory
| IntrospectionType::SqlText => {
// NOTE(aljoscha): We never remove from these
// collections. Someone, at some point needs to
// think about that! Issue:
// https://github.com/MaterializeInc/database-issues/issues/7666
Vec::new()
}
IntrospectionType::SourceStatusHistory => {
let last_status_per_id = partially_truncate_status_history(
self.id,
IntrospectionType::SourceStatusHistory,
&mut self.write_handle,
source_status_history_desc(¶meters),
self.now.clone(),
&storage_collections,
&txns_read,
&persist,
)
.await;
let status_col = MZ_SOURCE_STATUS_HISTORY_DESC
.get_by_name(&ColumnName::from("status"))
.expect("schema has not changed")
.0;
last_status_per_id
.into_iter()
.map(|(id, row)| {
(
id,
Status::from_str(
row.iter()
.nth(status_col)
.expect("schema has not changed")
.unwrap_str(),
)
.expect("statuses must be uncorrupted"),
)
})
.collect()
}
IntrospectionType::SinkStatusHistory => {
let last_status_per_id = partially_truncate_status_history(
self.id,
IntrospectionType::SinkStatusHistory,
&mut self.write_handle,
sink_status_history_desc(¶meters),
self.now.clone(),
&storage_collections,
&txns_read,
&persist,
)
.await;
let status_col = MZ_SINK_STATUS_HISTORY_DESC
.get_by_name(&ColumnName::from("status"))
.expect("schema has not changed")
.0;
last_status_per_id
.into_iter()
.map(|(id, row)| {
(
id,
Status::from_str(
row.iter()
.nth(status_col)
.expect("schema has not changed")
.unwrap_str(),
)
.expect("statuses must be uncorrupted"),
)
})
.collect()
}
introspection_type @ IntrospectionType::ShardMapping
| introspection_type @ IntrospectionType::Frontiers
| introspection_type @ IntrospectionType::ReplicaFrontiers
| introspection_type @ IntrospectionType::StorageSourceStatistics
| introspection_type @ IntrospectionType::StorageSinkStatistics
| introspection_type @ IntrospectionType::ComputeDependencies
| introspection_type @ IntrospectionType::ComputeOperatorHydrationStatus
| introspection_type @ IntrospectionType::ComputeMaterializedViewRefreshes
| introspection_type @ IntrospectionType::ComputeErrorCounts
| introspection_type @ IntrospectionType::ComputeHydrationTimes => {
unreachable!("not append-only collection: {introspection_type:?}")
}
};
if let Some(previous_statuses) = &mut self.previous_statuses {
previous_statuses.extend(initial_statuses);
}
}
async fn run(mut self) {
let mut interval = tokio::time::interval(Duration::from_millis(DEFAULT_TICK_MS));
const BATCH_SIZE: usize = 4096;
let mut batch: Vec<(Vec<_>, _)> = Vec::with_capacity(BATCH_SIZE);
'run: loop {
tokio::select! {
// Prefer sending actual updates over just bumping the upper, because sending
// updates also bump the upper.
biased;
// Listen for a shutdown signal so we can gracefully cleanup.
_ = &mut self.shutdown_rx => {
let mut senders = Vec::new();
// Prevent new messages from being sent.
self.rx.close();
// Get as many waiting senders as possible.
while let Ok((_batch, sender)) = self.rx.try_recv() {
senders.push(sender);
}
// Notify them that this collection is closed.
//
// Note: if a task is shutting down, that indicates the source has been
// dropped, at which point the identifier is invalid. Returning this
// error provides a better user experience.
notify_listeners(senders, || Err(StorageError::IdentifierInvalid(self.id)));
break 'run;
}
// Pull a chunk of queued updates off the channel.
count = self.rx.recv_many(&mut batch, BATCH_SIZE) => {
if count > 0 {
// To rate limit appends to persist we add artificial latency, and will
// finish no sooner than this instant.
let batch_duration_ms = match self.id {
GlobalId::User(_) => Duration::from_millis(self.user_batch_duration_ms.load(Ordering::Relaxed)),
// For non-user collections, always just use the default.
_ => STORAGE_MANAGED_COLLECTIONS_BATCH_DURATION_DEFAULT,
};
let use_batch_now = Instant::now();
let min_time_to_complete = use_batch_now + batch_duration_ms;
tracing::debug!(
?use_batch_now,
?batch_duration_ms,
?min_time_to_complete,
"batch duration",
);
// Reset the interval which is used to periodically bump the uppers
// because the uppers will get bumped with the following update. This
// makes it such that we will write at most once every `interval`.
//
// For example, let's say our `DEFAULT_TICK` interval is 10, so at
// `t + 10`, `t + 20`, ... we'll bump the uppers. If we receive an
// update at `t + 3` we want to shift this window so we bump the uppers
// at `t + 13`, `t + 23`, ... which resetting the interval accomplishes.
interval.reset();
let mut all_rows = Vec::with_capacity(batch.iter().map(|(rows, _)| rows.len()).sum());
let mut responders = Vec::with_capacity(batch.len());
for (updates, responder) in batch.drain(..) {
let rows = self.process_updates(updates);
all_rows.extend(rows.map(|(row, diff)| TimestamplessUpdate { row, diff}));
responders.push(responder);
}
if self.read_only {
tracing::warn!(%self.id, ?all_rows, "append while in read-only mode");
notify_listeners(responders, || Err(StorageError::ReadOnly));
continue;
}
// Append updates to persist!
let at_least = T::from((self.now)());
if !all_rows.is_empty() {
monotonic_append(&mut self.write_handle, all_rows, at_least).await;
}
// Notify all of our listeners.
notify_listeners(responders, || Ok(()));
// Wait until our artificial latency has completed.
//
// Note: if writing to persist took longer than `DEFAULT_TICK` this
// await will resolve immediately.
tokio::time::sleep_until(min_time_to_complete).await;
} else {
// Sender has been dropped, which means the collection should have been
// unregistered, break out of the run loop if we weren't already
// aborted.
break 'run;
}
}
// If we haven't received any updates, then we'll move the upper forward.
_ = interval.tick() => {
if self.read_only {
// Not bumping uppers while in read-only mode.
continue;
}
// Update our collection.
let now = T::from((self.now)());
let updates = vec![];
let at_least = now.clone();
// Failures don't matter when advancing collections' uppers. This might
// fail when a clusterd happens to be writing to this concurrently.
// Advancing uppers here is best-effort and only needs to succeed if no
// one else is advancing it; contention proves otherwise.
monotonic_append(&mut self.write_handle, updates, at_least).await;
},
}
}
info!("write_task-{} ending", self.id);
}
/// Deduplicate any [`mz_storage_client::client::StatusUpdate`] within `updates` and converts
/// `updates` to rows and diffs.
fn process_updates(
&mut self,
updates: Vec<AppendOnlyUpdate>,
) -> impl Iterator<Item = (Row, Diff)> {
let updates = if let Some(previous_statuses) = &mut self.previous_statuses {
let new: Vec<_> = updates
.into_iter()
.filter(|r| match r {
AppendOnlyUpdate::Row(_) => true,
AppendOnlyUpdate::Status(update) => {
match (previous_statuses.get(&update.id).as_deref(), &update.status) {
(None, _) => true,
(Some(old), new) => old.superseded_by(*new),
}
}
})
.collect();
previous_statuses.extend(new.iter().filter_map(|update| match update {
AppendOnlyUpdate::Row(_) => None,
AppendOnlyUpdate::Status(update) => Some((update.id, update.status)),
}));
new
} else {
updates
};
updates.into_iter().map(AppendOnlyUpdate::into_row)
}
}
/// Truncates the given metrics history by removing all entries older than that history's
/// configured retention interval.
///
/// # Panics
///
/// Panics if `collection` is not a metrics history.
async fn partially_truncate_metrics_history<T>(
id: GlobalId,
introspection_type: IntrospectionType,
write_handle: &mut WriteHandle<SourceData, (), T, Diff>,
config_set: Arc<ConfigSet>,
now: NowFn,
storage_collections: Arc<dyn StorageCollections<Timestamp = T> + Send + Sync>,
txns_read: TxnsRead<T>,
persist: Arc<PersistClientCache>,
) -> Result<(), anyhow::Error>
where
T: Codec64 + From<EpochMillis> + TimestampManipulation,
{
let (keep_duration, occurred_at_col) = match introspection_type {
IntrospectionType::ReplicaMetricsHistory => (
REPLICA_METRICS_HISTORY_RETENTION_INTERVAL.get(&config_set),
REPLICA_METRICS_HISTORY_DESC
.get_by_name(&ColumnName::from("occurred_at"))
.expect("schema has not changed")
.0,
),
IntrospectionType::WallclockLagHistory => (
WALLCLOCK_LAG_HISTORY_RETENTION_INTERVAL.get(&config_set),
WALLCLOCK_LAG_HISTORY_DESC
.get_by_name(&ColumnName::from("occurred_at"))
.expect("schema has not changed")
.0,
),
_ => panic!("not a metrics history: {introspection_type:?}"),
};
let upper = write_handle.fetch_recent_upper().await;
let Some(upper_ts) = upper.as_option() else {
bail!("collection is sealed");
};
let Some(as_of_ts) = upper_ts.step_back() else {
return Ok(()); // nothing to truncate
};
let mut rows = snapshot(id, as_of_ts, &storage_collections, &txns_read, &persist)
.await
.map_err(|e| anyhow!("reading snapshot: {e:?}"))?;
let now = mz_ore::now::to_datetime(now());
let keep_since = now - keep_duration;
// Produce retractions by inverting diffs of rows we want to delete and setting the diffs
// of all other rows to 0.
for (row, diff) in &mut rows {
let datums = row.unpack();
let occurred_at = datums[occurred_at_col].unwrap_timestamptz();
*diff = if *occurred_at < keep_since { -*diff } else { 0 };
}
// Consolidate to avoid superfluous writes.
consolidation::consolidate(&mut rows);
if rows.is_empty() {
return Ok(());
}
// It is very important that we append our retractions at the timestamp
// right after the timestamp at which we got our snapshot. Otherwise,
// it's possible for someone else to sneak in retractions or other
// unexpected changes.
let old_upper_ts = upper_ts.clone();
let write_ts = old_upper_ts.clone();
let new_upper_ts = TimestampManipulation::step_forward(&old_upper_ts);
let updates = rows
.into_iter()
.map(|(row, diff)| ((SourceData(Ok(row)), ()), write_ts.clone(), diff));
write_handle
.compare_and_append(
updates,
Antichain::from_elem(old_upper_ts),
Antichain::from_elem(new_upper_ts),
)
.await
.expect("valid usage")
.map_err(|e| anyhow!("appending retractions: {e:?}"))
}
/// Effectively truncates the status history shard based on its retention policy.
///
/// NOTE: The history collections are really append-only collections, but
/// every-now-and-then we want to retract old updates so that the collection
/// does not grow unboundedly. Crucially, these are _not_ incremental
/// collections, they are not derived from a state at some time `t` and we
/// cannot maintain a desired state for them.
///
/// Returns a map with latest unpacked row per key.
pub(crate) async fn partially_truncate_status_history<T, K>(
id: GlobalId,
introspection_type: IntrospectionType,
write_handle: &mut WriteHandle<SourceData, (), T, Diff>,
status_history_desc: StatusHistoryDesc<K>,
now: NowFn,
storage_collections: &Arc<dyn StorageCollections<Timestamp = T> + Send + Sync>,
txns_read: &TxnsRead<T>,
persist: &Arc<PersistClientCache>,
) -> BTreeMap<K, Row>
where
T: Codec64 + From<EpochMillis> + TimestampManipulation,
K: Clone + Debug + Ord + Send + Sync,
{
let upper = write_handle.fetch_recent_upper().await.clone();
let mut rows = match upper.as_option() {
Some(f) if f > &T::minimum() => {
let as_of = f.step_back().unwrap();
snapshot(id, as_of, storage_collections, txns_read, persist)
.await
.expect("snapshot succeeds")
}
// If collection is closed or the frontier is the minimum, we cannot
// or don't need to truncate (respectively).
_ => return BTreeMap::new(),
};
// BTreeMap to keep track of the row with the latest timestamp for each key.
let mut latest_row_per_key: BTreeMap<K, (CheckedTimestamp<DateTime<Utc>>, Row)> =
BTreeMap::new();
// Consolidate the snapshot, so we can process it correctly below.
differential_dataflow::consolidation::consolidate(&mut rows);
let mut deletions = vec![];
let mut handle_row = {
let latest_row_per_key = &mut latest_row_per_key;
move |row: &Row, diff| {
let datums = row.unpack();
let key = (status_history_desc.extract_key)(&datums);
let timestamp = (status_history_desc.extract_time)(&datums);
assert!(
diff > 0,
"only know how to operate over consolidated data with diffs > 0, \
found diff {diff} for object {key:?} in {introspection_type:?}",
);
// Keep track of the timestamp of the latest row per key.
match latest_row_per_key.get(&key) {
Some(existing) if &existing.0 > ×tamp => {}
_ => {
latest_row_per_key.insert(key.clone(), (timestamp, row.clone()));
}
};
(key, timestamp)
}
};
match status_history_desc.retention_policy {
StatusHistoryRetentionPolicy::LastN(n) => {
// BTreeMap to track the earliest events for each key.
let mut last_n_entries_per_key: BTreeMap<
K,
BinaryHeap<Reverse<(CheckedTimestamp<DateTime<Utc>>, Row)>>,
> = BTreeMap::new();
for (row, diff) in rows {
let (key, timestamp) = handle_row(&row, diff);
// Duplicate rows ARE possible if many status changes happen in VERY quick succession,
// so we handle duplicated rows separately.
let entries = last_n_entries_per_key.entry(key).or_default();
for _ in 0..diff {
// We CAN have multiple statuses (most likely Starting and Running) at the exact same
// millisecond, depending on how the `health_operator` is scheduled.
//
// Note that these will be arbitrarily ordered, so a Starting event might
// survive and a Running one won't. The next restart will remove the other,
// so we don't bother being careful about it.
//
// TODO(guswynn): unpack these into health-status objects and use
// their `Ord` impl.
entries.push(Reverse((timestamp, row.clone())));
// Retain some number of entries, using pop to mark the oldest entries for
// deletion.
while entries.len() > n {
if let Some(Reverse((_, r))) = entries.pop() {
deletions.push(r);
}
}
}
}
}
StatusHistoryRetentionPolicy::TimeWindow(time_window) => {
// Get the lower bound of our retention window
let now = mz_ore::now::to_datetime(now());
let keep_since = now - time_window;
// Mark any row outside the retention window for deletion
for (row, diff) in rows {
let (_, timestamp) = handle_row(&row, diff);
if *timestamp < keep_since {
deletions.push(row);
}
}
}
}
// It is very important that we append our retractions at the timestamp
// right after the timestamp at which we got our snapshot. Otherwise,
// it's possible for someone else to sneak in retractions or other
// unexpected changes.
let expected_upper = upper.into_option().expect("checked above");
let new_upper = TimestampManipulation::step_forward(&expected_upper);
// Updates are only deletes because everything else is already in the shard.
let updates = deletions
.into_iter()
.map(|row| ((SourceData(Ok(row)), ()), expected_upper.clone(), -1))
.collect::<Vec<_>>();
let res = write_handle
.compare_and_append(
updates,
Antichain::from_elem(expected_upper.clone()),
Antichain::from_elem(new_upper),
)
.await
.expect("usage was valid");
match res {
Ok(_) => {
// All good, yay!
}
Err(err) => {
// This is fine, it just means the upper moved because
// of continual upper advancement or because someone
// already appended some more retractions/updates.
//
// NOTE: We might want to attempt these partial
// retractions on an interval, instead of only when
// starting up!
info!(
%id, ?expected_upper, current_upper = ?err.current,
"failed to append partial truncation",
);
}
}
latest_row_per_key
.into_iter()
.map(|(key, (_, row))| (key, row))
.collect()
}
async fn monotonic_append<T: Timestamp + Lattice + Codec64 + TimestampManipulation>(
write_handle: &mut WriteHandle<SourceData, (), T, Diff>,
updates: Vec<TimestamplessUpdate>,
at_least: T,
) {
let mut expected_upper = write_handle.shared_upper();
loop {
if updates.is_empty() && expected_upper.is_empty() {
// Ignore timestamp advancement for
// closed collections. TODO? Make this a
// correctable error
return;
}
let upper = expected_upper
.into_option()
.expect("cannot append data to closed collection");
let lower = if upper.less_than(&at_least) {
at_least.clone()
} else {
upper.clone()
};
let new_upper = TimestampManipulation::step_forward(&lower);
let updates = updates
.iter()
.map(|TimestamplessUpdate { row, diff }| {
((SourceData(Ok(row.clone())), ()), lower.clone(), diff)
})
.collect::<Vec<_>>();
let res = write_handle
.compare_and_append(
updates,
Antichain::from_elem(upper),
Antichain::from_elem(new_upper),
)
.await
.expect("valid usage");
match res {
Ok(()) => return,
Err(err) => {
expected_upper = err.current;
continue;
}
}
}
}
// Helper method for notifying listeners.
fn notify_listeners<T>(
responders: impl IntoIterator<Item = oneshot::Sender<T>>,
result: impl Fn() -> T,
) {
for r in responders {
// We don't care if the listener disappeared.
let _ = r.send(result());
}
}
#[cfg(test)]
mod tests {
use std::collections::BTreeSet;
use super::*;
use mz_repr::{Datum, Row};
use mz_storage_client::client::StatusUpdate;
use mz_storage_client::healthcheck::{
MZ_SINK_STATUS_HISTORY_DESC, MZ_SOURCE_STATUS_HISTORY_DESC,
};
#[mz_ore::test]
fn test_row() {
let error_message = "error message";
let hint = "hint message";
let id = GlobalId::User(1);
let status = Status::Dropped;
let row = Row::from(StatusUpdate {
id,
timestamp: chrono::offset::Utc::now(),
status,
error: Some(error_message.to_string()),
hints: BTreeSet::from([hint.to_string()]),
namespaced_errors: Default::default(),
});
for (datum, column_type) in row.iter().zip(MZ_SINK_STATUS_HISTORY_DESC.iter_types()) {
assert!(datum.is_instance_of(column_type));
}
for (datum, column_type) in row.iter().zip(MZ_SOURCE_STATUS_HISTORY_DESC.iter_types()) {
assert!(datum.is_instance_of(column_type));
}
assert_eq!(row.iter().nth(1).unwrap(), Datum::String(&id.to_string()));
assert_eq!(row.iter().nth(2).unwrap(), Datum::String(status.to_str()));
assert_eq!(row.iter().nth(3).unwrap(), Datum::String(error_message));
let details = row
.iter()
.nth(4)
.unwrap()
.unwrap_map()
.iter()
.collect::<Vec<_>>();
assert_eq!(details.len(), 1);
let hint_datum = &details[0];
assert_eq!(hint_datum.0, "hints");
assert_eq!(
hint_datum.1.unwrap_list().iter().next().unwrap(),
Datum::String(hint)
);
}
#[mz_ore::test]
fn test_row_without_hint() {
let error_message = "error message";
let id = GlobalId::User(1);
let status = Status::Dropped;
let row = Row::from(StatusUpdate {
id,
timestamp: chrono::offset::Utc::now(),
status,
error: Some(error_message.to_string()),
hints: Default::default(),
namespaced_errors: Default::default(),
});
for (datum, column_type) in row.iter().zip(MZ_SINK_STATUS_HISTORY_DESC.iter_types()) {
assert!(datum.is_instance_of(column_type));
}
for (datum, column_type) in row.iter().zip(MZ_SOURCE_STATUS_HISTORY_DESC.iter_types()) {
assert!(datum.is_instance_of(column_type));
}
assert_eq!(row.iter().nth(1).unwrap(), Datum::String(&id.to_string()));
assert_eq!(row.iter().nth(2).unwrap(), Datum::String(status.to_str()));
assert_eq!(row.iter().nth(3).unwrap(), Datum::String(error_message));
assert_eq!(row.iter().nth(4).unwrap(), Datum::Null);
}
#[mz_ore::test]
fn test_row_without_error() {
let id = GlobalId::User(1);
let status = Status::Dropped;
let hint = "hint message";
let row = Row::from(StatusUpdate {
id,
timestamp: chrono::offset::Utc::now(),
status,
error: None,
hints: BTreeSet::from([hint.to_string()]),
namespaced_errors: Default::default(),
});
for (datum, column_type) in row.iter().zip(MZ_SINK_STATUS_HISTORY_DESC.iter_types()) {
assert!(datum.is_instance_of(column_type));
}
for (datum, column_type) in row.iter().zip(MZ_SOURCE_STATUS_HISTORY_DESC.iter_types()) {
assert!(datum.is_instance_of(column_type));
}
assert_eq!(row.iter().nth(1).unwrap(), Datum::String(&id.to_string()));
assert_eq!(row.iter().nth(2).unwrap(), Datum::String(status.to_str()));
assert_eq!(row.iter().nth(3).unwrap(), Datum::Null);
let details = row
.iter()
.nth(4)
.unwrap()
.unwrap_map()
.iter()
.collect::<Vec<_>>();
assert_eq!(details.len(), 1);
let hint_datum = &details[0];
assert_eq!(hint_datum.0, "hints");
assert_eq!(
hint_datum.1.unwrap_list().iter().next().unwrap(),
Datum::String(hint)
);
}
#[mz_ore::test]
fn test_row_with_namespaced() {
let error_message = "error message";
let id = GlobalId::User(1);
let status = Status::Dropped;
let row = Row::from(StatusUpdate {
id,
timestamp: chrono::offset::Utc::now(),
status,
error: Some(error_message.to_string()),
hints: Default::default(),
namespaced_errors: BTreeMap::from([("thing".to_string(), "error".to_string())]),
});
for (datum, column_type) in row.iter().zip(MZ_SINK_STATUS_HISTORY_DESC.iter_types()) {
assert!(datum.is_instance_of(column_type));
}
for (datum, column_type) in row.iter().zip(MZ_SOURCE_STATUS_HISTORY_DESC.iter_types()) {
assert!(datum.is_instance_of(column_type));
}
assert_eq!(row.iter().nth(1).unwrap(), Datum::String(&id.to_string()));
assert_eq!(row.iter().nth(2).unwrap(), Datum::String(status.to_str()));
assert_eq!(row.iter().nth(3).unwrap(), Datum::String(error_message));
let details = row
.iter()
.nth(4)
.unwrap()
.unwrap_map()
.iter()
.collect::<Vec<_>>();
assert_eq!(details.len(), 1);
let ns_datum = &details[0];
assert_eq!(ns_datum.0, "namespaced");
assert_eq!(
ns_datum.1.unwrap_map().iter().next().unwrap(),
("thing", Datum::String("error"))
);
}
#[mz_ore::test]
fn test_row_with_everything() {
let error_message = "error message";
let hint = "hint message";
let id = GlobalId::User(1);
let status = Status::Dropped;
let row = Row::from(StatusUpdate {
id,
timestamp: chrono::offset::Utc::now(),
status,
error: Some(error_message.to_string()),
hints: BTreeSet::from([hint.to_string()]),
namespaced_errors: BTreeMap::from([("thing".to_string(), "error".to_string())]),
});
for (datum, column_type) in row.iter().zip(MZ_SINK_STATUS_HISTORY_DESC.iter_types()) {
assert!(datum.is_instance_of(column_type));
}
for (datum, column_type) in row.iter().zip(MZ_SOURCE_STATUS_HISTORY_DESC.iter_types()) {
assert!(datum.is_instance_of(column_type));
}
assert_eq!(row.iter().nth(1).unwrap(), Datum::String(&id.to_string()));
assert_eq!(row.iter().nth(2).unwrap(), Datum::String(status.to_str()));
assert_eq!(row.iter().nth(3).unwrap(), Datum::String(error_message));
let details = row
.iter()
.nth(4)
.unwrap()
.unwrap_map()
.iter()
.collect::<Vec<_>>();
assert_eq!(details.len(), 2);
// These are always sorted
let hint_datum = &details[0];
let ns_datum = &details[1];
assert_eq!(hint_datum.0, "hints");
assert_eq!(
hint_datum.1.unwrap_list().iter().next().unwrap(),
Datum::String(hint)
);
assert_eq!(ns_datum.0, "namespaced");
assert_eq!(
ns_datum.1.unwrap_map().iter().next().unwrap(),
("thing", Datum::String("error"))
);
}
}