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
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
1001
1002
1003
1004
1005
1006
1007
1008
1009
1010
1011
1012
1013
1014
1015
1016
1017
1018
1019
1020
1021
1022
1023
1024
1025
1026
1027
1028
1029
1030
1031
1032
1033
1034
1035
1036
1037
1038
1039
1040
1041
1042
1043
1044
1045
1046
1047
1048
1049
1050
1051
// 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.

//! Functionality for creating S3 sources
//!
//! This source is constructed as a collection of Tokio tasks that communicate over local
//! (worker-pinned) queues to send data into dataflow. We spin up a single "downloader" task which
//! is responsible for performing s3 object downloads and shuffling the data into dataflow. The
//! downloader will currently download the entirety of each file and emit a degenerate `ByteStream`
//! into the dataflow containing a single chunk. This will have to be improved before we release S3
//! sources to production. Then, for each object source, we spin up another task which is
//! responsible for collecting object names from an object name source and sending that name to the
//! downloader.
//!
//! ```text
//! +----------------+
//! | bucket scanner +-                               -------
//! +----------------+ \---                         -/       \-
//! +----------------+     \--   +------------+    /           \
//! | sqs listener   +--------X->| downloader +--->| dataflow  |
//! +----------------+     /--   +------------+    \           /
//!        .  .  .  .   /--                         -\       /-
//!       etc .  .  . --                              -------
//!        .  .  .  .
//! ```

use std::collections::{BTreeMap, BTreeSet};
use std::convert::{From, TryInto};
use std::default::Default;
use std::ops::AddAssign;
use std::sync::Arc;

use async_compression::tokio::bufread::GzipDecoder;
use aws_sdk_s3::error::{GetObjectError, ListObjectsV2Error};
use aws_sdk_s3::types::SdkError;
use aws_sdk_s3::Client as S3Client;
use aws_sdk_sqs::model::{ChangeMessageVisibilityBatchRequestEntry, Message as SqsMessage};
use aws_sdk_sqs::Client as SqsClient;
use futures::{FutureExt, TryStreamExt};
use globset::GlobMatcher;
use timely::dataflow::operators::Capability;
use timely::progress::Antichain;
use timely::scheduling::SyncActivator;
use tokio::io::{AsyncBufReadExt, AsyncRead, AsyncReadExt, BufReader};
use tokio::sync::mpsc::{Receiver, Sender};
use tokio::time::{self, Duration};
use tokio_util::io::StreamReader;
use tracing::{debug, error, trace, warn};

use mz_cloud_resources::AwsExternalIdPrefix;
use mz_ore::retry::{Retry, RetryReader};
use mz_ore::task;
use mz_repr::GlobalId;
use mz_secrets::SecretsReader;
use mz_storage_client::types::connections::aws::AwsConfig;
use mz_storage_client::types::connections::ConnectionContext;
use mz_storage_client::types::sources::encoding::SourceDataEncoding;
use mz_storage_client::types::sources::{Compression, MzOffset, S3KeySource, S3SourceConnection};

use self::metrics::{BucketMetrics, ScanBucketMetrics};
use self::notifications::{Event, EventType, TestEvent};
use crate::source::commit::LogCommitter;
use crate::source::types::{ByteStream, SourceConnectionBuilder};
use crate::source::{
    NextMessage, SourceMessage, SourceMessageType, SourceReader, SourceReaderError,
};

use super::metrics::SourceBaseMetrics;

mod metrics;
mod notifications;

/// Information required to load data from S3
pub struct S3SourceReader {
    /// The name of the source that the user entered
    source_name: String,

    // differential control
    /// Global source ID
    id: GlobalId,
    /// Receiver channel that ingests records
    receiver_stream: Receiver<S3Result<Vec<u8>>>,
    dataflow_status: tokio::sync::watch::Sender<DataflowStatus>,
    /// Total number of records that this source has read
    offset: S3Offset,

