mz_compute/
server.rs

1// Copyright Materialize, Inc. and contributors. All rights reserved.
2//
3// Use of this software is governed by the Business Source License
4// included in the LICENSE file.
5//
6// As of the Change Date specified in that file, in accordance with
7// the Business Source License, use of this software will be governed
8// by the Apache License, Version 2.0.
9
10//! An interactive dataflow server.
11
12use std::cell::RefCell;
13use std::collections::{BTreeMap, BTreeSet};
14use std::convert::Infallible;
15use std::fmt::Debug;
16use std::path::PathBuf;
17use std::rc::Rc;
18use std::sync::{Arc, Mutex};
19use std::thread;
20use std::time::{Duration, Instant};
21
22use anyhow::Error;
23use crossbeam_channel::SendError;
24use mz_cluster::client::{ClusterClient, ClusterSpec};
25use mz_cluster_client::client::TimelyConfig;
26use mz_compute_client::protocol::command::ComputeCommand;
27use mz_compute_client::protocol::history::ComputeCommandHistory;
28use mz_compute_client::protocol::response::ComputeResponse;
29use mz_compute_client::service::ComputeClient;
30use mz_ore::halt;
31use mz_ore::metrics::MetricsRegistry;
32use mz_ore::tracing::TracingHandle;
33use mz_persist_client::cache::PersistClientCache;
34use mz_storage_types::connections::ConnectionContext;
35use mz_txn_wal::operator::TxnsContext;
36use timely::communication::Allocate;
37use timely::progress::Antichain;
38use timely::worker::Worker as TimelyWorker;
39use tokio::sync::mpsc;
40use tracing::{info, trace, warn};
41use uuid::Uuid;
42
43use crate::command_channel;
44use crate::compute_state::{ActiveComputeState, ComputeState, ReportedFrontier};
45use crate::metrics::{ComputeMetrics, WorkerMetrics};
46
47/// Caller-provided configuration for compute.
48#[derive(Clone, Debug)]
49pub struct ComputeInstanceContext {
50    /// A directory that can be used for scratch work.
51    pub scratch_directory: Option<PathBuf>,
52    /// Whether to set core affinity for Timely workers.
53    pub worker_core_affinity: bool,
54    /// Context required to connect to an external sink from compute,
55    /// like the `CopyToS3OneshotSink` compute sink.
56    pub connection_context: ConnectionContext,
57}
58
59/// Configures the server with compute-specific metrics.
60#[derive(Debug, Clone)]
61struct Config {
62    /// `persist` client cache.
63    pub persist_clients: Arc<PersistClientCache>,
64    /// Context necessary for rendering txn-wal operators.
65    pub txns_ctx: TxnsContext,
66    /// A process-global handle to tracing configuration.
67    pub tracing_handle: Arc<TracingHandle>,
68    /// Metrics exposed by compute replicas.
69    pub metrics: ComputeMetrics,
70    /// Other configuration for compute.
71    pub context: ComputeInstanceContext,
72}
73
74/// Initiates a timely dataflow computation, processing compute commands.
75pub async fn serve(
76    timely_config: TimelyConfig,
77    metrics_registry: &MetricsRegistry,
78    persist_clients: Arc<PersistClientCache>,
79    txns_ctx: TxnsContext,
80    tracing_handle: Arc<TracingHandle>,
81    context: ComputeInstanceContext,
82) -> Result<impl Fn() -> Box<dyn ComputeClient> + use<>, Error> {
83    let config = Config {
84        persist_clients,
85        txns_ctx,
86        tracing_handle,
87        metrics: ComputeMetrics::register_with(metrics_registry),
88        context,
89    };
90    let tokio_executor = tokio::runtime::Handle::current();
91
92    let timely_container = config.build_cluster(timely_config, tokio_executor).await?;
93    let timely_container = Arc::new(Mutex::new(timely_container));
94
95    let client_builder = move || {
96        let client = ClusterClient::new(Arc::clone(&timely_container));
97        let client: Box<dyn ComputeClient> = Box::new(client);
98        client
99    };
100
101    Ok(client_builder)
102}
103
104/// Error type returned on connection nonce changes.
105///
106/// A nonce change informs workers that subsequent commands come a from a new client connection
107/// and therefore require reconciliation.
108struct NonceChange(Uuid);
109
110/// Endpoint used by workers to receive compute commands.
111///
112/// Observes nonce changes in the command stream and converts them into receive errors.
113struct CommandReceiver {
114    /// The channel supplying commands.
115    inner: command_channel::Receiver,
116    /// The ID of the Timely worker.
117    worker_id: usize,
118    /// The nonce identifying the current cluster protocol incarnation.
119    nonce: Option<Uuid>,
120    /// A stash to enable peeking the next command, used in `try_recv`.
121    stashed_command: Option<ComputeCommand>,
122}
123
124impl CommandReceiver {
125    fn new(inner: command_channel::Receiver, worker_id: usize) -> Self {
126        Self {
127            inner,
128            worker_id,
129            nonce: None,
130            stashed_command: None,
131        }
132    }
133
134    /// Receive the next pending command, if any.
135    ///
136    /// If the next command has a different nonce, this method instead returns an `Err`
137    /// containing the new nonce.
138    fn try_recv(&mut self) -> Result<Option<ComputeCommand>, NonceChange> {
139        if let Some(command) = self.stashed_command.take() {
140            return Ok(Some(command));
141        }
142        let Some((command, nonce)) = self.inner.try_recv() else {
143            return Ok(None);
144        };
145
146        trace!(worker = self.worker_id, %nonce, ?command, "received command");
147
148        if Some(nonce) == self.nonce {
149            Ok(Some(command))
150        } else {
151            self.nonce = Some(nonce);
152            self.stashed_command = Some(command);
153            Err(NonceChange(nonce))
154        }
155    }
156}
157
158/// Endpoint used by workers to send sending compute responses.
159///
160/// Tags responses with the current nonce, allowing receivers to filter out responses intended for
161/// previous client connections.
162pub(crate) struct ResponseSender {
163    /// The channel consuming responses.
164    inner: crossbeam_channel::Sender<(ComputeResponse, Uuid)>,
165    /// The ID of the Timely worker.
166    worker_id: usize,
167    /// The nonce identifying the current cluster protocol incarnation.
168    nonce: Option<Uuid>,
169}
170
171impl ResponseSender {
172    fn new(inner: crossbeam_channel::Sender<(ComputeResponse, Uuid)>, worker_id: usize) -> Self {
173        Self {
174            inner,
175            worker_id,
176            nonce: None,
177        }
178    }
179
180    /// Set the cluster protocol nonce.
181    fn set_nonce(&mut self, nonce: Uuid) {
182        self.nonce = Some(nonce);
183    }
184
185    /// Send a compute response.
186    pub fn send(&self, response: ComputeResponse) -> Result<(), SendError<ComputeResponse>> {
187        let nonce = self.nonce.expect("nonce must be initialized");
188
189        trace!(worker = self.worker_id, %nonce, ?response, "sending response");
190        self.inner
191            .send((response, nonce))
192            .map_err(|SendError((resp, _))| SendError(resp))
193    }
194}
195
196/// State maintained for each worker thread.
197///
198/// Much of this state can be viewed as local variables for the worker thread,
199/// holding state that persists across function calls.
200struct Worker<'w, A: Allocate> {
201    /// The underlying Timely worker.
202    timely_worker: &'w mut TimelyWorker<A>,
203    /// The channel over which commands are received.
204    command_rx: CommandReceiver,
205    /// The channel over which responses are sent.
206    response_tx: ResponseSender,
207    compute_state: Option<ComputeState>,
208    /// Compute metrics.
209    metrics: WorkerMetrics,
210    /// A process-global cache of (blob_uri, consensus_uri) -> PersistClient.
211    /// This is intentionally shared between workers
212    persist_clients: Arc<PersistClientCache>,
213    /// Context necessary for rendering txn-wal operators.
214    txns_ctx: TxnsContext,
215    /// A process-global handle to tracing configuration.
216    tracing_handle: Arc<TracingHandle>,
217    context: ComputeInstanceContext,
218}
219
220impl ClusterSpec for Config {
221    type Command = ComputeCommand;
222    type Response = ComputeResponse;
223
224    fn run_worker<A: Allocate + 'static>(
225        &self,
226        timely_worker: &mut TimelyWorker<A>,
227        client_rx: crossbeam_channel::Receiver<(
228            Uuid,
229            crossbeam_channel::Receiver<ComputeCommand>,
230            mpsc::UnboundedSender<ComputeResponse>,
231        )>,
232    ) {
233        if self.context.worker_core_affinity {
234            set_core_affinity(timely_worker.index());
235        }
236
237        let worker_id = timely_worker.index();
238        let metrics = self.metrics.for_worker(worker_id);
239
240        // Create the command channel that broadcasts commands from worker 0 to other workers. We
241        // reuse this channel between client connections, to avoid bugs where different workers end
242        // up creating incompatible sides of the channel dataflow after reconnects.
243        // See database-issues#8964.
244        let (cmd_tx, cmd_rx) = command_channel::render(timely_worker);
245        let (resp_tx, resp_rx) = crossbeam_channel::unbounded();
246
247        spawn_channel_adapter(client_rx, cmd_tx, resp_rx, worker_id);
248
249        Worker {
250            timely_worker,
251            command_rx: CommandReceiver::new(cmd_rx, worker_id),
252            response_tx: ResponseSender::new(resp_tx, worker_id),
253            metrics,
254            context: self.context.clone(),
255            persist_clients: Arc::clone(&self.persist_clients),
256            txns_ctx: self.txns_ctx.clone(),
257            compute_state: None,
258            tracing_handle: Arc::clone(&self.tracing_handle),
259        }
260        .run()
261    }
262}
263
264/// Set the current thread's core affinity, based on the given `worker_id`.
265#[cfg(not(target_os = "macos"))]
266fn set_core_affinity(worker_id: usize) {
267    use tracing::error;
268
269    let Some(mut core_ids) = core_affinity::get_core_ids() else {
270        error!(worker_id, "unable to get core IDs for setting affinity");
271        return;
272    };
273
274    // The `get_core_ids` docs don't say anything about a guaranteed order of the returned Vec,
275    // so sort it just to be safe.
276    core_ids.sort_unstable_by_key(|i| i.id);
277
278    // On multi-process replicas `worker_id` might be greater than the number of available cores.
279    // However, we assume that we always have at least as many cores as there are local workers.
280    // Violating this assumption is safe but might lead to degraded performance due to skew in core
281    // utilization.
282    let idx = worker_id % core_ids.len();
283    let core_id = core_ids[idx];
284
285    if core_affinity::set_for_current(core_id) {
286        info!(
287            worker_id,
288            core_id = core_id.id,
289            "set core affinity for worker"
290        );
291    } else {
292        error!(
293            worker_id,
294            core_id = core_id.id,
295            "failed to set core affinity for worker"
296        )
297    }
298}
299
300/// Set the current thread's core affinity, based on the given `worker_id`.
301#[cfg(target_os = "macos")]
302fn set_core_affinity(_worker_id: usize) {
303    // Setting core affinity is known to not work on Apple Silicon:
304    // https://github.com/Elzair/core_affinity_rs/issues/22
305    info!("setting core affinity is not supported on macOS");
306}
307
308impl<'w, A: Allocate + 'static> Worker<'w, A> {
309    /// Runs a compute worker.
310    pub fn run(&mut self) {
311        // The command receiver is initialized without an nonce, so receiving the first command
312        // always triggers a nonce change.
313        let NonceChange(nonce) = self.recv_command().expect_err("change to first nonce");
314        self.set_nonce(nonce);
315
316        loop {
317            let Err(NonceChange(nonce)) = self.run_client();
318            self.set_nonce(nonce);
319        }
320    }
321
322    fn set_nonce(&mut self, nonce: Uuid) {
323        self.response_tx.set_nonce(nonce);
324    }
325
326    /// Handles commands for a client connection, returns when the nonce changes.
327    fn run_client(&mut self) -> Result<Infallible, NonceChange> {
328        self.reconcile()?;
329
330        // The last time we did periodic maintenance.
331        let mut last_maintenance = Instant::now();
332
333        // Commence normal operation.
334        loop {
335            // Get the maintenance interval, default to zero if we don't have a compute state.
336            let maintenance_interval = self
337                .compute_state
338                .as_ref()
339                .map_or(Duration::ZERO, |state| state.server_maintenance_interval);
340
341            let now = Instant::now();
342            // Determine if we need to perform maintenance, which is true if `maintenance_interval`
343            // time has passed since the last maintenance.
344            let sleep_duration;
345            if now >= last_maintenance + maintenance_interval {
346                last_maintenance = now;
347                sleep_duration = None;
348
349                // Report frontier information back the coordinator.
350                if let Some(mut compute_state) = self.activate_compute() {
351                    compute_state.compute_state.traces.maintenance();
352                    compute_state.report_frontiers();
353                    compute_state.report_dropped_collections();
354                    compute_state.report_metrics();
355                    compute_state.check_expiration();
356                }
357
358                self.metrics.record_shared_row_metrics();
359            } else {
360                // We didn't perform maintenance, sleep until the next maintenance interval.
361                let next_maintenance = last_maintenance + maintenance_interval;
362                sleep_duration = Some(next_maintenance.saturating_duration_since(now))
363            };
364
365            // Step the timely worker, recording the time taken.
366            let timer = self.metrics.timely_step_duration_seconds.start_timer();
367            self.timely_worker.step_or_park(sleep_duration);
368            timer.observe_duration();
369
370            self.handle_pending_commands()?;
371
372            if let Some(mut compute_state) = self.activate_compute() {
373                compute_state.process_peeks();
374                compute_state.process_subscribes();
375                compute_state.process_copy_tos();
376            }
377        }
378    }
379
380    fn handle_pending_commands(&mut self) -> Result<(), NonceChange> {
381        while let Some(cmd) = self.command_rx.try_recv()? {
382            self.handle_command(cmd);
383        }
384        Ok(())
385    }
386
387    fn handle_command(&mut self, cmd: ComputeCommand) {
388        match &cmd {
389            ComputeCommand::CreateInstance(_) => {
390                self.compute_state = Some(ComputeState::new(
391                    Arc::clone(&self.persist_clients),
392                    self.txns_ctx.clone(),
393                    self.metrics.clone(),
394                    Arc::clone(&self.tracing_handle),
395                    self.context.clone(),
396                ));
397            }
398            _ => (),
399        }
400        self.activate_compute().unwrap().handle_compute_command(cmd);
401    }
402
403    fn activate_compute(&mut self) -> Option<ActiveComputeState<'_, A>> {
404        if let Some(compute_state) = &mut self.compute_state {
405            Some(ActiveComputeState {
406                timely_worker: &mut *self.timely_worker,
407                compute_state,
408                response_tx: &mut self.response_tx,
409            })
410        } else {
411            None
412        }
413    }
414
415    /// Receive the next compute command.
416    ///
417    /// This method blocks if no command is currently available, but takes care to step the Timely
418    /// worker while doing so.
419    fn recv_command(&mut self) -> Result<ComputeCommand, NonceChange> {
420        loop {
421            if let Some(cmd) = self.command_rx.try_recv()? {
422                return Ok(cmd);
423            }
424
425            let start = Instant::now();
426            self.timely_worker.step_or_park(None);
427            self.metrics
428                .timely_step_duration_seconds
429                .observe(start.elapsed().as_secs_f64());
430        }
431    }
432
433    /// Extract commands until `InitializationComplete`, and make the worker reflect those commands.
434    ///
435    /// This method is meant to be a function of the commands received thus far (as recorded in the
436    /// compute state command history) and the new commands from `command_rx`. It should not be a
437    /// function of other characteristics, like whether the worker has managed to respond to a peek
438    /// or not. Some effort goes in to narrowing our view to only the existing commands we can be sure
439    /// are live at all other workers.
440    ///
441    /// The methodology here is to drain `command_rx` until an `InitializationComplete`, at which point
442    /// the prior commands are "reconciled" in. Reconciliation takes each goal dataflow and looks for an
443    /// existing "compatible" dataflow (per `compatible()`) it can repurpose, with some additional tests
444    /// to be sure that we can cut over from one to the other (no additional compaction, no tails/sinks).
445    /// With any connections established, old orphaned dataflows are allow to compact away, and any new
446    /// dataflows are created from scratch. "Kept" dataflows are allowed to compact up to any new `as_of`.
447    ///
448    /// Some additional tidying happens, cleaning up pending peeks, reported frontiers, and creating a new
449    /// subscribe response buffer. We will need to be vigilant with future modifications to `ComputeState` to
450    /// line up changes there with clean resets here.
451    fn reconcile(&mut self) -> Result<(), NonceChange> {
452        // To initialize the connection, we want to drain all commands until we receive a
453        // `ComputeCommand::InitializationComplete` command to form a target command state.
454        let mut new_commands = Vec::new();
455        loop {
456            match self.recv_command()? {
457                ComputeCommand::InitializationComplete => break,
458                command => new_commands.push(command),
459            }
460        }
461
462        // Commands we will need to apply before entering normal service.
463        // These commands may include dropping existing dataflows, compacting existing dataflows,
464        // and creating new dataflows, in addition to standard peek and compaction commands.
465        // The result should be the same as if dropping all dataflows and running `new_commands`.
466        let mut todo_commands = Vec::new();
467        // We only have a compute history if we are in an initialized state
468        // (i.e. after a `CreateInstance`).
469        // If this is not the case, just copy `new_commands` into `todo_commands`.
470        if let Some(compute_state) = &mut self.compute_state {
471            // Reduce the installed commands.
472            // Importantly, act as if all peeks may have been retired (as we cannot know otherwise).
473            compute_state.command_history.discard_peeks();
474            compute_state.command_history.reduce();
475
476            // At this point, we need to sort out which of the *certainly installed* dataflows are
477            // suitable replacements for the requested dataflows. A dataflow is "certainly installed"
478            // as of a frontier if its compaction allows it to go no further. We ignore peeks for this
479            // reasoning, as we cannot be certain that peeks still exist at any other worker.
480
481            // Having reduced our installed command history retaining no peeks (above), we should be able
482            // to use track down installed dataflows we can use as surrogates for requested dataflows (which
483            // have retained all of their peeks, creating a more demanding `as_of` requirement).
484            // NB: installed dataflows may still be allowed to further compact, and we should double check
485            // this before being too confident. It should be rare without peeks, but could happen with e.g.
486            // multiple outputs of a dataflow.
487
488            // The values with which a prior `CreateInstance` was called, if it was.
489            let mut old_instance_config = None;
490            // Index dataflows by `export_ids().collect()`, as this is a precondition for their compatibility.
491            let mut old_dataflows = BTreeMap::default();
492            // Maintain allowed compaction, in case installed identifiers may have been allowed to compact.
493            let mut old_frontiers = BTreeMap::default();
494            for command in compute_state.command_history.iter() {
495                match command {
496                    ComputeCommand::CreateInstance(config) => {
497                        old_instance_config = Some(config);
498                    }
499                    ComputeCommand::CreateDataflow(dataflow) => {
500                        let export_ids = dataflow.export_ids().collect::<BTreeSet<_>>();
501                        old_dataflows.insert(export_ids, dataflow);
502                    }
503                    ComputeCommand::AllowCompaction { id, frontier } => {
504                        old_frontiers.insert(id, frontier);
505                    }
506                    _ => {
507                        // Nothing to do in these cases.
508                    }
509                }
510            }
511
512            // Compaction commands that can be applied to existing dataflows.
513            let mut old_compaction = BTreeMap::default();
514            // Exported identifiers from dataflows we retain.
515            let mut retain_ids = BTreeSet::default();
516
517            // Traverse new commands, sorting out what remediation we can do.
518            for command in new_commands.iter() {
519                match command {
520                    ComputeCommand::CreateDataflow(dataflow) => {
521                        // Attempt to find an existing match for the dataflow.
522                        let as_of = dataflow.as_of.as_ref().unwrap();
523                        let export_ids = dataflow.export_ids().collect::<BTreeSet<_>>();
524
525                        if let Some(old_dataflow) = old_dataflows.get(&export_ids) {
526                            let compatible = old_dataflow.compatible_with(dataflow);
527                            let uncompacted = !export_ids
528                                .iter()
529                                .flat_map(|id| old_frontiers.get(id))
530                                .any(|frontier| {
531                                    !timely::PartialOrder::less_equal(
532                                        *frontier,
533                                        dataflow.as_of.as_ref().unwrap(),
534                                    )
535                                });
536
537                            // We cannot reconcile subscribe and copy-to sinks at the moment,
538                            // because the response buffer is shared, and to a first approximation
539                            // must be completely reformed.
540                            let subscribe_free = dataflow.subscribe_ids().next().is_none();
541                            let copy_to_free = dataflow.copy_to_ids().next().is_none();
542
543                            // If we have replaced any dependency of this dataflow, we need to
544                            // replace this dataflow, to make it use the replacement.
545                            let dependencies_retained = dataflow
546                                .imported_index_ids()
547                                .all(|id| retain_ids.contains(&id));
548
549                            if compatible
550                                && uncompacted
551                                && subscribe_free
552                                && copy_to_free
553                                && dependencies_retained
554                            {
555                                // Match found; remove the match from the deletion queue,
556                                // and compact its outputs to the dataflow's `as_of`.
557                                old_dataflows.remove(&export_ids);
558                                for id in export_ids.iter() {
559                                    old_compaction.insert(*id, as_of.clone());
560                                }
561                                retain_ids.extend(export_ids);
562                            } else {
563                                warn!(
564                                    ?export_ids,
565                                    ?compatible,
566                                    ?uncompacted,
567                                    ?subscribe_free,
568                                    ?copy_to_free,
569                                    ?dependencies_retained,
570                                    old_as_of = ?old_dataflow.as_of,
571                                    new_as_of = ?as_of,
572                                    "dataflow reconciliation failed",
573                                );
574
575                                // Dump the full dataflow plans if they are incompatible, to
576                                // simplify debugging hard-to-reproduce reconciliation failures.
577                                if !compatible {
578                                    warn!(
579                                        old = ?old_dataflow,
580                                        new = ?dataflow,
581                                        "incompatible dataflows in reconciliation",
582                                    );
583                                }
584
585                                todo_commands
586                                    .push(ComputeCommand::CreateDataflow(dataflow.clone()));
587                            }
588
589                            compute_state.metrics.record_dataflow_reconciliation(
590                                compatible,
591                                uncompacted,
592                                subscribe_free,
593                                copy_to_free,
594                                dependencies_retained,
595                            );
596                        } else {
597                            todo_commands.push(ComputeCommand::CreateDataflow(dataflow.clone()));
598                        }
599                    }
600                    ComputeCommand::CreateInstance(config) => {
601                        // Cluster creation should not be performed again!
602                        if old_instance_config.map_or(false, |old| !old.compatible_with(config)) {
603                            halt!(
604                                "new instance configuration not compatible with existing instance configuration:\n{:?}\nvs\n{:?}",
605                                config,
606                                old_instance_config,
607                            );
608                        }
609                    }
610                    // All other commands we apply as requested.
611                    command => {
612                        todo_commands.push(command.clone());
613                    }
614                }
615            }
616
617            // Issue compaction commands first to reclaim resources.
618            for (_, dataflow) in old_dataflows.iter() {
619                for id in dataflow.export_ids() {
620                    // We want to drop anything that has not yet been dropped,
621                    // and nothing that has already been dropped.
622                    if old_frontiers.get(&id) != Some(&&Antichain::new()) {
623                        old_compaction.insert(id, Antichain::new());
624                    }
625                }
626            }
627            for (&id, frontier) in &old_compaction {
628                let frontier = frontier.clone();
629                todo_commands.insert(0, ComputeCommand::AllowCompaction { id, frontier });
630            }
631
632            // Clean up worker-local state.
633            //
634            // Various aspects of `ComputeState` need to be either uninstalled, or return to a blank slate.
635            // All dropped dataflows should clean up after themselves, as we plan to install new dataflows
636            // re-using the same identifiers.
637            // All re-used dataflows should roll back any believed communicated information (e.g. frontiers)
638            // so that they recommunicate that information as if from scratch.
639
640            // Remove all pending peeks.
641            for (_, peek) in std::mem::take(&mut compute_state.pending_peeks) {
642                // Log dropping the peek request.
643                if let Some(logger) = compute_state.compute_logger.as_mut() {
644                    logger.log(&peek.as_log_event(false));
645                }
646            }
647
648            // Clear the list of dropped collections.
649            // We intended to report their dropping, but the controller does not expect to hear
650            // about them anymore.
651            compute_state.dropped_collections = Default::default();
652
653            for (&id, collection) in compute_state.collections.iter_mut() {
654                // Adjust reported frontiers:
655                //  * For dataflows we continue to use, reset to ensure we report something not
656                //    before the new `as_of` next.
657                //  * For dataflows we drop, set to the empty frontier, to ensure we don't report
658                //    anything for them.
659                let retained = retain_ids.contains(&id);
660                let compaction = old_compaction.remove(&id);
661                let new_reported_frontier = match (retained, compaction) {
662                    (true, Some(new_as_of)) => ReportedFrontier::NotReported { lower: new_as_of },
663                    (true, None) => {
664                        unreachable!("retained dataflows are compacted to the new as_of")
665                    }
666                    (false, Some(new_frontier)) => {
667                        assert!(new_frontier.is_empty());
668                        ReportedFrontier::Reported(new_frontier)
669                    }
670                    (false, None) => {
671                        // Logging dataflows are implicitly retained and don't have a new as_of.
672                        // Reset them to the minimal frontier.
673                        ReportedFrontier::new()
674                    }
675                };
676
677                collection.reset_reported_frontiers(new_reported_frontier);
678
679                // Sink tokens should be retained for retained dataflows, and dropped for dropped
680                // dataflows.
681                //
682                // Dropping the tokens of active subscribe and copy-tos makes them place
683                // `DroppedAt` responses into the respective response buffer. We drop those buffers
684                // in the next step, which ensures that we don't send out `DroppedAt` responses for
685                // subscribe/copy-tos dropped during reconciliation.
686                if !retained {
687                    collection.sink_token = None;
688                }
689            }
690
691            // We must drop the response buffers as they are global across all subscribe/copy-tos.
692            // If they were broken out by `GlobalId` then we could drop only the response buffers
693            // of dataflows we drop.
694            compute_state.subscribe_response_buffer = Rc::new(RefCell::new(Vec::new()));
695            compute_state.copy_to_response_buffer = Rc::new(RefCell::new(Vec::new()));
696
697            // The controller expects the logging collections to be readable from the minimum time
698            // initially. We cannot recreate the logging arrangements without restarting the
699            // instance, but we can pad the compacted times with empty data. Doing so is sound
700            // because logging collections from different replica incarnations are considered
701            // distinct TVCs, so the controller doesn't expect any historical consistency from
702            // these collections when it reconnects to a replica.
703            //
704            // TODO(database-issues#8152): Consider resolving this with controller-side reconciliation instead.
705            if let Some(config) = old_instance_config {
706                for id in config.logging.index_logs.values() {
707                    let trace = compute_state
708                        .traces
709                        .remove(id)
710                        .expect("logging trace exists");
711                    let padded = trace.into_padded();
712                    compute_state.traces.set(*id, padded);
713                }
714            }
715        } else {
716            todo_commands.clone_from(&new_commands);
717        }
718
719        // Execute the commands to bring us to `new_commands`.
720        for command in todo_commands.into_iter() {
721            self.handle_command(command);
722        }
723
724        // Overwrite `self.command_history` to reflect `new_commands`.
725        // It is possible that there still isn't a compute state yet.
726        if let Some(compute_state) = &mut self.compute_state {
727            let mut command_history = ComputeCommandHistory::new(self.metrics.for_history());
728            for command in new_commands.iter() {
729                command_history.push(command.clone());
730            }
731            compute_state.command_history = command_history;
732        }
733        Ok(())
734    }
735}
736
737/// Spawn a thread to bridge between [`ClusterClient`] and [`Worker`] channels.
738///
739/// The [`Worker`] expects a pair of persistent channels, with punctuation marking reconnects,
740/// while the [`ClusterClient`] provides a new pair of channels on each reconnect.
741fn spawn_channel_adapter(
742    client_rx: crossbeam_channel::Receiver<(
743        Uuid,
744        crossbeam_channel::Receiver<ComputeCommand>,
745        mpsc::UnboundedSender<ComputeResponse>,
746    )>,
747    command_tx: command_channel::Sender,
748    response_rx: crossbeam_channel::Receiver<(ComputeResponse, Uuid)>,
749    worker_id: usize,
750) {
751    thread::Builder::new()
752        // "cca" stands for "compute channel adapter". We need to shorten that because Linux has a
753        // 15-character limit for thread names.
754        .name(format!("cca-{worker_id}"))
755        .spawn(move || {
756            // To make workers aware of the individual client connections, we tag forwarded
757            // commands with the client nonce. Additionally, we use the nonce to filter out
758            // responses with a different nonce, which are intended for different client
759            // connections.
760            //
761            // It's possible that we receive responses with nonces from the past but also from the
762            // future: Worker 0 might have received a new nonce before us and broadcasted it to our
763            // Timely cluster. When we receive a response with a future nonce, we need to wait with
764            // forwarding it until we have received the same nonce from a client connection.
765            //
766            // Nonces are not ordered so we don't know whether a response nonce is from the past or
767            // the future. We thus assume that every response with an unknown nonce might be from
768            // the future and stash them all. Every time we reconnect, we immediately send all
769            // stashed responses with a matching nonce. Every time we receive a new response with a
770            // nonce that matches our current one, we can discard the entire response stash as we
771            // know that all stashed responses must be from the past.
772            let mut stashed_responses = BTreeMap::<Uuid, Vec<ComputeResponse>>::new();
773
774            while let Ok((nonce, command_rx, response_tx)) = client_rx.recv() {
775                // Send stashed responses for this client.
776                if let Some(resps) = stashed_responses.remove(&nonce) {
777                    for resp in resps {
778                        let _ = response_tx.send(resp);
779                    }
780                }
781
782                // Wait for a new response while forwarding received commands.
783                let serve_rx_channels = || loop {
784                    crossbeam_channel::select! {
785                        recv(command_rx) -> msg => match msg {
786                            Ok(cmd) => command_tx.send((cmd, nonce)),
787                            Err(_) => return Err(()),
788                        },
789                        recv(response_rx) -> msg => {
790                            return Ok(msg.expect("worker connected"));
791                        }
792                    }
793                };
794
795                // Serve this connection until we see any of the channels disconnect.
796                loop {
797                    let Ok((resp, resp_nonce)) = serve_rx_channels() else {
798                        break;
799                    };
800
801                    if resp_nonce == nonce {
802                        // Response for the current connection; forward it.
803                        stashed_responses.clear();
804                        if response_tx.send(resp).is_err() {
805                            break;
806                        }
807                    } else {
808                        // Response for a past or future connection; stash it.
809                        let stash = stashed_responses.entry(resp_nonce).or_default();
810                        stash.push(resp);
811                    }
812                }
813            }
814        })
815        .unwrap();
816}