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
// 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::io::{self, BufRead, Read};
use std::path::PathBuf;
use std::str::FromStr;
use std::sync::mpsc::{self, Receiver, TryRecvError};
use std::thread;

use anyhow::{Context, Error};
use flate2::read::MultiGzDecoder;
#[cfg(target_os = "linux")]
use inotify::{EventMask, Inotify, WatchMask};
use repr::MessagePayload;
use timely::scheduling::SyncActivator;
use tracing::error;

use dataflow_types::sources::{
    encoding::AvroOcfEncoding, encoding::DataEncoding, encoding::SourceDataEncoding, Compression,
    ExternalSourceConnector, MzOffset,
};
use expr::{PartitionId, SourceInstanceId};
use mz_avro::Block;
use mz_avro::BlockIter;
use mz_avro::{AvroRead, Schema, Skip};

use crate::logging::materialized::Logger;
use crate::source::{NextMessage, SourceMessage, SourceReader};

use super::metrics::SourceBaseMetrics;

/// Contains all information necessary to ingest data from file sources
pub struct FileSourceReader {
    /// Unique source ID
    id: SourceInstanceId,
    /// Receiver channel that ingests records
    receiver_stream: Receiver<Result<MessagePayload, Error>>,
    /// Current File Offset. This corresponds to the offset of last processed message
    /// (initially 0 if no records have been processed)
    current_file_offset: FileOffset,
}

#[derive(Copy, Clone)]
/// Represents an index into a file. Files are 1-indexed by Unix convention
pub struct FileOffset {
    pub offset: i64,
}

/// Convert from FileOffset to MzOffset (1-indexed)
impl From<FileOffset> for MzOffset {
    fn from(file_offset: FileOffset) -> Self {
        MzOffset {
            offset: file_offset.offset,
        }
    }
}

impl SourceReader for FileSourceReader {
    type Key = ();
    type Value = MessagePayload;

    fn new(
        _name: String,
        source_id: SourceInstanceId,
        worker_id: usize,
        _worker_count: usize,
        consumer_activator: SyncActivator,
        connector: ExternalSourceConnector,
        _restored_offsets: Vec<(PartitionId, Option<MzOffset>)>,
        encoding: SourceDataEncoding,
        _: Option<Logger>,
        _: SourceBaseMetrics,
    ) -> Result<(FileSourceReader, Option<PartitionId>), anyhow::Error> {
        let receiver = match connector {
            ExternalSourceConnector::File(fc) => {
                tracing::debug!("creating FileSourceReader worker_id={}", worker_id);
                let ctor = |fi| {
                    let mut br = std::io::BufReader::new(fi);
                    Ok(std::iter::from_fn(move || {
                        let chunk = match br.fill_buf() {
                            Err(e) => return Some(Err(e)),
                            Ok(b) => b.to_vec(), // TODO - avoid this copy?
                        };
                        br.consume(chunk.len());
                        if chunk.len() > 0 {
                            Some(Ok(MessagePayload::Data(chunk)))
                        } else {
                            None
                        }
                    })
                    .chain(std::iter::once(Ok(MessagePayload::EOF))))
                };
                let (tx, rx) = std::sync::mpsc::sync_channel(10000);
                let tail = if fc.tail {
                    FileReadStyle::TailFollowFd
                } else {
                    FileReadStyle::ReadOnce
                };
                std::thread::spawn(move || {
                    read_file_task(
                        fc.path,
                        tx,
                        Some(consumer_activator),
                        tail,
                        fc.compression,
                        ctor,
                    );
                });
                rx
            }
            ExternalSourceConnector::AvroOcf(fc) => {
                tracing::debug!("creating Avro FileSourceReader worker_id={}", worker_id);
                let value_encoding = match &encoding {
                    SourceDataEncoding::Single(enc) => enc,
                    SourceDataEncoding::KeyValue { .. } => {
                        unreachable!("A KeyValue encoding for an OCF source should have been rejected in the planner.")
                    }
                };
                let reader_schema = match value_encoding {
                    DataEncoding::AvroOcf(AvroOcfEncoding { reader_schema }) => &*reader_schema,
                    // We should be checking for this in the planner.
                    _ => unreachable!("Wrong encoding for OCF file"),
                };
                let reader_schema = Schema::from_str(reader_schema)?;
                let ctor = move |file| {
                    BlockIter::with_schema(&reader_schema, file).map(|bi| {
                        bi.map(|result| {
                            result.map(|Block { bytes, len: _ }| MessagePayload::Data(bytes))
                        })
                        .chain(std::iter::once(Ok(MessagePayload::EOF)))
                    })
                };

                let (tx, rx) = std::sync::mpsc::sync_channel(10000);
                let tail = if fc.tail {
                    FileReadStyle::TailFollowFd
                } else {
                    FileReadStyle::ReadOnce
                };
                std::thread::spawn(move || {
                    read_file_task(
                        fc.path,
                        tx,
                        Some(consumer_activator),
                        tail,
                        fc.compression,
                        ctor,
                    );
                });
                rx
            }
            _ => unreachable!(),
        };

        Ok((
            FileSourceReader {
                id: source_id,
                receiver_stream: receiver,
                current_file_offset: FileOffset { offset: 0 },
            },
            Some(PartitionId::None),
        ))
    }