    /// Capabilities used to produce messages
    data_capability: Capability<MzOffset>,
    upper_capability: Capability<MzOffset>,

    // S3 sources support single-threaded ingestion only, so only one of the
    // `S3SourceReader`s will actually produce data.
    active_read_worker: bool,
}

/// Current dataflow status
///
/// Used to signal the S3 and SQS services to shut down
#[derive(Debug, Clone, Copy, Eq, PartialEq)]
enum DataflowStatus {
    Running,
    Stopped,
}

/// Number of records This source has downloaded
///
/// Possibly this should be per-bucket or per-object, depending on the needs
/// for deterministic timestamping on restarts: issue #5715
#[derive(Clone, Copy, Debug)]
struct S3Offset(u64);

impl AddAssign<u64> for S3Offset {
    fn add_assign(&mut self, other: u64) {
        self.0 += other;
    }
}

impl From<S3Offset> for MzOffset {
    fn from(offset: S3Offset) -> MzOffset {
        MzOffset { offset: offset.0 }
    }
}

struct KeyInfo {
    bucket: String,
    key: String,
}

async fn download_objects_task(
    source_id: GlobalId,
    mut rx: Receiver<S3Result<KeyInfo>>,
    tx: Sender<S3Result<Vec<u8>>>,
    mut shutdown_rx: tokio::sync::watch::Receiver<DataflowStatus>,
    aws_config: AwsConfig,
    aws_external_id_prefix: Option<AwsExternalIdPrefix>,
    activator: SyncActivator,
    compression: Compression,
    metrics: SourceBaseMetrics,
    secrets_reader: Arc<dyn SecretsReader>,
) {
    let config = aws_config
        .load(
            aws_external_id_prefix.as_ref(),
            Some(&source_id),
            &*secrets_reader,
        )
        .await;
    let client = mz_aws_s3_util::new_client(&config);

    let source_id = source_id.to_string();

    struct BucketInfo {
        keys: BTreeSet<String>,
        metrics: BucketMetrics,
    }
    let mut seen_buckets: BTreeMap<String, BucketInfo> = BTreeMap::new();

    loop {
        let msg = tokio::select! {
            msg = rx.recv() => {
                if let Some(msg) = msg {
                    msg
                } else {
                    break;
                }
            }
            status = shutdown_rx.changed() => {
                if status.is_ok() {
                    if let DataflowStatus::Stopped = *shutdown_rx.borrow() {
                        debug!("source_id={} download_objects received dataflow shutdown message", source_id);
                        break;
                    }
                }
                continue;
            }
        };

        match msg {
            Ok(msg) => {
                if let Some(bi) = seen_buckets.get_mut(&msg.bucket) {
                    if bi.keys.contains(&msg.key) {
                        bi.metrics.objects_duplicate.inc();
                        debug!(
                            "source_id={} skipping object because it was already seen: {}/{}",
                            source_id, msg.bucket, msg.key
                        );
                        continue;
                    }
                } else {
                    let bi = BucketInfo {
                        keys: BTreeSet::new(),
                        metrics: BucketMetrics::new(&metrics, &source_id, &msg.bucket),
                    };
                    seen_buckets.insert(msg.bucket.clone(), bi);
                };

                let (tx, activator, client, msg_ref, sid) =
                    (&tx, &activator, &client, &msg, &source_id);

                let download_result = download_object(
                    tx,
                    activator,
                    client,
                    &msg_ref.bucket,
                    &msg_ref.key,
                    compression,
                    sid,
                )
                .await;

                // Extract and handle status updates
                match download_result {
                    Ok(update) => {
                        let bucket_info = seen_buckets.get_mut(&msg.bucket).expect("just inserted");
                        bucket_info.metrics.inc(1, update.bytes, update.messages);
                        debug!(
                            "source_id={} successfully downloaded {}/{}",
                            source_id, msg.bucket, msg.key
                        );
                        bucket_info.keys.insert(msg.key);
                    }
                    Err(DownloadError::Failed { err }) => {
                        if tx
                            .send(Err(S3Error::IoError {
                                bucket: msg_ref.bucket.clone(),
                                err,
                            }))
                            .await
                            .is_err()
                        {
                            rx.close();
                            break;
                        };
                    }
                    Err(DownloadError::SendFailed) => {
                        rx.close();
                        break;
                    }
                };
            }
            Err(e) => {
                if tx.send(Err(e)).await.is_err() {
                    rx.close();
                    break;
                }
            }
        }
    }
    debug!("source_id={} exiting download objects task", source_id);
}

