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
// 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::operators::arrange::ArrangeByKey;
use differential_dataflow::trace::cursor::Cursor;
use differential_dataflow::trace::BatchReader;
use differential_dataflow::Collection;

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

use dataflow_types::{
    sinks::{SinkAsOf, SinkDesc, TailSinkConnector},
    TailResponse,
};
use expr::GlobalId;
use repr::{Diff, Row, Timestamp};

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

use super::SinkBaseMetrics;

impl<G> SinkRender<G> for TailSinkConnector
where
    G: Scope<Timestamp = Timestamp>,
{
    fn uses_keys(&self) -> bool {
        false
    }

    fn get_key_indices(&self) -> Option<&[usize]> {
        None
    }

    fn get_relation_key_indices(&self) -> Option<&[usize]> {
        None
    }

    fn render_continuous_sink(
        &self,
        compute_state: &mut crate::render::ComputeState,
        sink: &SinkDesc,
        sink_id: GlobalId,
        sinked_collection: Collection<G, (Option<Row>, Option<Row>), Diff>,
        _metrics: &SinkBaseMetrics,
    ) -> Option<Rc<dyn Any>>
    where
        G: Scope<Timestamp = Timestamp>,
    {
        let scope = sinked_collection.scope();
        use differential_dataflow::Hashable;
        // `active_worker` indicates the index of the worker that receives all data.
        let active_worker = (sink_id.hashed() as usize) % scope.peers() == scope.index();
        // An encapsulation of the Tail response protocol.
        // Used to send rows and progress messages,
        // and alert if the dataflow was dropped before completing.
        // Set to `None` for instances that should not produce output.
        let tail_protocol_handle = Rc::new(RefCell::new(if active_worker {
            Some(TailProtocol {
                sink_id,
                tail_response_buffer: Some(Rc::clone(&compute_state.tail_response_buffer)),
            })
        } else {
            None
        }));
        let tail_protocol_weak = Rc::downgrade(&tail_protocol_handle);

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

        // 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, (Option<Row>, Option<Row>), Diff>,
    sink_id: GlobalId,
    as_of: SinkAsOf,
    tail_protocol_handle: Rc<RefCell<Option<TailProtocol>>>,
    active_worker: bool,
) where
    G: Scope<Timestamp = Timestamp>,
{
    // make sure all data is routed to one worker by keying on the sink id
    let batches = sinked_collection
        .map(move |(k, v)| {
            assert!(k.is_none(), "tail does not support keys");
            let v = v.expect("tail must have values");
            (sink_id, v)
        })
        .arrange_by_key()
        .stream;

    // Initialize to the minimal input frontier.
    let mut input_frontier = Antichain::from_elem(<G::Timestamp as TimelyTimestamp>::minimum());

    batches.sink(Pipeline, &format!("tail-{}", sink_id), move |input| {
        input.for_each(|_, batches| {
            let mut results = vec![];
            for batch in batches.iter() {
                let mut cursor = batch.cursor();
                while cursor.key_valid(&batch) {
                    while cursor.val_valid(&batch) {
                        let row = cursor.val(&batch);
                        cursor.map_times(&batch, |time, diff| {
                            let diff = *diff;
                            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((row.clone(), *time, diff));
                            }
                        });
                        cursor.step_val(&batch);
                    }
                    cursor.step_key(&batch);
                }
            }

            if active_worker {
                // Emit data if configured, otherwise it is an error to have data to send.
                if let Some(tail_protocol) = tail_protocol_handle.borrow_mut().deref_mut() {
                    tail_protocol.send_rows(results);
                }
            } else {
                assert!(
                    results.is_empty(),
                    "Observed data at inactive TAIL instance"
                );
            }

            if let Some(batch) = batches.last() {
                let progress = update_progress(&mut input_frontier, batch.desc.upper().borrow());
                if let (Some(tail_protocol), Some(progress)) =
                    (tail_protocol_handle.borrow_mut().deref_mut(), progress)
                {
                    tail_protocol.send_progress(progress);
                }
            }
        });

        let progress = update_progress(&mut input_frontier, input.frontier().frontier());

        // Only emit updates if this operator/worker received actual data for emission.
        if let (Some(tail_protocol), Some(progress)) =
            (tail_protocol_handle.borrow_mut().deref_mut(), progress)
        {
            tail_protocol.send_progress(progress);
        }
    })
}

/// 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)>>>>,
}

impl TailProtocol {
    /// Send the current upper frontier of the tail.
    fn send_progress(&mut self, upper: Antichain<Timestamp>) {
        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::Progress(upper)));
        if input_exhausted {
            // The dataflow's input has been exhausted; clear the channel,
            // to avoid sending `TailResponse::Dropped`.
            self.tail_response_buffer = None;
        }
    }

    /// Send further rows as responses.
    fn send_rows(&mut self, rows: Vec<(Row, Timestamp, Diff)>) {
        let buffer = self
            .tail_response_buffer
            .as_mut()
            .expect("Unexpectedly saw more rows! The tail response buffer should only have been cleared when either the dataflow was dropped or the input was exhausted.");
        buffer
            .borrow_mut()
            .push((self.sink_id, TailResponse::Rows(rows)));
    }
}

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::Dropped));
        }
    }
}

// Checks if there is progress between `current_input_frontier` and
// `new_input_frontier`. If yes, updates the `current_input_frontier` to
// `new_input_frontier` and returns a [`Row`] that encodes this progress. If
// there is no progress, returns [`None`].
fn update_progress(
    current_input_frontier: &mut Antichain<Timestamp>,
    new_input_frontier: AntichainRef<Timestamp>,
) -> Option<Antichain<Timestamp>> {
    // Test to see if strict progress has occurred. This is true if the new
    // frontier is not less or equal to the old frontier.
    let progress = !PartialOrder::less_equal(&new_input_frontier, &current_input_frontier.borrow());

    if progress {
        current_input_frontier.clear();
        current_input_frontier.extend(new_input_frontier.iter().cloned());

        Some(new_input_frontier.to_owned())
    } else {
        None
    }
}