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
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements.  See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership.  The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License.  You may obtain a copy of the License at
//
//   http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied.  See the License for the
// specific language governing permissions and limitations
// under the License.

//! Utility for streaming newline delimited files from object storage

use std::collections::VecDeque;

use bytes::Bytes;
use futures::{Stream, StreamExt};
use snafu::{ensure, Snafu};

use super::Result;

#[derive(Debug, Snafu)]
enum Error {
    #[snafu(display("encountered unterminated string"))]
    UnterminatedString,

    #[snafu(display("encountered trailing escape character"))]
    TrailingEscape,
}

impl From<Error> for super::Error {
    fn from(err: Error) -> Self {
        Self::Generic {
            store: "LineDelimiter",
            source: Box::new(err),
        }
    }
}

/// The ASCII encoding of `"`
const QUOTE: u8 = b'"';

/// The ASCII encoding of `\n`
const NEWLINE: u8 = b'\n';

/// The ASCII encoding of `\`
const ESCAPE: u8 = b'\\';

/// [`LineDelimiter`] is provided with a stream of [`Bytes`] and returns an iterator
/// of [`Bytes`] containing a whole number of new line delimited records
#[derive(Debug, Default)]
struct LineDelimiter {
    /// Complete chunks of [`Bytes`]
    complete: VecDeque<Bytes>,
    /// Remainder bytes that form the next record
    remainder: Vec<u8>,
    /// True if the last character was the escape character
    is_escape: bool,
    /// True if currently processing a quoted string
    is_quote: bool,
}

impl LineDelimiter {
    /// Creates a new [`LineDelimiter`] with the provided delimiter
    fn new() -> Self {
        Self::default()
    }

    /// Adds the next set of [`Bytes`]
    fn push(&mut self, val: impl Into<Bytes>) {
        let val: Bytes = val.into();

        let is_escape = &mut self.is_escape;
        let is_quote = &mut self.is_quote;
        let mut record_ends = val.iter().enumerate().filter_map(|(idx, v)| {
            if *is_escape {
                *is_escape = false;
                None
            } else if *v == ESCAPE {
                *is_escape = true;
                None
            } else if *v == QUOTE {
                *is_quote = !*is_quote;
                None
            } else if *is_quote {
                None
            } else {
                (*v == NEWLINE).then_some(idx + 1)
            }
        });

        let start_offset = match self.remainder.is_empty() {
            true => 0,
            false => match record_ends.next() {
                Some(idx) => {
                    self.remainder.extend_from_slice(&val[0..idx]);
                    self.complete
                        .push_back(Bytes::from(std::mem::take(&mut self.remainder)));
                    idx
                }
                None => {
                    self.remainder.extend_from_slice(&val);
                    return;
                }
            },
        };
        let end_offset = record_ends.last().unwrap_or(start_offset);
        if start_offset != end_offset {
            self.complete.push_back(val.slice(start_offset..end_offset));
        }

        if end_offset != val.len() {
            self.remainder.extend_from_slice(&val[end_offset..])
        }
    }

    /// Marks the end of the stream, delimiting any remaining bytes
    ///
    /// Returns `true` if there is no remaining data to be read
    fn finish(&mut self) -> Result<bool> {
        if !self.remainder.is_empty() {
            ensure!(!self.is_quote, UnterminatedStringSnafu);
            ensure!(!self.is_quote, TrailingEscapeSnafu);

            self.complete
                .push_back(Bytes::from(std::mem::take(&mut self.remainder)))
        }
        Ok(self.complete.is_empty())
    }
}

impl Iterator for LineDelimiter {
    type Item = Bytes;

    fn next(&mut self) -> Option<Self::Item> {
        self.complete.pop_front()
    }
}