async fn scan_bucket_task(
    bucket: String,
    source_id: GlobalId,
    glob: Option<GlobMatcher>,
    aws_config: AwsConfig,
    aws_external_id_prefix: Option<AwsExternalIdPrefix>,
    tx: Sender<S3Result<KeyInfo>>,
    base_metrics: SourceBaseMetrics,
    secrets_reader: Arc<dyn SecretsReader>,
) {
    let config = aws_config
        .load(
            aws_external_id_prefix.as_ref(),
            Some(&source_id),
            &*secrets_reader,
        )
        .await;
    let client = mz_aws_s3_util::new_client(&config);

    let source_id = source_id.to_string();

    let glob = glob.as_ref();
    let prefix = glob.map(|g| find_prefix(g.glob().glob()));

    // for the special case of a single object in a matching clause, don't go through the ListObject
    // dance.
    //
    // This isn't a meaningful performance optimization, it just makes it easy for folks to import a
    // single object without granting Materialize the ListObjects IAM permission
    let is_literal_object = glob.is_some() && prefix.as_deref() == glob.map(|g| g.glob().glob());
    if is_literal_object {
        let key = glob.unwrap().glob().glob();
        debug!(
            "source_id={} downloading single object from s3 bucket={} key={}",
            source_id, bucket, key
        );
        if let Err(e) = tx
            .send(Ok(KeyInfo {
                bucket,
                key: key.to_string(),
            }))
            .await
        {
            debug!(
                "source_id={} Unable to send single key to downloader: {}",
                source_id, e
            );
        };

        return;
    } else {
        debug!(
            "source_id={} scanning bucket to find objects to download bucket={}",
            source_id, bucket
        );
    }

    let scan_metrics = ScanBucketMetrics::new(&base_metrics, &source_id, &bucket);

    let mut continuation_token = None;
    loop {
        let response = Retry::default()
            .max_duration(Duration::from_secs(30))
            .retry_async(|_| {
                client
                    .list_objects_v2()
                    .bucket(&bucket)
                    .set_prefix(prefix.clone())
                    .set_continuation_token(continuation_token.clone())
                    .send()
            })
            .await;

        match response {
            Ok(response) => {
                if let Some(c) = response.contents {
                    let keys = c
                        .into_iter()
                        .filter_map(|obj| obj.key)
                        .filter(|k| glob.map(|g| g.is_match(k)).unwrap_or(true));

                    for key in keys {
                        let res = tx
                            .send(Ok(KeyInfo {
                                bucket: bucket.clone(),
                                key,
                            }))
                            .await;

                        match res {
                            Ok(_) => scan_metrics.objects_discovered.inc(),
                            Err(e) => {
                                debug!("unable to send keys to downloader: {}", e);
                                break;
                            }
                        }
                    }
                }

                if response.next_continuation_token.is_none() {
                    break;
                }
                continuation_token = response.next_continuation_token;
            }
            Err(err) => {
                tx.send(Err(S3Error::ListObjectsFailed {
                    bucket: bucket.clone(),
                    err,
                }))
                .await
                .unwrap_or_else(|e| debug!("Source queue has been shut down: {}", e));

                break;
            }
        }
    }
    debug!(
        "source_id={} exiting bucket scan task bucket={}",
        source_id, bucket
    );
}