    fn get_next_message(&mut self) -> Result<NextMessage<Self::Key, Self::Value>, anyhow::Error> {
        match self.receiver_stream.try_recv() {
            Ok(Ok(record)) => {
                self.current_file_offset.offset += 1;
                let message = SourceMessage {
                    partition: PartitionId::None,
                    offset: self.current_file_offset.into(),
                    upstream_time_millis: None,
                    key: (),
                    value: record,
                };
                Ok(NextMessage::Ready(message))
            }
            Ok(Err(e)) => {
                error!("Failed to read file for {}. Error: {}.", self.id, e);
                Err(e)
            }
            Err(TryRecvError::Empty) => Ok(NextMessage::Pending),
            Err(TryRecvError::Disconnected) => Ok(NextMessage::Finished),
        }
    }
}

/// Blocking logic to read from a file, intended for its own thread.
pub fn read_file_task<Ctor, I, Err>(
    path: PathBuf,
    tx: std::sync::mpsc::SyncSender<Result<MessagePayload, anyhow::Error>>,
    activator: Option<SyncActivator>,
    read_style: FileReadStyle,
    compression: Compression,
    iter_ctor: Ctor,
) where
    I: IntoIterator<Item = Result<MessagePayload, Err>> + Send + 'static,
    Ctor: FnOnce(Box<dyn AvroRead + Send>) -> Result<I, Err>,
    Err: Into<anyhow::Error>,
{
    tracing::trace!("reading file {}", path.display());
    let file = match std::fs::File::open(&path).with_context(|| {
        format!(
            "file source: unable to open file at path {}",
            path.to_string_lossy(),
        )
    }) {
        Ok(file) => file,
        Err(err) => {
            // If we fail to send an error, it's likely due to a race condition
            // with the source being closed.
            let _ = tx.send(Err(err));
            return;
        }
    };

    let file_stream = match open_file_stream(path.clone(), file, read_style) {
        Ok(f) => f,
        Err(err) => {
            // If we fail to send an error, it's likely due to a race condition
            // with the source being closed.
            let _ = tx.send(Err(err));
            return;
        }
    };

    let file_stream: Box<dyn AvroRead + Send> = match compression {
        Compression::Gzip => Box::new(MultiGzDecoder::new(file_stream)),
        Compression::None => Box::new(file_stream),
    };

    let iter = iter_ctor(file_stream);

    match iter.map_err(Into::into).with_context(|| {
        format!(
            "Failed to obtain records from file at path {}",
            path.to_string_lossy(),
        )
    }) {
        Ok(i) => send_records(i, tx, activator),
        Err(e) => {
            // If we fail to send an error, it's likely due to a race condition
            // with the source being closed.
            let _ = tx.send(Err(e));
        }
    };
}

