faucet-source-s3 1.0.0

AWS S3 source connector for the faucet-stream ecosystem
Documentation
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
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
//! S3 source stream executor.

use crate::config::{S3FileFormat, S3SourceConfig};
use async_trait::async_trait;
use aws_sdk_s3::Client;
use faucet_core::{FaucetError, Stream, StreamPage};
use futures::stream::{self, StreamExt, TryStreamExt};
use serde_json::Value;
use std::pin::Pin;
use tokio::io::AsyncBufReadExt;

/// An S3 source that lists and reads objects from a bucket.
pub struct S3Source {
    config: S3SourceConfig,
    client: Client,
}

impl S3Source {
    /// Create a new S3 source from the given configuration.
    ///
    /// Builds the S3 client eagerly so it is reused across calls.
    pub async fn new(config: S3SourceConfig) -> Result<Self, FaucetError> {
        let client = Self::build_client(&config).await?;
        Ok(Self { config, client })
    }

    /// Build an S3 client from the configuration.
    async fn build_client(config: &S3SourceConfig) -> Result<Client, FaucetError> {
        let mut config_loader = aws_config::defaults(aws_config::BehaviorVersion::latest());

        if let Some(ref region) = config.region {
            config_loader = config_loader.region(aws_config::Region::new(region.clone()));
        }

        if let Some(ref endpoint) = config.endpoint_url {
            config_loader = config_loader.endpoint_url(endpoint);
        }

        let sdk_config = config_loader.load().await;
        let client = Client::new(&sdk_config);
        Ok(client)
    }

    /// List object keys matching the configured bucket and prefix.
    ///
    /// When `prefix_override` is `Some`, it is used instead of `self.config.prefix`
    /// (used for parent-context substitution).
    async fn list_object_keys(
        &self,
        prefix_override: Option<&str>,
    ) -> Result<Vec<String>, FaucetError> {
        let mut keys = Vec::new();
        let mut continuation_token: Option<String> = None;

        let effective_prefix = prefix_override.or(self.config.prefix.as_deref());

        loop {
            let mut req = self.client.list_objects_v2().bucket(&self.config.bucket);

            if let Some(prefix) = effective_prefix {
                req = req.prefix(prefix);
            }

            if let Some(ref token) = continuation_token {
                req = req.continuation_token(token);
            }

            let response = req.send().await.map_err(|e| {
                FaucetError::Source(format!(
                    "S3 list objects error for bucket '{}': {e}",
                    self.config.bucket
                ))
            })?;

            for object in response.contents() {
                let key: &str = object.key().unwrap_or_default();
                if key.is_empty() {
                    continue;
                }
                keys.push(key.to_string());

                if let Some(max) = self.config.max_objects
                    && keys.len() >= max
                {
                    return Ok(keys);
                }
            }

            if response.is_truncated() == Some(true) {
                continuation_token = response.next_continuation_token().map(String::from);
            } else {
                break;
            }
        }

        Ok(keys)
    }

    /// Read and parse a single S3 object into records.
    async fn read_object(&self, key: &str) -> Result<Vec<Value>, FaucetError> {
        let text = self.read_object_text(key).await?;
        self.parse_content(key, &text)
    }

    /// Read the full body of a single S3 object into a UTF-8 `String`.
    ///
    /// Streams the (optionally decompressed) body straight into one `String`
    /// via [`open_object_reader`](Self::open_object_reader) rather than
    /// buffering the raw bytes AND the decompressed bytes AND the `String`
    /// at once (#78/#25). The whole object is still one unit for
    /// `JsonArray` / `RawText`, but peak memory is now ~1× the decoded size.
    async fn read_object_text(&self, key: &str) -> Result<String, FaucetError> {
        use tokio::io::AsyncReadExt as _;
        let mut reader = self.open_object_reader(key).await?;
        let mut text = String::new();
        reader.read_to_string(&mut text).await.map_err(|e| {
            FaucetError::Source(format!(
                "S3 read/decode error for key '{key}' (not valid UTF-8?): {e}"
            ))
        })?;
        Ok(text)
    }