async fn read_sqs_task(
    source_id: GlobalId,
    glob: Option<GlobMatcher>,
    queue: String,
    aws_config: AwsConfig,
    aws_external_id_prefix: Option<AwsExternalIdPrefix>,
    tx: Sender<S3Result<KeyInfo>>,
    mut shutdown_rx: tokio::sync::watch::Receiver<DataflowStatus>,
    base_metrics: SourceBaseMetrics,
    secrets_reader: Arc<dyn SecretsReader>,
) {
    debug!(
        "source_id={} starting read sqs task queue={}",
        source_id, queue,
    );

    let config = aws_config
        .load(
            aws_external_id_prefix.as_ref(),
            Some(&source_id),
            &*secrets_reader,
        )
        .await;
    let client = aws_sdk_sqs::Client::new(&config);

    let source_id = source_id.to_string();

    let glob = glob.as_ref();

    // TODO: accept a full url
    let queue_url = match client.get_queue_url().queue_name(&queue).send().await {
        Ok(response) => {
            if let Some(url) = response.queue_url {
                url
            } else {
                error!("Empty queue url response for queue {}", queue);
                return;
            }
        }
        Err(e) => {
            error!("Unable to retrieve queue url for queue {}: {}", queue, e);
            return;
        }
    };

    let mut metrics: BTreeMap<String, ScanBucketMetrics> = BTreeMap::new();

    let mut allowed_errors = 10;
    'outer: loop {
        let sqs_fut = client
            .receive_message()
            .max_number_of_messages(10)
            .queue_url(&queue_url)
            .visibility_timeout(500)
            // the maximum possible time for a long poll
            .wait_time_seconds(20)
            .send();
        let response = tokio::select! {
            response = sqs_fut => response,
            status = shutdown_rx.changed() => {
                if status.is_ok() {
                    if let DataflowStatus::Stopped = *shutdown_rx.borrow() {
                        debug!("source_id={} scan_sqs received dataflow shutdown message", source_id);
                        break;
                    }
                }
                continue;
            }
        };

        match response {
            Ok(response) => {
                let messages = if let Some(m) = response.messages {
                    if tx.is_closed() {
                        release_messages(
                            &client,
                            None,
                            m.into_iter(),
                            queue_url.clone(),
                            &source_id,
                            None,
                        )
                        .await;
                        break;
                    }

                    m
                } else {
                    if tx.is_closed() {
                        break;
                    }
                    continue;
                };
                allowed_errors = 10;

                let mut msgs_iter = messages.into_iter();
                while let Some(message) = msgs_iter.next() {
                    let canceled = process_message(
                        message,
                        glob,
                        base_metrics.clone(),
                        &mut metrics,
                        &source_id,
                        &tx,
                        &client,
                        &queue_url,
                    )
                    .await;
                    if let Some((canceled_message, key)) = canceled {
                        release_messages(
                            &client,
                            Some(canceled_message),
                            msgs_iter,
                            queue_url.clone(),
                            &source_id,
                            Some(key),
                        )
                        .await;
                        break 'outer;
                    }
                }
            }

            Err(e) => {
                allowed_errors -= 1;
                if allowed_errors == 0 {
                    error!("failed to read from SQS queue {}: {}", queue, e);
                    break;
                } else {
                    warn!(
                        "unable to read from SQS queue {}: {} ({} retries remaining)",
                        queue, e, allowed_errors
                    );
                }

                time::sleep(Duration::from_secs(1)).await;
            }
        }
    }
    debug!("source_id={} exiting sqs reader queue={}", source_id, queue);
}