/// Given a [`Stream`] of [`Bytes`] returns a [`Stream`] where each
/// yielded [`Bytes`] contains a whole number of new line delimited records
/// accounting for `\` style escapes and `"` quotes
pub fn newline_delimited_stream<S>(s: S) -> impl Stream<Item = Result<Bytes>>
where
    S: Stream<Item = Result<Bytes>> + Unpin,
{
    let delimiter = LineDelimiter::new();

    futures::stream::unfold(
        (s, delimiter, false),
        |(mut s, mut delimiter, mut exhausted)| async move {
            loop {
                if let Some(next) = delimiter.next() {
                    return Some((Ok(next), (s, delimiter, exhausted)));
                } else if exhausted {
                    return None;
                }

                match s.next().await {
                    Some(Ok(bytes)) => delimiter.push(bytes),
                    Some(Err(e)) => return Some((Err(e), (s, delimiter, exhausted))),
                    None => {
                        exhausted = true;
                        match delimiter.finish() {
                            Ok(true) => return None,
                            Ok(false) => continue,
                            Err(e) => return Some((Err(e), (s, delimiter, exhausted))),
                        }
                    }
                }
            }
        },
    )
}

#[cfg(test)]
mod tests {
    use futures::stream::{BoxStream, TryStreamExt};

    use super::*;

    #[test]
    fn test_delimiter() {
        let mut delimiter = LineDelimiter::new();
        delimiter.push("hello\nworld");
        delimiter.push("\n\n");

        assert_eq!(delimiter.next().unwrap(), Bytes::from("hello\n"));
        assert_eq!(delimiter.next().unwrap(), Bytes::from("world\n"));
        assert_eq!(delimiter.next().unwrap(), Bytes::from("\n"));
        assert!(delimiter.next().is_none());
    }

    #[test]
    fn test_delimiter_escaped() {
        let mut delimiter = LineDelimiter::new();
        delimiter.push("");
        delimiter.push("fo\\\n\"foo");
        delimiter.push("bo\n\"bar\n");
        delimiter.push("\"he");
        delimiter.push("llo\"\n");
        assert_eq!(
            delimiter.next().unwrap(),
            Bytes::from("fo\\\n\"foobo\n\"bar\n")
        );
        assert_eq!(delimiter.next().unwrap(), Bytes::from("\"hello\"\n"));
        assert!(delimiter.next().is_none());

        // Verify can push further data
        delimiter.push("\"foo\nbar\",\"fiz\\\"inner\\\"\"\nhello");
        assert!(!delimiter.finish().unwrap());

        assert_eq!(
            delimiter.next().unwrap(),
            Bytes::from("\"foo\nbar\",\"fiz\\\"inner\\\"\"\n")
        );
        assert_eq!(delimiter.next().unwrap(), Bytes::from("hello"));
        assert!(delimiter.finish().unwrap());
        assert!(delimiter.next().is_none());
    }

    #[tokio::test]
    async fn test_delimiter_stream() {
        let input = vec!["hello\nworld\nbin", "go\ncup", "cakes"];
        let input_stream = futures::stream::iter(input.into_iter().map(|s| Ok(Bytes::from(s))));
        let stream = newline_delimited_stream(input_stream);

        let results: Vec<_> = stream.try_collect().await.unwrap();
        assert_eq!(
            results,
            vec![
                Bytes::from("hello\nworld\n"),
                Bytes::from("bingo\n"),
                Bytes::from("cupcakes")
            ]
        )
    }
    #[tokio::test]
    async fn test_delimiter_unfold_stream() {
        let input_stream: BoxStream<'static, Result<Bytes>> = futures::stream::unfold(
            VecDeque::from(["hello\nworld\nbin", "go\ncup", "cakes"]),
            |mut input| async move {
                if !input.is_empty() {
                    Some((Ok(Bytes::from(input.pop_front().unwrap())), input))
                } else {
                    None
                }
            },
        )
        .boxed();
        let stream = newline_delimited_stream(input_stream);

        let results: Vec<_> = stream.try_collect().await.unwrap();
        assert_eq!(
            results,
            vec![
                Bytes::from("hello\nworld\n"),
                Bytes::from("bingo\n"),
                Bytes::from("cupcakes")
            ]
        )
    }
}