    /// Open an S3 object as an [`AsyncBufRead`](tokio::io::AsyncBufRead) over
    /// its body. Used by [`Source::stream_pages`](faucet_core::Source::stream_pages)
    /// to decode `JsonLines` objects line-by-line without buffering the
    /// whole file.
    async fn open_object_reader(
        &self,
        key: &str,
    ) -> Result<std::pin::Pin<Box<dyn tokio::io::AsyncBufRead + Send + Unpin>>, FaucetError> {
        let response = self
            .client
            .get_object()
            .bucket(&self.config.bucket)
            .key(key)
            .send()
            .await
            .map_err(|e| {
                FaucetError::Source(format!("S3 get object error for key '{key}': {e}"))
            })?;

        // `ByteStream::into_async_read` returns `impl AsyncRead`; wrap in a
        // `BufReader` so `.lines()` is usable and ownership is `Unpin`.
        let buffered = tokio::io::BufReader::new(response.body.into_async_read());
        #[cfg(feature = "compression")]
        {
            let codec = self.config.compression.resolve(key);
            faucet_core::compression::warn_mismatch(key, codec);
            Ok(faucet_core::compression::wrap_async_reader(buffered, codec))
        }
        #[cfg(not(feature = "compression"))]
        {
            Ok(Box::pin(buffered))
        }
    }

    /// Parse file content into records based on the configured file format.
    fn parse_content(&self, key: &str, text: &str) -> Result<Vec<Value>, FaucetError> {
        match self.config.file_format {
            S3FileFormat::JsonLines => {
                let mut records = Vec::new();
                for (line_num, line) in text.lines().enumerate() {
                    let trimmed = line.trim();
                    if trimmed.is_empty() {
                        continue;
                    }
                    let value: Value = serde_json::from_str(trimmed).map_err(|e| {
                        FaucetError::Source(format!(
                            "S3 JSON parse error in '{key}' at line {}: {e}",
                            line_num + 1
                        ))
                    })?;
                    records.push(value);
                }
                Ok(records)
            }
            S3FileFormat::JsonArray => {
                let value: Value = serde_json::from_str(text).map_err(|e| {
                    FaucetError::Source(format!("S3 JSON parse error in '{key}': {e}"))
                })?;
                match value {
                    Value::Array(arr) => Ok(arr),
                    _ => Err(FaucetError::Source(format!(
                        "S3 expected JSON array in '{key}', got {}",
                        value_type_name(&value)
                    ))),
                }
            }
            S3FileFormat::RawText => {
                let record = serde_json::json!({
                    "key": key,
                    "content": text,
                });
                Ok(vec![record])
            }
        }
    }
}

#[async_trait]
impl faucet_core::Source for S3Source {
    async fn fetch_with_context(
        &self,
        context: &std::collections::HashMap<String, serde_json::Value>,
    ) -> Result<Vec<Value>, FaucetError> {
        // Substitute context into prefix when parent context is provided.
        let substituted_prefix: Option<String> = if !context.is_empty() {
            self.config
                .prefix
                .as_ref()
                .map(|p| faucet_core::util::substitute_context(p, context))
        } else {
            None
        };

        let keys = self.list_object_keys(substituted_prefix.as_deref()).await?;

        tracing::info!(
            bucket = %self.config.bucket,
            objects = keys.len(),
            "Listed S3 objects"
        );

        let concurrency = self.config.concurrency.max(1);

        let results: Vec<Vec<Value>> = stream::iter(keys)
            .map(|key| async move {
                let records = self.read_object(&key).await?;
                tracing::debug!(key = %key, records = records.len(), "Read S3 object");
                Ok::<Vec<Value>, FaucetError>(records)
            })
            .buffer_unordered(concurrency)
            .try_collect()
            .await?;

        let all_records: Vec<Value> = results.into_iter().flatten().collect();

        tracing::info!(total_records = all_records.len(), "S3 fetch complete");
        Ok(all_records)
    }