/// Send the relevant parts of the message to the download objects task
///
/// Returns any message that wasn't able to be processed to be released back to
/// the SQS service, as well as the specific key that we failed to process from
/// that message.
async fn process_message(
    message: SqsMessage,
    glob: Option<&GlobMatcher>,
    base_metrics: SourceBaseMetrics,
    metrics: &mut BTreeMap<String, ScanBucketMetrics>,
    source_id: &str,
    tx: &Sender<S3Result<KeyInfo>>,
    client: &SqsClient,
    queue_url: &str,
) -> Option<(SqsMessage, String)> {
    if let Some(body) = message.body.as_ref() {
        let event: Result<Event, _> = serde_json::from_str(body);
        match event {
            Ok(event) => {
                if event.records.is_empty() {
                    debug!(
                        "source_id={} sqs event is surprisingly empty {:#?}",
                        source_id, event
                    );
                }

                for record in event.records {
                    trace!(
                        "source_id={} processing message from sqs for key={} type={:?}",
                        source_id,
                        record.s3.object.key,
                        record.event_type
                    );

                    if matches!(
                        record.event_type,
                        EventType::ObjectCreatedPut
                            | EventType::ObjectCreatedPost
                            | EventType::ObjectCreatedCompleteMultipartUpload
                    ) {
                        let key = record.s3.object.key;
                        if glob.map(|g| g.is_match(&key)).unwrap_or(true) {
                            if let Some(m) = metrics.get(&record.s3.bucket.name) {
                                m.objects_discovered.inc()
                            } else {
                                let m = ScanBucketMetrics::new(
                                    &base_metrics,
                                    source_id,
                                    &record.s3.bucket.name,
                                );
                                m.objects_discovered.inc();
                                metrics.insert(record.s3.bucket.name.clone(), m);
                            }

                            let ki = Ok(KeyInfo {
                                bucket: record.s3.bucket.name,
                                key: key.clone(),
                            });
                            if tx.send(ki).await.is_err() {
                                debug!(
                                    "source_id={} sqs reader is closed, marking message as visible",
                                    source_id
                                );
                                return Some((message, key));
                            }
                        }
                    }
                }
            }
            Err(_) => {
                let test: Result<TestEvent, _> = serde_json::from_str(body);
                match test {
                    Ok(_) => {
                        trace!("got test event for new queue");
                    }
                    Err(_) => {
                        error!(
                            "[customer-data] Unrecognized message from SQS queue {}: {}",
                            queue_url, body,
                        )
                    }
                }
            }
        }
    }

    if let Err(e) = client
        .delete_message()
        .queue_url(queue_url)
        .receipt_handle(
            message
                .receipt_handle
                .expect("receipt handle is always returned"),
        )
        .send()
        .await
    {
        warn!(
            "source_id={} Error deleting processed SQS message: {}",
            source_id, e
        )
    }

    None
}

#[derive(Debug, Default)]
struct DownloadMetricUpdate {
    bytes: u64,
    messages: u64,
}

#[derive(Debug)]
enum DownloadError {
    Failed {
        err: std::io::Error,
    },
    /// Unable to send data to the `get_next_message` function, dataflow has shut down
    SendFailed,
}

#[derive(Debug, thiserror::Error)]
enum S3Error {
    #[error("Unable to get S3 object {}/{}: {}", bucket, key, err)]
    GetObjectError {
        bucket: String,
        key: String,
        #[source]
        err: SdkError<GetObjectError>,
    },
    #[error("Unable to list S3 object {}: {}", bucket, err)]
    ListObjectsFailed {
        bucket: String,
        #[source]
        err: SdkError<ListObjectsV2Error>,
    },
    #[error("IO Error for S3 bucket {}: {}", bucket, err)]
    IoError {
        bucket: String,
        #[source]
        err: std::io::Error,
    },
}

impl From<S3Error> for std::io::Error {
    fn from(err: S3Error) -> Self {
        Self::new(std::io::ErrorKind::Other, Box::new(err))
    }
}

type S3Result<R> = Result<R, S3Error>;

