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
// 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.
// Tonic generates code that violates clippy lints.
// TODO: Remove this once tonic does not produce this code anymore.
#![allow(clippy::as_conversions, clippy::clone_on_ref_ptr)]
//! Compute layer client and server.
use std::collections::BTreeMap;
use async_trait::async_trait;
use bytesize::ByteSize;
use differential_dataflow::consolidation::consolidate_updates;
use differential_dataflow::lattice::Lattice;
use mz_expr::row::RowCollection;
use mz_ore::assert_none;
use mz_ore::cast::CastFrom;
use mz_repr::{Diff, GlobalId, Row};
use mz_service::client::{GenericClient, Partitionable, PartitionedState};
use mz_service::grpc::{GrpcClient, GrpcServer, ProtoServiceTypes, ResponseStream};
use timely::progress::frontier::{Antichain, MutableAntichain};
use timely::PartialOrder;
use tonic::{Request, Status, Streaming};
use uuid::Uuid;
use crate::controller::ComputeControllerTimestamp;
use crate::metrics::ReplicaMetrics;
use crate::protocol::command::{ComputeCommand, ProtoComputeCommand};
use crate::protocol::response::{
ComputeResponse, CopyToResponse, FrontiersResponse, PeekResponse, ProtoComputeResponse,
SubscribeBatch, SubscribeResponse,
};
use crate::service::proto_compute_server::ProtoCompute;
include!(concat!(env!("OUT_DIR"), "/mz_compute_client.service.rs"));
/// A client to a compute server.
pub trait ComputeClient<T = mz_repr::Timestamp>:
GenericClient<ComputeCommand<T>, ComputeResponse<T>>
{
}
impl<C, T> ComputeClient<T> for C where C: GenericClient<ComputeCommand<T>, ComputeResponse<T>> {}
#[async_trait]
impl<T: Send> GenericClient<ComputeCommand<T>, ComputeResponse<T>> for Box<dyn ComputeClient<T>> {
async fn send(&mut self, cmd: ComputeCommand<T>) -> Result<(), anyhow::Error> {
(**self).send(cmd).await
}
/// # Cancel safety
///
/// This method is cancel safe. If `recv` is used as the event in a [`tokio::select!`]
/// statement and some other branch completes first, it is guaranteed that no messages were
/// received by this client.
async fn recv(&mut self) -> Result<Option<ComputeResponse<T>>, anyhow::Error> {
// `GenericClient::recv` is required to be cancel safe.
(**self).recv().await
}
}
/// TODO(database-issues#7533): Add documentation.
#[derive(Debug, Clone)]
pub enum ComputeProtoServiceTypes {}
impl ProtoServiceTypes for ComputeProtoServiceTypes {
type PC = ProtoComputeCommand;
type PR = ProtoComputeResponse;
type STATS = ReplicaMetrics;
const URL: &'static str = "/mz_compute_client.service.ProtoCompute/CommandResponseStream";
}
/// TODO(database-issues#7533): Add documentation.
pub type ComputeGrpcClient = GrpcClient<ComputeProtoServiceTypes>;
#[async_trait]
impl<F, G> ProtoCompute for GrpcServer<F>
where
F: Fn() -> G + Send + Sync + 'static,
G: ComputeClient + 'static,
{
type CommandResponseStreamStream = ResponseStream<ProtoComputeResponse>;
async fn command_response_stream(
&self,
request: Request<Streaming<ProtoComputeCommand>>,
) -> Result<tonic::Response<Self::CommandResponseStreamStream>, Status> {
self.forward_bidi_stream(request).await
}
}
/// Maintained state for partitioned compute clients.
///
/// This helper type unifies the responses of multiple partitioned workers in order to present as a
/// single worker:
///
/// * It emits `Frontiers` responses reporting the minimum/meet of frontiers reported by the
/// individual workers.
/// * It emits `PeekResponse`s and `SubscribeResponse`s reporting the union of the responses
/// received from the workers.
///
/// In the compute communication stack, this client is instantiated several times:
///
/// * One instance on the controller side, dispatching between cluster processes.
/// * One instance in each cluster process, dispatching between timely worker threads.
///
/// Note that because compute commands, except `CreateTimely` and `UpdateConfiguration`, are only
/// sent to the first process, the cluster-side instances of `PartitionedComputeState` are not
/// guaranteed to see all compute commands. Or more specifically: The instance running inside
/// process 0 sees all commands, whereas the instances running inside the other processes only see
/// `CreateTimely` and `UpdateConfiguration`. The `PartitionedComputeState` implementation must be
/// able to cope with this limited visiblity. It does so by performing most of its state management
/// based on observed compute responses rather than commands.
#[derive(Debug)]
pub struct PartitionedComputeState<T> {
/// Number of partitions the state machine represents.
parts: usize,
/// The maximum result size this state machine can return.
///
/// This is updated upon receiving [`ComputeCommand::UpdateConfiguration`]s.
max_result_size: u64,
/// Tracked frontiers for indexes and sinks.
///
/// Frontier tracking for a collection is initialized when the first `Frontiers` response
/// for that collection is received. Frontier tracking is ceased when all shards have reported
/// advancement to the empty frontier for all frontier kinds.
///
/// The compute protocol requires that shards always emit `Frontiers` responses reporting empty
/// frontiers for all frontier kinds when a collection is dropped. It further requires that no
/// further `Frontier` responses are emitted for a collection after the empty frontiers were
/// reported. These properties ensure that a) we always cease frontier tracking for collections
/// that have been dropped and b) frontier tracking for a collection is not re-initialized
/// after it was ceased.
frontiers: BTreeMap<GlobalId, TrackedFrontiers<T>>,
/// Pending responses for a peek; returnable once all are available.
///
/// Tracking of responses for a peek is initialized when the first `PeekResponse` for that peek
/// is received. Once all shards have provided a `PeekResponse`, a unified peek response is
/// emitted and the peek tracking state is dropped again.
///
/// The compute protocol requires that exactly one response is emitted for each peek. This
/// property ensures that a) we can eventually drop the tracking state maintained for a peek
/// and b) we won't re-initialize tracking for a peek we have already served.
peek_responses: BTreeMap<Uuid, BTreeMap<usize, PeekResponse>>,
/// Pending responses for a copy to; returnable once all are available.
///
/// Tracking of responses for a COPY TO is initialized when the first `CopyResponse` for that command
/// is received. Once all shards have provided a `CopyResponse`, a unified copy response is
/// emitted and the copy_to tracking state is dropped again.
///
/// The compute protocol requires that exactly one response is emitted for each COPY TO command. This
/// property ensures that a) we can eventually drop the tracking state maintained for a copy
/// and b) we won't re-initialize tracking for a copy we have already served.
copy_to_responses: BTreeMap<GlobalId, BTreeMap<usize, CopyToResponse>>,
/// Tracks in-progress `SUBSCRIBE`s, and the stashed rows we are holding back until their
/// timestamps are complete.
///
/// The updates may be `Err` if any of the batches have reported an error, in which case the
/// subscribe is permanently borked.
///
/// Tracking of a subscribe is initialized when the first `SubscribeResponse` for that
/// subscribe is received. Once all shards have emitted an "end-of-subscribe" response the
/// subscribe tracking state is dropped again.
///
/// The compute protocol requires that for a subscribe that shuts down an end-of-subscribe
/// response is emitted:
///
/// * Either a `Batch` response reporting advancement to the empty frontier...
/// * ... or a `DroppedAt` response reporting that the subscribe was dropped before
/// completing.
///
/// The compute protocol further requires that no further `SubscribeResponse`s are emitted for
/// a subscribe after an end-of-subscribe was reported.
///
/// These two properties ensure that a) once a subscribe has shut down, we can eventually drop
/// the tracking state maintained for it and b) we won't re-initialize tracking for a subscribe
/// we have already dropped.
pending_subscribes: BTreeMap<GlobalId, PendingSubscribe<T>>,
}
impl<T> Partitionable<ComputeCommand<T>, ComputeResponse<T>>
for (ComputeCommand<T>, ComputeResponse<T>)
where
T: ComputeControllerTimestamp,
{
type PartitionedState = PartitionedComputeState<T>;
fn new(parts: usize) -> PartitionedComputeState<T> {
PartitionedComputeState {
parts,
max_result_size: u64::MAX,
frontiers: BTreeMap::new(),
peek_responses: BTreeMap::new(),
pending_subscribes: BTreeMap::new(),
copy_to_responses: BTreeMap::new(),
}
}
}
impl<T> PartitionedComputeState<T>
where
T: ComputeControllerTimestamp,
{
fn reset(&mut self) {
let PartitionedComputeState {
parts: _,
max_result_size: _,
frontiers,
peek_responses,
pending_subscribes,
copy_to_responses,
} = self;
frontiers.clear();
peek_responses.clear();
pending_subscribes.clear();
copy_to_responses.clear();
}
/// Observes commands that move past, and prepares state for responses.
pub fn observe_command(&mut self, command: &ComputeCommand<T>) {
match command {
ComputeCommand::CreateTimely { .. } => self.reset(),
ComputeCommand::UpdateConfiguration(config) => {
if let Some(max_result_size) = config.max_result_size {
self.max_result_size = max_result_size;
}
}
_ => {
// We are not guaranteed to observe other compute commands. We
// must therefore not add any logic here that relies on doing so.
}
}
}
}
impl<T> PartitionedState<ComputeCommand<T>, ComputeResponse<T>> for PartitionedComputeState<T>
where
T: ComputeControllerTimestamp,
{
fn split_command(&mut self, command: ComputeCommand<T>) -> Vec<Option<ComputeCommand<T>>> {
self.observe_command(&command);
// As specified by the compute protocol:
// * Forward `CreateTimely` and `UpdateConfiguration` commands to all shards.
// * Forward all other commands to the first shard only.
match command {
ComputeCommand::CreateTimely { config, epoch } => {
let timely_cmds = config.split_command(self.parts);
timely_cmds
.into_iter()
.map(|config| Some(ComputeCommand::CreateTimely { config, epoch }))
.collect()
}
command @ ComputeCommand::UpdateConfiguration(_) => {
vec![Some(command); self.parts]
}
command => {
let mut r = vec![None; self.parts];
r[0] = Some(command);
r
}
}
}
fn absorb_response(
&mut self,
shard_id: usize,
message: ComputeResponse<T>,
) -> Option<Result<ComputeResponse<T>, anyhow::Error>> {
match message {
ComputeResponse::Frontiers(id, frontiers) => {
// Initialize frontier tracking state for this collection, if necessary.
let tracked = self
.frontiers
.entry(id)
.or_insert_with(|| TrackedFrontiers::new(self.parts));
let write_frontier = frontiers
.write_frontier
.and_then(|f| tracked.update_write_frontier(shard_id, &f));
let input_frontier = frontiers
.input_frontier
.and_then(|f| tracked.update_input_frontier(shard_id, &f));
let output_frontier = frontiers
.output_frontier
.and_then(|f| tracked.update_output_frontier(shard_id, &f));
let frontiers = FrontiersResponse {
write_frontier,
input_frontier,
output_frontier,
};
let result = frontiers
.has_updates()
.then_some(Ok(ComputeResponse::Frontiers(id, frontiers)));
if tracked.all_empty() {
// All shards have reported advancement to the empty frontier, so we do not
// expect further updates for this collection.
self.frontiers.remove(&id);
}
result
}
ComputeResponse::PeekResponse(uuid, response, otel_ctx) => {
// Incorporate new peek responses; awaiting all responses.
let entry = self
.peek_responses
.entry(uuid)
.or_insert_with(Default::default);
let novel = entry.insert(shard_id, response);
assert_none!(novel, "Duplicate peek response");
// We may be ready to respond.
if entry.len() == self.parts {
let mut response = PeekResponse::Rows(RowCollection::default());
for (_part, r) in std::mem::take(entry).into_iter() {
response = match (response, r) {
(_, PeekResponse::Canceled) => PeekResponse::Canceled,
(PeekResponse::Canceled, _) => PeekResponse::Canceled,
(_, PeekResponse::Error(e)) => PeekResponse::Error(e),
(PeekResponse::Error(e), _) => PeekResponse::Error(e),
(PeekResponse::Rows(mut rows), PeekResponse::Rows(other)) => {
let total_byte_size =
rows.byte_len().saturating_add(other.byte_len());
if total_byte_size > usize::cast_from(self.max_result_size) {
// Note: We match on this specific error message in tests
// so it's important that nothing else returns the same
// string.
let err = format!(
"total result exceeds max size of {}",
ByteSize::b(self.max_result_size)
);
PeekResponse::Error(err)
} else {
rows.merge(&other);
PeekResponse::Rows(rows)
}
}
};
}
self.peek_responses.remove(&uuid);
// We take the otel_ctx from the last peek, but they should all be the same
Some(Ok(ComputeResponse::PeekResponse(uuid, response, otel_ctx)))
} else {
None
}
}
ComputeResponse::SubscribeResponse(id, response) => {
// Initialize tracking for this subscribe, if necessary.
let entry = self
.pending_subscribes
.entry(id)
.or_insert_with(|| PendingSubscribe::new(self.parts));
let emit_response = match response {
SubscribeResponse::Batch(batch) => {
let frontiers = &mut entry.frontiers;
let old_frontier = frontiers.frontier().to_owned();
frontiers.update_iter(batch.lower.into_iter().map(|t| (t, -1)));
frontiers.update_iter(batch.upper.into_iter().map(|t| (t, 1)));
let new_frontier = frontiers.frontier().to_owned();
match (&mut entry.stashed_updates, batch.updates) {
(Err(_), _) => {
// Subscribe is borked; nothing to do.
// TODO: Consider refreshing error?
}
(_, Err(text)) => {
entry.stashed_updates = Err(text);
}
(Ok(stashed_updates), Ok(updates)) => {
stashed_updates.extend(updates);
}
}
// If the frontier has advanced, it is time to announce subscribe progress.
// Unless we have already announced that the subscribe has been dropped, in
// which case we must keep quiet.
if old_frontier != new_frontier && !entry.dropped {
let updates = match &mut entry.stashed_updates {
Ok(stashed_updates) => {
// The compute protocol requires us to only send out
// consolidated batches.
consolidate_updates(stashed_updates);
let mut ship = Vec::new();
let mut keep = Vec::new();
for (time, data, diff) in stashed_updates.drain(..) {
if new_frontier.less_equal(&time) {
keep.push((time, data, diff));
} else {
ship.push((time, data, diff));
}
}
entry.stashed_updates = Ok(keep);
Ok(ship)
}
Err(text) => Err(text.clone()),
};
Some(Ok(ComputeResponse::SubscribeResponse(
id,
SubscribeResponse::Batch(SubscribeBatch {
lower: old_frontier,
upper: new_frontier,
updates,
}),
)))
} else {
None
}
}
SubscribeResponse::DroppedAt(frontier) => {
entry
.frontiers
.update_iter(frontier.iter().map(|t| (t.clone(), -1)));
if entry.dropped {
None
} else {
entry.dropped = true;
Some(Ok(ComputeResponse::SubscribeResponse(
id,
SubscribeResponse::DroppedAt(frontier),
)))
}
}
};
if entry.frontiers.frontier().is_empty() {
// All shards have reported advancement to the empty frontier or dropping, so
// we do not expect further updates for this subscribe.
self.pending_subscribes.remove(&id);
}
emit_response
}
ComputeResponse::CopyToResponse(id, response) => {
// Incorporate new copy to responses; awaiting all responses.
let entry = self
.copy_to_responses
.entry(id)
.or_insert_with(Default::default);
let novel = entry.insert(shard_id, response);
assert_none!(novel, "Duplicate copy to response");
// We may be ready to respond.
if entry.len() == self.parts {
let mut response = CopyToResponse::RowCount(0);
for (_part, r) in std::mem::take(entry).into_iter() {
response = match (response, r) {
// It's important that we receive all the `Dropped` messages as well
// so that the `copy_to_responses` state can be cleared.
(_, CopyToResponse::Dropped) => CopyToResponse::Dropped,
(CopyToResponse::Dropped, _) => CopyToResponse::Dropped,
(_, CopyToResponse::Error(e)) => CopyToResponse::Error(e),
(CopyToResponse::Error(e), _) => CopyToResponse::Error(e),
(CopyToResponse::RowCount(r1), CopyToResponse::RowCount(r2)) => {
CopyToResponse::RowCount(r1 + r2)
}
};
}
self.copy_to_responses.remove(&id);
Some(Ok(ComputeResponse::CopyToResponse(id, response)))
} else {
None
}
}
response @ ComputeResponse::Status(_) => {
// Pass through status responses.
Some(Ok(response))
}
}
}
}
/// Tracked frontiers for an index or a sink collection.
///
/// Each frontier is maintained both as a `MutableAntichain` across all partitions and individually
/// for each partition.
#[derive(Debug)]
struct TrackedFrontiers<T> {
/// The tracked write frontier.
write_frontier: (MutableAntichain<T>, Vec<Antichain<T>>),
/// The tracked input frontier.
input_frontier: (MutableAntichain<T>, Vec<Antichain<T>>),
/// The tracked output frontier.
output_frontier: (MutableAntichain<T>, Vec<Antichain<T>>),
}
impl<T> TrackedFrontiers<T>
where
T: timely::progress::Timestamp + Lattice,
{
/// Initializes frontier tracking state for a new collection.
fn new(parts: usize) -> Self {
// TODO(benesch): fix this dangerous use of `as`.
#[allow(clippy::as_conversions)]
let parts_diff = parts as i64;
let mut frontier = MutableAntichain::new();
frontier.update_iter([(T::minimum(), parts_diff)]);
let part_frontiers = vec![Antichain::from_elem(T::minimum()); parts];
let frontier_entry = (frontier, part_frontiers);
Self {
write_frontier: frontier_entry.clone(),
input_frontier: frontier_entry.clone(),
output_frontier: frontier_entry,
}
}
/// Returns whether all tracked frontiers have advanced to the empty frontier.
fn all_empty(&self) -> bool {
self.write_frontier.0.frontier().is_empty()
&& self.input_frontier.0.frontier().is_empty()
&& self.output_frontier.0.frontier().is_empty()
}
/// Updates write frontier tracking with a new shard frontier.
///
/// If this causes the global write frontier to advance, the advanced frontier is returned.
fn update_write_frontier(
&mut self,
shard_id: usize,
new_shard_frontier: &Antichain<T>,
) -> Option<Antichain<T>> {
Self::update_frontier(&mut self.write_frontier, shard_id, new_shard_frontier)
}
/// Updates input frontier tracking with a new shard frontier.
///
/// If this causes the global input frontier to advance, the advanced frontier is returned.
fn update_input_frontier(
&mut self,
shard_id: usize,
new_shard_frontier: &Antichain<T>,
) -> Option<Antichain<T>> {
Self::update_frontier(&mut self.input_frontier, shard_id, new_shard_frontier)
}
/// Updates output frontier tracking with a new shard frontier.
///
/// If this causes the global output frontier to advance, the advanced frontier is returned.
fn update_output_frontier(
&mut self,
shard_id: usize,
new_shard_frontier: &Antichain<T>,
) -> Option<Antichain<T>> {
Self::update_frontier(&mut self.output_frontier, shard_id, new_shard_frontier)
}
/// Updates the provided frontier entry with a new shard frontier.
fn update_frontier(
entry: &mut (MutableAntichain<T>, Vec<Antichain<T>>),
shard_id: usize,
new_shard_frontier: &Antichain<T>,
) -> Option<Antichain<T>> {
let (frontier, shard_frontiers) = entry;
let old_frontier = frontier.frontier().to_owned();
let shard_frontier = &mut shard_frontiers[shard_id];
frontier.update_iter(shard_frontier.iter().map(|t| (t.clone(), -1)));
shard_frontier.join_assign(new_shard_frontier);
frontier.update_iter(shard_frontier.iter().map(|t| (t.clone(), 1)));
let new_frontier = frontier.frontier();
if PartialOrder::less_than(&old_frontier.borrow(), &new_frontier) {
Some(new_frontier.to_owned())
} else {
None
}
}
}
#[derive(Debug)]
struct PendingSubscribe<T> {
/// The subscribe frontiers of the partitioned shards.
frontiers: MutableAntichain<T>,
/// The updates we are holding back until their timestamps are complete.
stashed_updates: Result<Vec<(T, Row, Diff)>, String>,
/// Whether we have already emitted a `DroppedAt` response for this subscribe.
///
/// This field is used to ensure we emit such a response only once.
dropped: bool,
}
impl<T: ComputeControllerTimestamp> PendingSubscribe<T> {
fn new(parts: usize) -> Self {
let mut frontiers = MutableAntichain::new();
// TODO(benesch): fix this dangerous use of `as`.
#[allow(clippy::as_conversions)]
frontiers.update_iter([(T::minimum(), parts as i64)]);
Self {
frontiers,
stashed_updates: Ok(Vec::new()),
dropped: false,
}
}
}