    /// Stream records from listed S3 objects without buffering the full
    /// scan. Each emitted [`StreamPage`] holds up to
    /// [`S3SourceConfig::batch_size`] records.
    ///
    /// The trait-level `batch_size` argument is ignored in favour of the
    /// config field — the config is the user-facing knob the README
    /// documents, and routing the pipeline-supplied hint through it would
    /// silently override an explicit config value.
    ///
    /// Behaviour by format:
    ///
    /// - `JsonLines` / `RawText`: the object body is decoded line-by-line
    ///   via [`tokio::io::AsyncBufReadExt::lines`] so client-side memory is
    ///   bounded at `O(batch_size)` per object. Multi-object scans are
    ///   flattened — a single page may carry lines drawn from any object.
    /// - `JsonArray`: each object is buffered fully (the JSON value can
    ///   only be parsed once the array is complete) and then its records
    ///   are chunked into pages of `batch_size`. See the README "Streaming
    ///   and batching" section for the caveat.
    ///
    /// `batch_size = 0` is the "no batching" sentinel: one [`StreamPage`]
    /// is emitted per S3 object (no within-object chunking and no
    /// cross-object accumulation). The S3 source has no
    /// incremental-replication mode today, so every emitted page carries
    /// `bookmark: None`.
    fn stream_pages<'a>(
        &'a self,
        context: &'a std::collections::HashMap<String, Value>,
        _batch_size: usize,
    ) -> Pin<Box<dyn Stream<Item = Result<StreamPage, FaucetError>> + Send + 'a>> {
        let batch_size = self.config.batch_size;

        Box::pin(async_stream::try_stream! {
            // Substitute context into prefix when parent context is provided.
            let substituted_prefix: Option<String> = if !context.is_empty() {
                self.config
                    .prefix
                    .as_ref()
                    .map(|p| faucet_core::util::substitute_context(p, context))
            } else {
                None
            };

            let keys = self.list_object_keys(substituted_prefix.as_deref()).await?;
            tracing::info!(
                bucket = %self.config.bucket,
                objects = keys.len(),
                "Listed S3 objects (stream)",
            );

            let chunk = if batch_size == 0 { usize::MAX } else { batch_size };
            let initial_capacity = if batch_size == 0 { 1024 } else { batch_size };
            let mut buffer: Vec<Value> = Vec::with_capacity(initial_capacity);
            let mut total = 0usize;

            for key in &keys {
                match self.config.file_format {
                    S3FileFormat::JsonLines => {
                        let reader = self.open_object_reader(key).await?;
                        let mut lines = reader.lines();
                        let mut line_num: usize = 0;
                        while let Some(line) = lines
                            .next_line()
                            .await
                            .map_err(|e| FaucetError::Source(format!(
                                "S3 read body error for key '{key}': {e}"
                            )))?
                        {
                            line_num += 1;
                            let trimmed = line.trim();
                            if trimmed.is_empty() {
                                continue;
                            }
                            let value: Value =
                                serde_json::from_str(trimmed).map_err(|e| {
                                    FaucetError::Source(format!(
                                        "S3 JSON parse error in '{key}' at line {line_num}: {e}",
                                    ))
                                })?;
                            buffer.push(value);
                            if batch_size != 0 && buffer.len() >= chunk {
                                let page = std::mem::replace(
                                    &mut buffer,
                                    Vec::with_capacity(initial_capacity),
                                );
                                total += page.len();
                                yield StreamPage { records: page, bookmark: None };
                            }
                        }
                        if batch_size == 0 && !buffer.is_empty() {
                            let page = std::mem::take(&mut buffer);
                            total += page.len();
                            yield StreamPage { records: page, bookmark: None };
                        }
                    }
                    S3FileFormat::RawText => {
                        // RawText emits a single record per object; the
                        // `key` + `content` shape is unchanged so we
                        // continue to buffer the body fully. This still
                        // streams *across* objects.
                        let text = self.read_object_text(key).await?;
                        let record = serde_json::json!({
                            "key": key,
                            "content": text,
                        });
                        buffer.push(record);
                        if batch_size == 0 {
                            let page = std::mem::take(&mut buffer);
                            total += page.len();
                            yield StreamPage { records: page, bookmark: None };
                        } else if buffer.len() >= chunk {
                            let page = std::mem::replace(
                                &mut buffer,
                                Vec::with_capacity(initial_capacity),
                            );
                            total += page.len();
                            yield StreamPage { records: page, bookmark: None };
                        }
                    }
                    S3FileFormat::JsonArray => {
                        // JSON-array files cannot be parsed incrementally
                        // (the closing `]` is required to validate the
                        // structure), so each object is buffered fully and
                        // then chunked. The caveat is documented in the
                        // crate README.
                        let text = self.read_object_text(key).await?;
                        let value: Value = serde_json::from_str(&text).map_err(|e| {
                            FaucetError::Source(format!("S3 JSON parse error in '{key}': {e}"))
                        })?;
                        let array = match value {
                            Value::Array(arr) => arr,
                            other => Err(FaucetError::Source(format!(
                                "S3 expected JSON array in '{key}', got {}",
                                value_type_name(&other)
                            )))?,
                        };
                        if batch_size == 0 {
                            // Flush any cross-object buffer first (none
                            // here because each iteration completes its
                            // own object — but keep symmetric with the
                            // line-shaped branches).
                            if !buffer.is_empty() {
                                let page = std::mem::take(&mut buffer);
                                total += page.len();
                                yield StreamPage { records: page, bookmark: None };
                            }
                            total += array.len();
                            yield StreamPage { records: array, bookmark: None };
                        } else {
                            for record in array {
                                buffer.push(record);
                                if buffer.len() >= chunk {
                                    let page = std::mem::replace(
                                        &mut buffer,
                                        Vec::with_capacity(initial_capacity),
                                    );
                                    total += page.len();
                                    yield StreamPage { records: page, bookmark: None };
                                }
                            }
                        }
                    }
                }
            }

            if !buffer.is_empty() {
                let page = std::mem::take(&mut buffer);
                total += page.len();
                yield StreamPage { records: page, bookmark: None };
            }

            tracing::info!(
                total_records = total,
                batch_size,
                objects = keys.len(),
                "S3 source stream complete",
            );
        })
    }

    fn config_schema(&self) -> serde_json::Value {
        serde_json::to_value(faucet_core::schema_for!(S3SourceConfig))
            .expect("schema serialization")
    }
}