async fn download_object(
    tx: &Sender<S3Result<Vec<u8>>>,
    activator: &SyncActivator,
    client: &S3Client,
    bucket: &str,
    key: &str,
    compression: Compression,
    source_id: &str,
) -> Result<DownloadMetricUpdate, DownloadError> {
    let retry_reader: RetryReader<_, _, _> = RetryReader::new(|state, offset| async move {
        let range = if offset == 0 {
            None
        } else {
            debug!(
                "Failed to download object: {}/{} attempt={} read={}",
                bucket, key, state.i, offset,
            );
            Some(format!("bytes={}-", offset))
        };

        let obj = client
            .get_object()
            .bucket(bucket)
            .key(key)
            .set_range(range)
            .send()
            .await
            .or_else(|err| {
                Err(S3Error::GetObjectError {
                    bucket: bucket.to_string(),
                    key: key.to_string(),
                    err,
                })
            })?;

        // If the Content-Encoding does not match the compression specified for this
        // source, emit a debug message and trust the user-specified compression
        if let Some(s) = obj.content_encoding.as_deref() {
            match (s, compression) {
                ("gzip", Compression::Gzip) => (),
                ("identity", Compression::None) => (),
                ("identity" | "gzip", _) => {
                    debug!("object {} has mismatched Content-Encoding: {}", key, s)
                }
                _ => debug!("object {} has unrecognized Content-Encoding: {}", key, s),
            }
        }

        Ok(StreamReader::new(obj.body.map_err(|e| {
            std::io::Error::new(std::io::ErrorKind::Other, e)
        })))
    });

    let mut reader = Box::pin(BufReader::new(retry_reader));

    // Check for empty files by filling up the buffer of bufreader and checking if it got any bytes
    match reader.fill_buf().await {
        Ok(buf) => {
            if buf.is_empty() {
                trace!("source_id={} empty object {}/{}", source_id, bucket, key);
                return Ok(Default::default());
            }
        }
        Err(err) => return Err(DownloadError::Failed { err }),
    };

    let download_result = match compression {
        Compression::None => read_object_chunked(reader, tx).await,
        Compression::Gzip => {
            let decoder = GzipDecoder::new(reader);
            read_object_chunked(decoder, tx).await
        }
    };

    debug!(
        "source_id={} {}/{} download_result={:?}",
        source_id, bucket, key, download_result,
    );

    activator.activate().expect("s3 reader activation failed");
    download_result
}

async fn read_object_chunked<R>(
    mut reader: R,
    tx: &Sender<Result<Vec<u8>, S3Error>>,
) -> Result<DownloadMetricUpdate, DownloadError>
where
    R: Unpin + AsyncRead,
{
    let mut data = vec![];

    let bytes_read = match reader.read_to_end(&mut data).await {
        Ok(bytes_read) => bytes_read,
        Err(err) => return Err(DownloadError::Failed { err }),
    };

    if tx.send(Ok(data)).await.is_err() {
        return Err(DownloadError::SendFailed);
    }

    Ok(DownloadMetricUpdate {
        bytes: bytes_read.try_into().expect("usize <= u64"),
        messages: 1,
    })
}

impl SourceConnectionBuilder for S3SourceConnection {
    type Reader = S3SourceReader;
    type OffsetCommitter = LogCommitter;

