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
// 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.

use std::any::Any;
use std::cell::RefCell;
use std::ops::DerefMut;
use std::rc::Rc;

use differential_dataflow::Collection;

use timely::dataflow::channels::pact::Pipeline;
use timely::dataflow::operators::Operator;
use timely::dataflow::Scope;
use timely::progress::timestamp::Timestamp as TimelyTimestamp;
use timely::progress::Antichain;

use mz_compute_client::response::{TailBatch, TailResponse};
use mz_compute_client::sinks::{ComputeSinkDesc, SinkAsOf, TailSinkConnection};
use mz_repr::{Diff, GlobalId, Row, Timestamp};
use mz_storage::controller::CollectionMetadata;
use mz_storage::types::errors::DataflowError;

use crate::render::sinks::SinkRender;

impl<G> SinkRender<G> for TailSinkConnection
where
    G: Scope<Timestamp = Timestamp>,
{
    fn render_continuous_sink(
        &self,
        compute_state: &mut crate::compute_state::ComputeState,
        sink: &ComputeSinkDesc<CollectionMetadata>,
        sink_id: GlobalId,
        sinked_collection: Collection<G, Row, Diff>,
        // TODO(benesch): errors should stream out through the sink,
        // if we figure out a protocol for that.
        _err_collection: Collection<G, DataflowError, Diff>,
    ) -> Option<Rc<dyn Any>>
    where
        G: Scope<Timestamp = Timestamp>,
    {
        // An encapsulation of the Tail response protocol.
        // Used to send rows and progress messages,
        // and alert if the dataflow was dropped before completing.
        let tail_protocol_handle = Rc::new(RefCell::new(Some(TailProtocol {
            sink_id,
            tail_response_buffer: Some(Rc::clone(&compute_state.tail_response_buffer)),
            prev_upper: Antichain::from_elem(Timestamp::minimum()),
        })));
        let tail_protocol_weak = Rc::downgrade(&tail_protocol_handle);

        tail(
            sinked_collection,
            sink_id,
            sink.as_of.clone(),
            tail_protocol_handle,
        );

        // Inform the coordinator that we have been dropped,
        // and destroy the tail protocol so the sink operator
        // can't send spurious messages while shutting down.
        Some(Rc::new(scopeguard::guard((), move |_| {
            if let Some(tail_protocol_handle) = tail_protocol_weak.upgrade() {
                std::mem::drop(tail_protocol_handle.borrow_mut().take())
            }
        })))
    }
}

fn tail<G>(
    sinked_collection: Collection<G, Row, Diff>,
    sink_id: GlobalId,
    as_of: SinkAsOf,
    tail_protocol_handle: Rc<RefCell<Option<TailProtocol>>>,
) where
    G: Scope<Timestamp = Timestamp>,
{
    let mut results = Vec::new();
    sinked_collection
        .inner
        .sink(Pipeline, &format!("tail-{}", sink_id), move |input| {
            input.for_each(|_, rows| {
                for (row, time, diff) in rows.iter() {
                    let should_emit = if as_of.strict {
                        as_of.frontier.less_than(time)
                    } else {
                        as_of.frontier.less_equal(time)
                    };
                    if should_emit {
                        results.push((*time, row.clone(), *diff));
                    }
                }
            });

            if let Some(tail_protocol) = tail_protocol_handle.borrow_mut().deref_mut() {
                tail_protocol.send_batch(input.frontier().frontier().to_owned(), &mut results);
            }
        })
}

/// A type that guides the transmission of rows back to the coordinator.
///
/// A protocol instance may `send` rows indefinitely, and is consumed by `complete`,
/// which is used only to indicate the end of a stream. The `Drop` implementation
/// otherwise sends an indication that the protocol has finished without completion.
struct TailProtocol {
    pub sink_id: GlobalId,
    pub tail_response_buffer: Option<Rc<RefCell<Vec<(GlobalId, TailResponse)>>>>,
    pub prev_upper: Antichain<Timestamp>,
}

impl TailProtocol {
    fn send_batch(&mut self, upper: Antichain<Timestamp>, rows: &mut Vec<(Timestamp, Row, Diff)>) {
        if self.prev_upper != upper {
            let mut ship = Vec::new();
            let mut keep = Vec::new();
            differential_dataflow::consolidation::consolidate_updates(rows);
            for (time, data, diff) in rows.drain(..) {
                if upper.less_equal(&time) {
                    keep.push((time, data, diff));
                } else {
                    ship.push((time, data, diff));
                }
            }
            *rows = keep;

            let input_exhausted = upper.is_empty();
            let buffer = self
                .tail_response_buffer
                .as_mut()
                .expect("The tail response buffer is only cleared on drop.");
            buffer.borrow_mut().push((
                self.sink_id,
                TailResponse::Batch(TailBatch {
                    lower: self.prev_upper.clone(),
                    upper: upper.clone(),
                    updates: ship,
                }),
            ));
            self.prev_upper = upper;
            if input_exhausted {
                // The dataflow's input has been exhausted; clear the channel,
                // to avoid sending `TailResponse::DroppedAt`.
                self.tail_response_buffer = None;
            }
        }
    }
}

impl Drop for TailProtocol {
    fn drop(&mut self) {
        if let Some(buffer) = self.tail_response_buffer.take() {
            buffer.borrow_mut().push((
                self.sink_id,
                TailResponse::DroppedAt(self.prev_upper.clone()),
            ));
        }
    }
}