fn open_file_stream(
    _path: PathBuf,
    file: std::fs::File,
    read_style: FileReadStyle,
) -> Result<Box<dyn AvroRead + Send>, anyhow::Error> {
    match read_style {
        FileReadStyle::ReadOnce => Ok(Box::new(file)),
        FileReadStyle::TailFollowFd => {
            let (notice_tx, notice_rx) = mpsc::channel();

            // FSEvents doesn't raise events until you close the file, making it
            // useless for tailing log files that are kept open by the daemon
            // writing to them.
            //
            // Avoid this issue by just waking up and polling the file on macOS
            // every 100ms. We don't want to use notify::PollWatcher, since that
            // occasionally misses updates if the file is changed twice within
            // one second (it uses an mtime granularity of 1s). Plus it's not
            // actually more efficient; our call to poll_read will be as fast as
            // the PollWatcher's call to stat, and it actually saves a syscall
            // if the file has data available.
            //
            // https://github.com/notify-rs/notify/issues/240
            #[cfg(not(target_os = "linux"))]
            thread::spawn(move || {
                while let Ok(()) = notice_tx.send(()) {
                    thread::sleep(std::time::Duration::from_millis(100));
                }
            });

            #[cfg(target_os = "linux")]
            {
                let mut inotify = Inotify::init()
                    .with_context(|| format!("file source: failed to initialize inotify"))?;
                inotify
                    .add_watch(&_path, WatchMask::ALL_EVENTS)
                    .with_context(|| format!("failed to add watch for file {}", _path.display()))?;
                thread::spawn(move || {
                    // This buffer must be at least `sizeof(struct inotify_event) + NAME_MAX + 1`.
                    // The `inotify` crate documentation uses 1KB, so that's =
                    // what we do too.
                    let mut buf = [0; 1024];
                    loop {
                        match inotify.read_events_blocking(&mut buf) {
                            Err(err) => {
                                if notice_tx
                                    .send(Err(format!(
                                    "file source: failed to get events for file: {:#} (path: {})",
                                    err,
                                    _path.display()
                                )))
                                    .is_err()
                                {
                                    // If the notice_tx returns an error, it's because
                                    // the source has been dropped. Just exit the
                                    // thread.
                                    return;
                                }
                                // We have no method for recovering from this error
                                // Close this thread and log an error message (which duplicates the err above)
                                error!(
                                    "file source: closing stream due to read errors (path: {})",
                                    _path.display()
                                );
                                return;
                            }
                            Ok(mut events) => {
                                if events.any(|x| x.mask == EventMask::ATTRIB) && !_path.exists() {
                                    error!(
                                        "file source: closing stream due to deleted file (path: {})",
                                        _path.display()
                                    );
                                    return;
                                }
                            }
                        }
                        if notice_tx.send(Ok(())).is_err() {
                            // If the notice_tx returns an error, it's because
                            // the source has been dropped. Just exit the
                            // thread.
                            return;
                        }
                    }
                });
            };

            Ok(Box::new(ForeverTailedFile {
                rx: notice_rx,
                inner: file,
            }))
        }
    }
}

/// Strategies for streaming content from a file.
#[derive(PartialEq, Eq)]
pub enum FileReadStyle {
    /// File is read once and marked complete once the last line is read.
    ReadOnce,
    /// File is read and continually checked for new content, indefinitely.
    TailFollowFd,
    // TODO: TailFollowName,
}

/// Wraps a file, producing a stream that is tailed forever.
///
/// This involves silently swallowing EOFs,
/// and waiting on a Notify handle for more data to be written.
struct ForeverTailedFile<Ev> {
    rx: std::sync::mpsc::Receiver<Ev>,
    inner: std::fs::File,
}

impl<Ev> Skip for ForeverTailedFile<Ev> {
    fn skip(&mut self, len: usize) -> Result<(), io::Error> {
        self.inner.skip(len)
    }
}

impl<Ev> Read for ForeverTailedFile<Ev> {
    fn read(&mut self, buf: &mut [u8]) -> std::io::Result<usize> {
        loop {
            // First drain the buffer of pending events from notify.
            for _ in self.rx.try_iter() {}
            match self.inner.read(buf)? {
                0 => {
                    if self.rx.recv().is_ok() {
                        // Notify thinks there might be new data. Go around
                        // the loop again to check.
                    } else {
                        error!("notify hung up while tailing file");
                        return Ok(0);
                    }
                }
                n => {
                    return Ok(n);
                }
            }
        }
    }
}

/// Sends a sequence of records and activates a timely operator for each.
fn send_records<I, Out, Err>(
    iter: I,
    tx: std::sync::mpsc::SyncSender<Result<Out, anyhow::Error>>,
    activator: Option<SyncActivator>,
) where
    I: IntoIterator<Item = Result<Out, Err>>,
    Err: Into<anyhow::Error>,
{
    let mut records = 0;
    for record in iter {
        records += 1;
        let record = record.map_err(Into::into);
        // TODO: each call to `send` allocates and performs some
        // atomic work; we could aim to batch up transmissions.
        if tx.send(record).is_err() {
            // The receiver went away, probably due to `DROP SOURCE`
            break;
        }
        // TODO: this is very spammy for the timely activator; it
        // appends an address to a list for each activation which
        // looks like it will be per-record in this case.
        if let Some(activator) = &activator {
            activator.activate().expect("activation failed");
        }
    }
    tracing::trace!("sent {} records to reader", records);
}