    fn into_reader(
        self,
        source_name: String,
        source_id: GlobalId,
        worker_id: usize,
        worker_count: usize,
        consumer_activator: SyncActivator,
        data_capability: Capability<<Self::Reader as SourceReader>::Time>,
        upper_capability: Capability<<Self::Reader as SourceReader>::Time>,
        _resume_upper: Antichain<<Self::Reader as SourceReader>::Time>,
        _encoding: SourceDataEncoding,
        metrics: crate::source::metrics::SourceBaseMetrics,
        connection_context: ConnectionContext,
    ) -> Result<(Self::Reader, Self::OffsetCommitter), anyhow::Error> {
        let active_read_worker =
            crate::source::responsible_for(&source_id, worker_id, worker_count, ());

        // a single arbitrary worker is responsible for scanning the bucket
        let (receiver, shutdowner) = if active_read_worker {
            let (dataflow_tx, dataflow_rx) = tokio::sync::mpsc::channel(10_000);
            let (keys_tx, keys_rx) = tokio::sync::mpsc::channel(10_000);
            let (shutdowner, shutdown_rx) = tokio::sync::watch::channel(DataflowStatus::Running);
            let glob = self.pattern.map(|g| g.compile_matcher());

            task::spawn(|| format!("s3_download:{}", source_id), {
                let secrets_reader = Arc::clone(&connection_context.secrets_reader);
                download_objects_task(
                    source_id,
                    keys_rx,
                    dataflow_tx,
                    shutdown_rx.clone(),
                    self.aws.clone(),
                    connection_context.aws_external_id_prefix.clone(),
                    consumer_activator,
                    self.compression,
                    metrics.clone(),
                    secrets_reader,
                )
            });
            for key_source in self.key_sources {
                match key_source {
                    S3KeySource::Scan { bucket } => {
                        debug!(
                            "source_id={} reading s3 bucket={} worker={}",
                            source_id, bucket, worker_id
                        );
                        // TODO(guswynn): see if we can avoid this formatting
                        let task_name = format!("s3_scan:{}:{}", source_id, bucket);
                        task::spawn(|| task_name, {
                            let secrets_reader = Arc::clone(&connection_context.secrets_reader);
                            scan_bucket_task(
                                bucket,
                                source_id,
                                glob.clone(),
                                self.aws.clone(),
                                connection_context.aws_external_id_prefix.clone(),
                                keys_tx.clone(),
                                metrics.clone(),
                                secrets_reader,
                            )
                        });
                    }
                    S3KeySource::SqsNotifications { queue } => {
                        debug!(
                            "source_id={} reading sqs queue={} worker={}",
                            source_id, queue, worker_id
                        );
                        task::spawn(|| format!("s3_read_sqs:{}", source_id), {
                            let secrets_reader = Arc::clone(&connection_context.secrets_reader);
                            read_sqs_task(
                                source_id,
                                glob.clone(),
                                queue,
                                self.aws.clone(),
                                connection_context.aws_external_id_prefix.clone(),
                                keys_tx.clone(),
                                shutdown_rx.clone(),
                                metrics.clone(),
                                secrets_reader,
                            )
                        });
                    }
                }
            }
            (dataflow_rx, shutdowner)
        } else {
            let (_dataflow_tx, dataflow_rx) = tokio::sync::mpsc::channel(1);
            let (shutdowner, _shutdown_rx) = tokio::sync::watch::channel(DataflowStatus::Stopped);

            (dataflow_rx, shutdowner)
        };

        Ok((
            S3SourceReader {
                source_name,
                id: source_id,
                receiver_stream: receiver,
                dataflow_status: shutdowner,
                data_capability,
                upper_capability,
                offset: S3Offset(0),
                active_read_worker,
            },
            LogCommitter {
                source_id,
                worker_id,
                worker_count,
            },
        ))
    }
}

impl SourceReader for S3SourceReader {
    type Key = ();
    type Value = ByteStream;
    type Time = MzOffset;
    type Diff = u32;

    fn get_next_message(&mut self) -> NextMessage<Self::Key, Self::Value, Self::Time, Self::Diff> {
        if !self.active_read_worker {
            return NextMessage::Finished;
        }

        match self.receiver_stream.recv().now_or_never() {
            Some(Some(Ok(record))) => {
                self.offset += 1;
                let msg = SourceMessage {
                    output: 0,
                    upstream_time_millis: None,
                    key: (),
                    value: ByteStream::from_vec(record),
                    headers: None,
                };
                let ts = MzOffset::from(self.offset);
                let cap = self.data_capability.delayed(&ts);
                let next_ts = ts + 1;
                self.data_capability.downgrade(&next_ts);
                self.upper_capability.downgrade(&next_ts);
                NextMessage::Ready(SourceMessageType::Message(Ok(msg), cap, 1))
            }
            Some(Some(Err(e))) => match e {
                S3Error::GetObjectError { .. } => {
                    warn!(
                        "when reading source '{}' ({}): {}",
                        self.source_name, self.id, e
                    );
                    NextMessage::Pending
                }
                e @ (S3Error::ListObjectsFailed { .. } | S3Error::IoError { .. }) => {
                    let err = SourceReaderError::other_definite(anyhow::Error::new(e));
                    // XXX(petrosagg): We are fabricating a timestamp here. Is this error truly
                    // definite?
                    let not_definite_ts = MzOffset::from(self.offset);
                    let cap = self.data_capability.delayed(&not_definite_ts);
                    let next_ts = not_definite_ts + 1;
                    self.data_capability.downgrade(&next_ts);
                    self.upper_capability.downgrade(&next_ts);
                    NextMessage::Ready(SourceMessageType::Message(Err(err), cap, 1))
                }
            },
            None => NextMessage::Pending,
            Some(None) => NextMessage::Finished,
        }
    }
}