/// Return a human-readable name for a JSON value type.
fn value_type_name(v: &Value) -> &'static str {
    match v {
        Value::Null => "null",
        Value::Bool(_) => "boolean",
        Value::Number(_) => "number",
        Value::String(_) => "string",
        Value::Array(_) => "array",
        Value::Object(_) => "object",
    }
}

#[cfg(test)]
mod tests {
    use super::*;
    use crate::config::S3SourceConfig;
    use serde_json::json;

    /// Helper to build an S3Source synchronously for parse-only tests.
    /// We construct it directly to avoid needing an async runtime for unit tests
    /// that only exercise `parse_content`.
    fn test_source(config: S3SourceConfig) -> S3Source {
        // Build a dummy client — these tests never make network calls.
        let sdk_config = aws_config::SdkConfig::builder()
            .behavior_version(aws_config::BehaviorVersion::latest())
            .build();
        let client = Client::new(&sdk_config);
        S3Source { config, client }
    }

    #[test]
    fn parse_json_lines() {
        let source = test_source(S3SourceConfig::new("test"));
        let text = r#"{"id":1,"name":"Alice"}
{"id":2,"name":"Bob"}
"#;
        let records = source.parse_content("test.jsonl", text).unwrap();
        assert_eq!(records.len(), 2);
        assert_eq!(records[0]["id"], 1);
        assert_eq!(records[1]["name"], "Bob");
    }

    #[test]
    fn parse_json_lines_skips_empty() {
        let source = test_source(S3SourceConfig::new("test"));
        let text = r#"{"id":1}

{"id":2}

"#;
        let records = source.parse_content("test.jsonl", text).unwrap();
        assert_eq!(records.len(), 2);
    }

    #[test]
    fn parse_json_lines_invalid() {
        let source = test_source(S3SourceConfig::new("test"));
        let text = "not json\n";
        let result = source.parse_content("test.jsonl", text);
        assert!(result.is_err());
        let err = result.unwrap_err().to_string();
        assert!(err.contains("JSON parse error"));
        assert!(err.contains("line 1"));
    }

    #[test]
    fn parse_json_array() {
        let source = test_source(S3SourceConfig::new("test").file_format(S3FileFormat::JsonArray));
        let text = r#"[{"id":1},{"id":2}]"#;
        let records = source.parse_content("test.json", text).unwrap();
        assert_eq!(records.len(), 2);
        assert_eq!(records[0]["id"], 1);
    }

    #[test]
    fn parse_json_array_not_array() {
        let source = test_source(S3SourceConfig::new("test").file_format(S3FileFormat::JsonArray));
        let text = r#"{"id":1}"#;
        let result = source.parse_content("test.json", text);
        assert!(result.is_err());
        let err = result.unwrap_err().to_string();
        assert!(err.contains("expected JSON array"));
    }

    #[test]
    fn parse_raw_text() {
        let source = test_source(S3SourceConfig::new("test").file_format(S3FileFormat::RawText));
        let text = "hello world\nline two";
        let records = source.parse_content("data/file.txt", text).unwrap();
        assert_eq!(records.len(), 1);
        assert_eq!(
            records[0],
            json!({"key": "data/file.txt", "content": "hello world\nline two"})
        );
    }

    #[cfg(feature = "compression")]
    #[test]
    fn compression_default_is_auto() {
        let cfg = S3SourceConfig::new("bucket");
        assert_eq!(cfg.compression, faucet_core::CompressionConfig::Auto);
    }
}