impl Drop for S3SourceReader {
    fn drop(&mut self) {
        debug!("source_id={} Dropping S3SourceReader", self.id);
        if self.dataflow_status.send(DataflowStatus::Stopped).is_err() {
            debug!("source_id={} already shutdown", self.id);
        };
    }
}

// Helper utilities

/// Set the SQS visibility timeout back to zero, allowing the messages to be sent to other clients
async fn release_messages(
    client: &SqsClient,
    message: Option<SqsMessage>,
    messages: impl Iterator<Item = SqsMessage>,
    queue_url: String,
    source_id: &str,
    failed_key: Option<String>,
) {
    if let Err(e) = client
        .change_message_visibility_batch()
        .set_entries(Some(
            message
                .into_iter()
                .chain(messages.into_iter())
                .filter_map(|m| m.receipt_handle)
                .enumerate()
                .map(|(i, receipt_handle)| {
                    debug!(
                        "source_id={} releasing message unprocessed_key={}",
                        source_id,
                        failed_key.as_deref().unwrap_or("<none>")
                    );
                    ChangeMessageVisibilityBatchRequestEntry::builder()
                        .id(i.to_string())
                        .receipt_handle(receipt_handle)
                        .visibility_timeout(0)
                        .build()
                })
                .collect(),
        ))
        .queue_url(queue_url)
        .send()
        .await
    {
        warn!("unexpected error releasing SQS messages: {}", e);
    };
}

/// Find the unambiguous prefix of a glob
fn find_prefix(glob: &str) -> String {
    let mut escaped = false;
    let mut escaped_filter = false;
    glob.chars()
        .take_while(|c| match (c, &escaped) {
            ('*', false) => false,
            ('[', false) => false, // a character class is a form of glob
            ('{', false) => false, // a group class is a form of glob
            ('\\', false) => {
                escaped = true;
                true
            }
            (_, false) => true,
            (_, true) => {
                escaped = false;
                true
            }
        })
        .filter(|c| match (c, &escaped_filter) {
            (_, true) => {
                escaped_filter = false;
                true
            }
            ('\\', false) => {
                escaped_filter = true;
                false
            }
            (_, _) => true,
        })
        .collect()
}

#[cfg(test)]
mod test {
    use super::*;

    #[test]
    fn glob_prefix() {
        assert_eq!(&find_prefix("foo/**"), "foo/");
        assert_eq!(&find_prefix("foo/"), "foo/");
        assert_eq!(&find_prefix(""), "");
        assert_eq!(&find_prefix("**/*.json"), "");
        assert_eq!(&find_prefix(r"foo/\*/bar/*.json"), r"foo/*/bar/");
        assert_eq!(&find_prefix("foo/[*]/**"), "foo/");
        assert_eq!(&find_prefix("foo/{a,b}"), "foo/");
        assert_eq!(&find_prefix(r"class/\[*.json"), "class/[");
        assert_eq!(&find_prefix(r"class/\[ab]/**"), "class/[ab]/");
        assert_eq!(&find_prefix(r"alt/\{a,b}/**"), "alt/{a,b}/");
    }
}