Skip to main content

mz_storage_operators/
oneshot_source.rs

1// Copyright Materialize, Inc. and contributors. All rights reserved.
2//
3// Use of this software is governed by the Business Source License
4// included in the LICENSE file.
5//
6// As of the Change Date specified in that file, in accordance with
7// the Business Source License, use of this software will be governed
8// by the Apache License, Version 2.0.
9
10//! "Oneshot" sources are a one-time ingestion of data from an external system, unlike traditional
11//! sources, they __do not__ run continuously. Oneshot sources are generally used for `COPY FROM`
12//! SQL statements.
13//!
14//! The implementation of reading and parsing data is behind the [`OneshotSource`] and
15//! [`OneshotFormat`] traits, respectively. Users looking to add new sources or formats, should
16//! only need to add new implementations for these traits.
17//!
18//! * [`OneshotSource`] is an interface for listing and reading from an external system, e.g. an
19//!   HTTP server.
20//! * [`OneshotFormat`] is an interface for how to parallelize and parse data, e.g. CSV.
21//!
22//! Given a [`OneshotSource`] and a [`OneshotFormat`] we build a dataflow structured like the
23//! following:
24//!
25//! ```text
26//!             ┏━━━━━━━━━━━━━━━┓
27//!             ┃    Discover   ┃
28//!             ┃    objects    ┃
29//!             ┗━━━━━━━┯━━━━━━━┛
30//!           ┌───< Distribute >───┐
31//!           │                    │
32//!     ┏━━━━━v━━━━┓         ┏━━━━━v━━━━┓
33//!     ┃  Split   ┃   ...   ┃  Split   ┃
34//!     ┃  Work 1  ┃         ┃  Work n  ┃
35//!     ┗━━━━━┯━━━━┛         ┗━━━━━┯━━━━┛
36//!           │                    │
37//!           ├───< Distribute >───┤
38//!           │                    │
39//!     ┏━━━━━v━━━━┓         ┏━━━━━v━━━━┓
40//!     ┃  Fetch   ┃   ...   ┃  Fetch   ┃
41//!     ┃  Work 1  ┃         ┃  Work n  ┃
42//!     ┗━━━━━┯━━━━┛         ┗━━━━━┯━━━━┛
43//!           │                    │
44//!           ├───< Distribute >───┤
45//!           │                    │
46//!     ┏━━━━━v━━━━┓         ┏━━━━━v━━━━┓
47//!     ┃  Decode  ┃   ...   ┃  Decode  ┃
48//!     ┃  Chunk 1 ┃         ┃  Chunk n ┃
49//!     ┗━━━━━┯━━━━┛         ┗━━━━━┯━━━━┛
50//!           │                    │
51//!           │                    │
52//!     ┏━━━━━v━━━━┓         ┏━━━━━v━━━━┓
53//!     ┃  Stage   ┃   ...   ┃  Stage   ┃
54//!     ┃  Batch 1 ┃         ┃  Batch n ┃
55//!     ┗━━━━━┯━━━━┛         ┗━━━━━┯━━━━┛
56//!           │                    │
57//!           └─────────┬──────────┘
58//!               ┏━━━━━v━━━━┓
59//!               ┃  Result  ┃
60//!               ┃ Callback ┃
61//!               ┗━━━━━━━━━━┛
62//! ```
63//!
64
65use std::fmt;
66use std::sync::Arc;
67
68use aws_sdk_s3::error::DisplayErrorContext;
69use bytes::Bytes;
70use differential_dataflow::Hashable;
71use futures::stream::BoxStream;
72use futures::{StreamExt, TryStreamExt};
73use mz_expr::SafeMfpPlan;
74use mz_ore::cast::CastFrom;
75use mz_ore::error::ErrorExt;
76use mz_persist_client::Diagnostics;
77use mz_persist_client::batch::ProtoBatch;
78use mz_persist_client::cache::PersistClientCache;
79use mz_persist_types::Codec;
80use mz_persist_types::codec_impls::UnitSchema;
81use mz_repr::{DatumVec, GlobalId, Row, RowArena, Timestamp};
82use mz_storage_types::StorageDiff;
83use mz_storage_types::connections::ConnectionContext;
84use mz_storage_types::controller::CollectionMetadata;
85use mz_storage_types::oneshot_sources::{
86    ContentFilter, ContentFormat, ContentSource, OneshotIngestionRequest,
87};
88use mz_storage_types::sources::SourceData;
89use mz_timely_util::builder_async::{
90    Event as AsyncEvent, OperatorBuilder as AsyncOperatorBuilder, PressOnDropButton,
91};
92use mz_timely_util::pact::Distribute;
93use serde::de::DeserializeOwned;
94use serde::{Deserialize, Serialize};
95use std::collections::{BTreeSet, LinkedList};
96use std::fmt::{Debug, Display};
97use std::future::Future;
98use timely::container::CapacityContainerBuilder;
99use timely::dataflow::channels::pact::Pipeline;
100use timely::dataflow::{Scope, StreamVec};
101use timely::progress::Antichain;
102use tracing::info;
103
104use crate::oneshot_source::aws_source::{AwsS3Source, S3Checksum, S3Object};
105use crate::oneshot_source::csv::{CsvDecoder, CsvRecord, CsvWorkRequest};
106use crate::oneshot_source::http_source::{
107    HttpChecksum, HttpObject, HttpOneshotSource, build_http_client,
108};
109use crate::oneshot_source::parquet::{ParquetFormat, ParquetRowGroup, ParquetWorkRequest};
110
111pub mod csv;
112pub mod parquet;
113
114pub mod aws_source;
115pub mod http_source;
116
117mod util;
118
119/// Render a dataflow to do a "oneshot" ingestion.
120///
121/// Roughly the operators we render do the following:
122///
123/// 1. Discover objects with a [`OneshotSource`].
124/// 2. Split objects into separate units of work based on the [`OneshotFormat`].
125/// 3. Fetch individual units of work (aka fetch byte blobs) with the
126///    [`OneshotFormat`] and [`OneshotSource`].
127/// 4. Decode the fetched byte blobs into [`Row`]s.
128/// 5. Stage the [`Row`]s into Persist returning [`ProtoBatch`]es.
129///
130/// TODO(cf3): Benchmark combining operators 3, 4, and 5. Currently we keep them
131/// separate for the [`CsvDecoder`]. CSV decoding is hard to do in parallel so we
132/// currently have a single worker Fetch an entire file, and then distributes
133/// chunks for parallel Decoding. We should benchmark if this is actually faster
134/// than just a single worker both fetching and decoding.
135///
136pub fn render<'scope, F>(
137    scope: Scope<'scope, Timestamp>,
138    persist_clients: Arc<PersistClientCache>,
139    connection_context: ConnectionContext,
140    collection_id: GlobalId,
141    collection_meta: CollectionMetadata,
142    request: OneshotIngestionRequest,
143    enforce_external_addresses: bool,
144    worker_callback: F,
145) -> Vec<PressOnDropButton>
146where
147    F: FnOnce(Result<Option<ProtoBatch>, String>) -> () + 'static,
148{
149    let OneshotIngestionRequest {
150        source,
151        format,
152        filter,
153        shape,
154    } = request;
155
156    let source = match source {
157        ContentSource::Http { url } => {
158            let client = match build_http_client(enforce_external_addresses) {
159                Ok(client) => client,
160                Err(err) => {
161                    worker_callback(Err(format!(
162                        "failed to build HTTP client for COPY FROM: {err}"
163                    )));
164                    return Vec::new();
165                }
166            };
167            let source = HttpOneshotSource::new(client, url);
168            SourceKind::Http(source)
169        }
170        ContentSource::AwsS3 {
171            connection,
172            connection_id,
173            uri,
174        } => {
175            // Checksum validation does not work with GCS when using ranges, which happens with parquet.
176            // So, we disable checksum if both the endpoint is overridden to a non-AWS endpoint and the format is Parquet.
177            let use_checksum = !(connection.endpoint.is_some()
178                && !connection.endpoint.as_ref().unwrap().contains("amazonaws"))
179                || format != ContentFormat::Parquet;
180            if !use_checksum {
181                tracing::info!(
182                    "disabling checksum validation for S3 source because endpoint: {:?} is overridden and format is Parquet",
183                    connection.endpoint
184                );
185            }
186            let source = AwsS3Source::new(
187                connection,
188                connection_id,
189                connection_context,
190                uri,
191                use_checksum,
192                enforce_external_addresses,
193            );
194            SourceKind::AwsS3(source)
195        }
196    };
197    tracing::info!(?source, "created oneshot source");
198
199    let format = match format {
200        ContentFormat::Csv(params) => {
201            let format = CsvDecoder::new(params, &shape.source_desc);
202            FormatKind::Csv(format)
203        }
204        ContentFormat::Parquet => {
205            let format = ParquetFormat::new(shape.source_desc);
206            FormatKind::Parquet(format)
207        }
208    };
209
210    // Discover what objects are available to copy.
211    let (objects_stream, discover_token) =
212        render_discover_objects(scope.clone(), collection_id, source.clone(), filter);
213    // Split the objects into individual units of work.
214    let (work_stream, split_token) = render_split_work(
215        scope.clone(),
216        collection_id,
217        objects_stream,
218        source.clone(),
219        format.clone(),
220    );
221    // Fetch each unit of work, returning chunks of records.
222    let (records_stream, fetch_token) = render_fetch_work(
223        scope.clone(),
224        collection_id,
225        source.clone(),
226        format.clone(),
227        work_stream,
228    );
229    // Parse chunks of records into Rows.
230    let (rows_stream, decode_token) = render_decode_chunk(
231        scope.clone(),
232        format.clone(),
233        records_stream,
234        shape.source_mfp,
235    );
236    // Stage the Rows in Persist.
237    let (batch_stream, batch_token) = render_stage_batches_operator(
238        scope.clone(),
239        collection_id,
240        &collection_meta,
241        persist_clients,
242        rows_stream,
243    );
244
245    // Collect all results together and notify the upstream of whether or not we succeeded.
246    render_completion_operator(scope, batch_stream, worker_callback);
247
248    let tokens = vec![
249        discover_token,
250        split_token,
251        fetch_token,
252        decode_token,
253        batch_token,
254    ];
255
256    tokens
257}
258
259/// Render an operator that using a [`OneshotSource`] will discover what objects are available
260/// for fetching.
261pub fn render_discover_objects<'scope, S>(
262    scope: Scope<'scope, Timestamp>,
263    collection_id: GlobalId,
264    source: S,
265    filter: ContentFilter,
266) -> (
267    StreamVec<'scope, Timestamp, Result<(S::Object, S::Checksum), StorageErrorX>>,
268    PressOnDropButton,
269)
270where
271    S: OneshotSource + 'static,
272{
273    // Only a single worker is responsible for discovering objects.
274    let worker_id = scope.index();
275    let num_workers = scope.peers();
276    let active_worker_id = usize::cast_from((collection_id, "discover").hashed()) % num_workers;
277    let is_active_worker = worker_id == active_worker_id;
278
279    let mut builder = AsyncOperatorBuilder::new("CopyFrom-discover".to_string(), scope.clone());
280
281    let (start_handle, start_stream) = builder.new_output::<CapacityContainerBuilder<_>>();
282
283    let shutdown = builder.build(move |caps| async move {
284        let [start_cap] = caps.try_into().unwrap();
285
286        if !is_active_worker {
287            return;
288        }
289
290        let filter = match ObjectFilter::try_new(filter) {
291            Ok(filter) => filter,
292            Err(err) => {
293                tracing::warn!(?err, "failed to create filter");
294                start_handle.give(&start_cap, Err(StorageErrorXKind::generic(err).into()));
295                return;
296            }
297        };
298
299        let work = source.list().await.context("list");
300        match work {
301            Ok(objects) => {
302                let (include, exclude): (Vec<_>, Vec<_>) = objects
303                    .into_iter()
304                    .partition(|(o, _checksum)| filter.filter::<S>(o));
305                tracing::info!(%worker_id, ?include, ?exclude, "listed objects");
306                if include.is_empty() {
307                    let err = StorageErrorXKind::NoMatchingFiles.with_context("discover");
308                    start_handle.give(&start_cap, Err(err));
309                    return;
310                }
311                include
312                    .into_iter()
313                    .for_each(|object| start_handle.give(&start_cap, Ok(object)))
314            }
315            Err(err) => {
316                tracing::warn!(?err, "failed to list oneshot source");
317                start_handle.give(&start_cap, Err(err))
318            }
319        }
320    });
321
322    (start_stream, shutdown.press_on_drop())
323}
324
325/// Render an operator that given a stream of [`OneshotSource::Object`]s will split them into units
326/// of work based on the provided [`OneshotFormat`].
327pub fn render_split_work<'scope, T, S, F>(
328    scope: Scope<'scope, T>,
329    collection_id: GlobalId,
330    objects: StreamVec<'scope, T, Result<(S::Object, S::Checksum), StorageErrorX>>,
331    source: S,
332    format: F,
333) -> (
334    StreamVec<'scope, T, Result<F::WorkRequest<S>, StorageErrorX>>,
335    PressOnDropButton,
336)
337where
338    T: timely::progress::Timestamp,
339    S: OneshotSource + Send + Sync + 'static,
340    F: OneshotFormat + Send + Sync + 'static,
341{
342    let worker_id = scope.index();
343    let mut builder = AsyncOperatorBuilder::new("CopyFrom-split_work".to_string(), scope.clone());
344
345    let (request_handle, request_stream) = builder.new_output::<CapacityContainerBuilder<_>>();
346    let mut objects_handle = builder.new_input_for(objects, Distribute, &request_handle);
347
348    let shutdown = builder.build(move |caps| async move {
349        let [_objects_cap] = caps.try_into().unwrap();
350
351        info!(%collection_id, %worker_id, "CopyFrom Split Work");
352
353        while let Some(event) = objects_handle.next().await {
354            let (capability, maybe_objects) = match event {
355                AsyncEvent::Data(cap, req) => (cap, req),
356                AsyncEvent::Progress(_) => continue,
357            };
358
359            // Nest the `split_work(...)` method in an async-block so we can use the `?`
360            // without returning from the entire operator, and so we can add context.
361            let result = async {
362                let mut requests = Vec::new();
363
364                for maybe_object in maybe_objects {
365                    // Return early if the upstream Discover step failed.
366                    let (object, checksum) = maybe_object?;
367
368                    let format_ = format.clone();
369                    let source_ = source.clone();
370                    let work_requests = mz_ore::task::spawn(|| "split-work", async move {
371                        info!(%worker_id, object = %object.name(), "splitting object");
372                        format_.split_work(source_.clone(), object, checksum).await
373                    })
374                    .await?;
375
376                    requests.extend(work_requests);
377                }
378
379                Ok::<_, StorageErrorX>(requests)
380            }
381            .await
382            .context("split");
383
384            match result {
385                Ok(requests) => requests
386                    .into_iter()
387                    .for_each(|req| request_handle.give(&capability, Ok(req))),
388                Err(err) => request_handle.give(&capability, Err(err)),
389            }
390        }
391    });
392
393    (request_stream, shutdown.press_on_drop())
394}
395
396/// Render an operator that given a stream [`OneshotFormat::WorkRequest`]s will fetch chunks of the
397/// remote [`OneshotSource::Object`] and return a stream of [`OneshotFormat::RecordChunk`]s that
398/// can be decoded into [`Row`]s.
399pub fn render_fetch_work<'scope, T, S, F>(
400    scope: Scope<'scope, T>,
401    collection_id: GlobalId,
402    source: S,
403    format: F,
404    work_requests: StreamVec<'scope, T, Result<F::WorkRequest<S>, StorageErrorX>>,
405) -> (
406    StreamVec<'scope, T, Result<F::RecordChunk, StorageErrorX>>,
407    PressOnDropButton,
408)
409where
410    T: timely::progress::Timestamp,
411    S: OneshotSource + Sync + 'static,
412    F: OneshotFormat + Sync + 'static,
413{
414    let worker_id = scope.index();
415    let mut builder = AsyncOperatorBuilder::new("CopyFrom-fetch_work".to_string(), scope.clone());
416
417    let (record_handle, record_stream) = builder.new_output::<CapacityContainerBuilder<_>>();
418    let mut work_requests_handle = builder.new_input_for(work_requests, Distribute, &record_handle);
419
420    let shutdown = builder.build(move |caps| async move {
421        let [_work_cap] = caps.try_into().unwrap();
422
423        info!(%collection_id, %worker_id, "CopyFrom Fetch Work");
424
425        while let Some(event) = work_requests_handle.next().await {
426            let (capability, maybe_requests) = match event {
427                AsyncEvent::Data(cap, req) => (cap, req),
428                AsyncEvent::Progress(_) => continue,
429            };
430
431            // Wrap our work in a block to capture `?`.
432            let result = async {
433                // Process each stream of work, one at a time.
434                for maybe_request in maybe_requests {
435                    let request = maybe_request?;
436
437                    let mut work_stream = format.fetch_work(&source, request);
438                    while let Some(result) = work_stream.next().await {
439                        // Returns early and stop consuming from the stream if we hit an error.
440                        let record_chunk = result.context("fetch worker")?;
441
442                        // Note: We want to give record chunks as we receive them, because a work
443                        // request may be for an entire file.
444                        //
445                        // TODO(cf3): Investigate if some small batching here would help perf.
446                        record_handle.give(&capability, Ok(record_chunk));
447                    }
448                }
449
450                Ok::<_, StorageErrorX>(())
451            }
452            .await
453            .context("fetch work");
454
455            if let Err(err) = result {
456                tracing::warn!(?err, "failed to fetch");
457                record_handle.give(&capability, Err(err))
458            }
459        }
460    });
461
462    (record_stream, shutdown.press_on_drop())
463}
464
465/// Render an operator that given a stream of [`OneshotFormat::RecordChunk`]s will decode these
466/// chunks into a stream of [`Row`]s.
467pub fn render_decode_chunk<'scope, T, F>(
468    scope: Scope<'scope, T>,
469    format: F,
470    record_chunks: StreamVec<'scope, T, Result<F::RecordChunk, StorageErrorX>>,
471    mfp: SafeMfpPlan,
472) -> (
473    StreamVec<'scope, T, Result<Row, StorageErrorX>>,
474    PressOnDropButton,
475)
476where
477    T: timely::progress::Timestamp,
478    F: OneshotFormat + 'static,
479{
480    let mut builder = AsyncOperatorBuilder::new("CopyFrom-decode_chunk".to_string(), scope.clone());
481
482    let (row_handle, row_stream) = builder.new_output::<CapacityContainerBuilder<_>>();
483    let mut record_chunk_handle = builder.new_input_for(record_chunks, Distribute, &row_handle);
484
485    let shutdown = builder.build(move |caps| async move {
486        let [_row_cap] = caps.try_into().unwrap();
487
488        let mut datum_vec = DatumVec::default();
489        let row_arena = RowArena::default();
490        let mut row_buf = Row::default();
491
492        while let Some(event) = record_chunk_handle.next().await {
493            let (capability, maybe_chunks) = match event {
494                AsyncEvent::Data(cap, data) => (cap, data),
495                AsyncEvent::Progress(_) => continue,
496            };
497
498            let result = async {
499                let mut rows = Vec::new();
500                for maybe_chunk in maybe_chunks {
501                    let chunk = maybe_chunk?;
502                    format.decode_chunk(chunk, &mut rows)?;
503                }
504                Ok::<_, StorageErrorX>(rows)
505            }
506            .await
507            .context("decode chunk");
508
509            match result {
510                Ok(rows) => {
511                    // For each row of source data, we pass it through an MFP to re-arrange column
512                    // orders and/or fill in default values for missing columns.
513                    for row in rows {
514                        let mut datums = datum_vec.borrow_with(&row);
515                        let result = mfp
516                            .evaluate_into(&mut *datums, &row_arena, &mut row_buf)
517                            .map(|row| row.cloned());
518
519                        match result {
520                            Ok(Some(row)) => row_handle.give(&capability, Ok(row)),
521                            Ok(None) => {
522                                // We only expect the provided MFP to map rows from the source data
523                                // and project default values.
524                                mz_ore::soft_panic_or_log!("oneshot source MFP filtered out data!");
525                            }
526                            Err(e) => {
527                                let err = StorageErrorXKind::MfpEvalError(e.to_string().into())
528                                    .with_context("decode");
529                                row_handle.give(&capability, Err(err))
530                            }
531                        }
532                    }
533                }
534                Err(err) => row_handle.give(&capability, Err(err)),
535            }
536        }
537    });
538
539    (row_stream, shutdown.press_on_drop())
540}
541
542/// Render an operator that given a stream of [`Row`]s will stage them in Persist and return a
543/// stream of [`ProtoBatch`]es that can later be linked into a shard.
544pub fn render_stage_batches_operator<'scope, T>(
545    scope: Scope<'scope, T>,
546    collection_id: GlobalId,
547    collection_meta: &CollectionMetadata,
548    persist_clients: Arc<PersistClientCache>,
549    rows_stream: StreamVec<'scope, T, Result<Row, StorageErrorX>>,
550) -> (
551    StreamVec<'scope, T, Result<ProtoBatch, StorageErrorX>>,
552    PressOnDropButton,
553)
554where
555    T: timely::progress::Timestamp,
556{
557    let persist_location = collection_meta.persist_location.clone();
558    let shard_id = collection_meta.data_shard;
559    let collection_desc = Arc::new(collection_meta.relation_desc.clone());
560
561    let mut builder =
562        AsyncOperatorBuilder::new("CopyFrom-stage_batches".to_string(), scope.clone());
563
564    let (proto_batch_handle, proto_batch_stream) =
565        builder.new_output::<CapacityContainerBuilder<_>>();
566    let mut rows_handle = builder.new_input_for(rows_stream, Pipeline, &proto_batch_handle);
567
568    let shutdown = builder.build(move |caps| async move {
569        let [proto_batch_cap] = caps.try_into().unwrap();
570
571        // Open a Persist handle that we can use to stage a batch.
572        let persist_client = persist_clients
573            .open(persist_location)
574            .await
575            .expect("failed to open Persist client");
576        let persist_diagnostics = Diagnostics {
577            shard_name: collection_id.to_string(),
578            handle_purpose: "CopyFrom::stage_batches".to_string(),
579        };
580        let write_handle = persist_client
581            .open_writer::<SourceData, (), mz_repr::Timestamp, StorageDiff>(
582                shard_id,
583                Arc::clone(&collection_desc),
584                Arc::new(UnitSchema),
585                persist_diagnostics,
586            )
587            .await
588            .expect("could not open Persist shard");
589
590        // Create a batch using the minimum timestamp since these batches will
591        // get sent back to `environmentd` and their timestamps re-written
592        // before being finally appended.
593        let lower = mz_repr::Timestamp::MIN;
594        let upper = Antichain::from_elem(lower.step_forward());
595
596        let mut batch_builder = write_handle.builder(Antichain::from_elem(lower));
597
598        while let Some(event) = rows_handle.next().await {
599            let AsyncEvent::Data(_, row_batch) = event else {
600                continue;
601            };
602
603            // Pull Rows off our stream and stage them into a Batch.
604            for maybe_row in row_batch {
605                let maybe_row = maybe_row.and_then(|row| {
606                    Row::validate(&row, &*collection_desc).map_err(|e| {
607                        StorageErrorXKind::invalid_record_batch(e).with_context("stage_batches")
608                    })?;
609                    Ok(row)
610                });
611                match maybe_row {
612                    // Happy path, add the Row to our batch!
613                    Ok(row) => {
614                        let data = SourceData(Ok(row));
615                        batch_builder
616                            .add(&data, &(), &lower, &1)
617                            .await
618                            .expect("failed to add Row to batch");
619                    }
620                    // Sad path, something upstream hit an error.
621                    Err(err) => {
622                        // Clean up our in-progress batch so we don't leak data.
623                        let batch = batch_builder
624                            .finish(upper)
625                            .await
626                            .expect("failed to cleanup batch");
627                        batch.delete().await;
628
629                        // Pass on the error.
630                        proto_batch_handle
631                            .give(&proto_batch_cap, Err(err).context("stage batches"));
632                        return;
633                    }
634                }
635            }
636        }
637
638        let batch = batch_builder
639            .finish(upper)
640            .await
641            .expect("failed to create Batch");
642
643        // Turn our Batch into a ProtoBatch that will later be linked in to
644        // the shard.
645        //
646        // Note: By turning this into a ProtoBatch, the onus is now on us to
647        // cleanup the Batch if it's never linked into the shard.
648        //
649        // TODO(cf2): Make sure these batches get cleaned up if another
650        // worker encounters an error.
651        let proto_batch = batch.into_transmittable_batch();
652        proto_batch_handle.give(&proto_batch_cap, Ok(proto_batch));
653    });
654
655    (proto_batch_stream, shutdown.press_on_drop())
656}
657
658/// Render an operator that given a stream of [`ProtoBatch`]es will call our `worker_callback` to
659/// report the results upstream.
660pub fn render_completion_operator<'scope, T, F>(
661    scope: Scope<'scope, T>,
662    results_stream: StreamVec<'scope, T, Result<ProtoBatch, StorageErrorX>>,
663    worker_callback: F,
664) where
665    T: timely::progress::Timestamp,
666    F: FnOnce(Result<Option<ProtoBatch>, String>) -> () + 'static,
667{
668    let mut builder = AsyncOperatorBuilder::new("CopyFrom-completion".to_string(), scope.clone());
669    let mut results_input = builder.new_disconnected_input(results_stream, Pipeline);
670
671    builder.build(move |_| async move {
672        let result = async move {
673            let mut maybe_payload: Option<ProtoBatch> = None;
674
675            while let Some(event) = results_input.next().await {
676                if let AsyncEvent::Data(_cap, results) = event {
677                    let [result] = results
678                        .try_into()
679                        .expect("only 1 event on the result stream");
680
681                    // TODO(cf2): Lift this restriction.
682                    if maybe_payload.is_some() {
683                        panic!("expected only one batch!");
684                    }
685
686                    maybe_payload = Some(result.map_err(|e| e.to_string())?);
687                }
688            }
689
690            Ok(maybe_payload)
691        }
692        .await;
693
694        // Report to the caller of our final status.
695        worker_callback(result);
696    });
697}
698
699/// An object that will be fetched from a [`OneshotSource`].
700pub trait OneshotObject {
701    /// Name of the object, including any extensions.
702    fn name(&self) -> &str;
703
704    /// Path of the object within the remote source.
705    fn path(&self) -> &str;
706
707    /// Size of this object in bytes.
708    fn size(&self) -> usize;
709
710    /// Encodings of the _entire_ object, if any.
711    ///
712    /// Note: The object may internally use compression, e.g. a Parquet file
713    /// could compress its column chunks, but if the Parquet file itself is not
714    /// compressed then this would return `None`.
715    fn encodings(&self) -> &[Encoding];
716}
717
718/// Encoding of a [`OneshotObject`].
719#[derive(Debug, Copy, Clone, Serialize, Deserialize)]
720pub enum Encoding {
721    Bzip2,
722    Gzip,
723    Xz,
724    Zstd,
725}
726
727/// Defines a remote system that we can fetch data from for a "one time" ingestion.
728pub trait OneshotSource: Clone + Send + Unpin {
729    /// An individual unit within the source, e.g. a file.
730    type Object: OneshotObject
731        + Debug
732        + Clone
733        + Send
734        + Unpin
735        + Serialize
736        + DeserializeOwned
737        + 'static;
738    /// Checksum for a [`Self::Object`].
739    type Checksum: Debug + Clone + Send + Unpin + Serialize + DeserializeOwned + 'static;
740
741    /// Returns all of the objects for this source.
742    fn list<'a>(
743        &'a self,
744    ) -> impl Future<Output = Result<Vec<(Self::Object, Self::Checksum)>, StorageErrorX>> + Send;
745
746    /// Resturns a stream of the data for a specific object.
747    fn get<'s>(
748        &'s self,
749        object: Self::Object,
750        checksum: Self::Checksum,
751        range: Option<std::ops::RangeInclusive<usize>>,
752    ) -> BoxStream<'s, Result<Bytes, StorageErrorX>>;
753}
754
755/// An enum wrapper around [`OneshotSource`]s.
756///
757/// An alternative to this wrapper would be to use `Box<dyn OneshotSource>`, but that requires
758/// making the trait object safe and it's easier to just wrap it in an enum. Also, this wrapper
759/// provides a convenient place to add [`StorageErrorXContext::context`] for all of our source
760/// types.
761#[derive(Clone, Debug)]
762pub(crate) enum SourceKind {
763    Http(HttpOneshotSource),
764    AwsS3(AwsS3Source),
765}
766
767impl OneshotSource for SourceKind {
768    type Object = ObjectKind;
769    type Checksum = ChecksumKind;
770
771    async fn list<'a>(&'a self) -> Result<Vec<(Self::Object, Self::Checksum)>, StorageErrorX> {
772        match self {
773            SourceKind::Http(http) => {
774                let objects = http.list().await.context("http")?;
775                let objects = objects
776                    .into_iter()
777                    .map(|(object, checksum)| {
778                        (ObjectKind::Http(object), ChecksumKind::Http(checksum))
779                    })
780                    .collect();
781                Ok(objects)
782            }
783            SourceKind::AwsS3(s3) => {
784                let objects = s3.list().await.context("s3")?;
785                let objects = objects
786                    .into_iter()
787                    .map(|(object, checksum)| {
788                        (ObjectKind::AwsS3(object), ChecksumKind::AwsS3(checksum))
789                    })
790                    .collect();
791                Ok(objects)
792            }
793        }
794    }
795
796    fn get<'s>(
797        &'s self,
798        object: Self::Object,
799        checksum: Self::Checksum,
800        range: Option<std::ops::RangeInclusive<usize>>,
801    ) -> BoxStream<'s, Result<Bytes, StorageErrorX>> {
802        match (self, object, checksum) {
803            (SourceKind::Http(http), ObjectKind::Http(object), ChecksumKind::Http(checksum)) => {
804                http.get(object, checksum, range)
805                    .map(|result| result.context("http"))
806                    .boxed()
807            }
808            (SourceKind::AwsS3(s3), ObjectKind::AwsS3(object), ChecksumKind::AwsS3(checksum)) => s3
809                .get(object, checksum, range)
810                .map(|result| result.context("aws_s3"))
811                .boxed(),
812            (SourceKind::AwsS3(_) | SourceKind::Http(_), _, _) => {
813                unreachable!("programming error! wrong source, object, and checksum kind");
814            }
815        }
816    }
817}
818
819/// Enum wrapper for [`OneshotSource::Object`], see [`SourceKind`] for more details.
820#[derive(Debug, Clone, Serialize, Deserialize)]
821pub(crate) enum ObjectKind {
822    Http(HttpObject),
823    AwsS3(S3Object),
824}
825
826impl OneshotObject for ObjectKind {
827    fn name(&self) -> &str {
828        match self {
829            ObjectKind::Http(object) => object.name(),
830            ObjectKind::AwsS3(object) => object.name(),
831        }
832    }
833
834    fn path(&self) -> &str {
835        match self {
836            ObjectKind::Http(object) => object.path(),
837            ObjectKind::AwsS3(object) => object.path(),
838        }
839    }
840
841    fn size(&self) -> usize {
842        match self {
843            ObjectKind::Http(object) => object.size(),
844            ObjectKind::AwsS3(object) => object.size(),
845        }
846    }
847
848    fn encodings(&self) -> &[Encoding] {
849        match self {
850            ObjectKind::Http(object) => object.encodings(),
851            ObjectKind::AwsS3(object) => object.encodings(),
852        }
853    }
854}
855
856/// Enum wrapper for [`OneshotSource::Checksum`], see [`SourceKind`] for more details.
857#[derive(Debug, Clone, Serialize, Deserialize)]
858pub(crate) enum ChecksumKind {
859    Http(HttpChecksum),
860    AwsS3(S3Checksum),
861}
862
863/// Defines a format that we fetch for a "one time" ingestion.
864pub trait OneshotFormat: Clone {
865    /// A single unit of work for decoding this format, e.g. a single Parquet RowGroup.
866    type WorkRequest<S>: Debug + Clone + Send + Serialize + DeserializeOwned + 'static
867    where
868        S: OneshotSource;
869    /// A chunk of records in this format that can be decoded into Rows.
870    type RecordChunk: Debug + Clone + Send + Serialize + DeserializeOwned + 'static;
871
872    /// Given an upstream object, defines how we should parse this object in parallel.
873    ///
874    /// Note: It's totally fine to not process an object in parallel, and just return a single
875    /// [`Self::WorkRequest`] here.
876    fn split_work<S: OneshotSource + Send>(
877        &self,
878        source: S,
879        object: S::Object,
880        checksum: S::Checksum,
881    ) -> impl Future<Output = Result<Vec<Self::WorkRequest<S>>, StorageErrorX>> + Send;
882
883    /// Given a work request, fetch data from the [`OneshotSource`] and return it in a format that
884    /// can later be decoded.
885    fn fetch_work<'a, S: OneshotSource + Sync + 'static>(
886        &'a self,
887        source: &'a S,
888        request: Self::WorkRequest<S>,
889    ) -> BoxStream<'a, Result<Self::RecordChunk, StorageErrorX>>;
890
891    /// Decode a chunk of records into [`Row`]s.
892    fn decode_chunk(
893        &self,
894        chunk: Self::RecordChunk,
895        rows: &mut Vec<Row>,
896    ) -> Result<usize, StorageErrorX>;
897}
898
899/// An enum wrapper around [`OneshotFormat`]s.
900///
901/// An alternative to this wrapper would be to use `Box<dyn OneshotFormat>`, but that requires
902/// making the trait object safe and it's easier to just wrap it in an enum. Also, this wrapper
903/// provides a convenient place to add [`StorageErrorXContext::context`] for all of our format
904/// types.
905#[derive(Clone, Debug)]
906pub(crate) enum FormatKind {
907    Csv(CsvDecoder),
908    Parquet(ParquetFormat),
909}
910
911impl OneshotFormat for FormatKind {
912    type WorkRequest<S>
913        = RequestKind<S::Object, S::Checksum>
914    where
915        S: OneshotSource;
916    type RecordChunk = RecordChunkKind;
917
918    async fn split_work<S: OneshotSource + Send>(
919        &self,
920        source: S,
921        object: S::Object,
922        checksum: S::Checksum,
923    ) -> Result<Vec<Self::WorkRequest<S>>, StorageErrorX> {
924        match self {
925            FormatKind::Csv(csv) => {
926                let work = csv
927                    .split_work(source, object, checksum)
928                    .await
929                    .context("csv")?
930                    .into_iter()
931                    .map(RequestKind::Csv)
932                    .collect();
933                Ok(work)
934            }
935            FormatKind::Parquet(parquet) => {
936                let work = parquet
937                    .split_work(source, object, checksum)
938                    .await
939                    .context("parquet")?
940                    .into_iter()
941                    .map(RequestKind::Parquet)
942                    .collect();
943                Ok(work)
944            }
945        }
946    }
947
948    fn fetch_work<'a, S: OneshotSource + Sync + 'static>(
949        &'a self,
950        source: &'a S,
951        request: Self::WorkRequest<S>,
952    ) -> BoxStream<'a, Result<Self::RecordChunk, StorageErrorX>> {
953        match (self, request) {
954            (FormatKind::Csv(csv), RequestKind::Csv(request)) => csv
955                .fetch_work(source, request)
956                .map_ok(RecordChunkKind::Csv)
957                .map(|result| result.context("csv"))
958                .boxed(),
959            (FormatKind::Parquet(parquet), RequestKind::Parquet(request)) => parquet
960                .fetch_work(source, request)
961                .map_ok(RecordChunkKind::Parquet)
962                .map(|result| result.context("parquet"))
963                .boxed(),
964            (FormatKind::Parquet(_), RequestKind::Csv(_))
965            | (FormatKind::Csv(_), RequestKind::Parquet(_)) => {
966                unreachable!("programming error, {self:?}")
967            }
968        }
969    }
970
971    fn decode_chunk(
972        &self,
973        chunk: Self::RecordChunk,
974        rows: &mut Vec<Row>,
975    ) -> Result<usize, StorageErrorX> {
976        match (self, chunk) {
977            (FormatKind::Csv(csv), RecordChunkKind::Csv(chunk)) => {
978                csv.decode_chunk(chunk, rows).context("csv")
979            }
980            (FormatKind::Parquet(parquet), RecordChunkKind::Parquet(chunk)) => {
981                parquet.decode_chunk(chunk, rows).context("parquet")
982            }
983            (FormatKind::Parquet(_), RecordChunkKind::Csv(_))
984            | (FormatKind::Csv(_), RecordChunkKind::Parquet(_)) => {
985                unreachable!("programming error, {self:?}")
986            }
987        }
988    }
989}
990
991#[derive(Clone, Debug, Serialize, Deserialize)]
992pub(crate) enum RequestKind<O, C> {
993    Csv(CsvWorkRequest<O, C>),
994    Parquet(ParquetWorkRequest<O, C>),
995}
996
997#[derive(Clone, Debug, Serialize, Deserialize)]
998pub(crate) enum RecordChunkKind {
999    Csv(CsvRecord),
1000    Parquet(ParquetRowGroup),
1001}
1002
1003pub(crate) enum ObjectFilter {
1004    None,
1005    Files(BTreeSet<Box<str>>),
1006    Pattern(glob::Pattern),
1007}
1008
1009impl ObjectFilter {
1010    pub fn try_new(filter: ContentFilter) -> Result<Self, anyhow::Error> {
1011        match filter {
1012            ContentFilter::None => Ok(ObjectFilter::None),
1013            ContentFilter::Files(files) => {
1014                let files = files.into_iter().map(|f| f.into()).collect();
1015                Ok(ObjectFilter::Files(files))
1016            }
1017            ContentFilter::Pattern(pattern) => {
1018                let pattern = glob::Pattern::new(&pattern)?;
1019                Ok(ObjectFilter::Pattern(pattern))
1020            }
1021        }
1022    }
1023
1024    /// Returns if the object should be included.
1025    pub fn filter<S: OneshotSource>(&self, object: &S::Object) -> bool {
1026        match self {
1027            ObjectFilter::None => true,
1028            ObjectFilter::Files(files) => files.contains(object.path()),
1029            ObjectFilter::Pattern(pattern) => pattern.matches(object.path()),
1030        }
1031    }
1032}
1033
1034/// Experimental Error Type.
1035///
1036/// The goal of this type is to combine concepts from both `thiserror` and
1037/// `anyhow`. Having "stongly typed" errors from `thiserror` is useful for
1038/// determining what action to take and tracking the context of an error like
1039/// `anyhow` is useful for determining where an error came from.
1040#[derive(Debug, Clone, Deserialize, Serialize)]
1041pub struct StorageErrorX {
1042    kind: StorageErrorXKind,
1043    context: LinkedList<String>,
1044}
1045
1046impl fmt::Display for StorageErrorX {
1047    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
1048        writeln!(f, "error: {}", self.kind)?;
1049        writeln!(f, "causes: {:?}", self.context)?;
1050        Ok(())
1051    }
1052}
1053
1054/// Experimental Error Type, see [`StorageErrorX`].
1055#[derive(Debug, Clone, Serialize, Deserialize, thiserror::Error)]
1056pub enum StorageErrorXKind {
1057    #[error("csv decoding error: {0}")]
1058    CsvDecoding(Arc<str>),
1059    #[error("parquet error: {0}")]
1060    ParquetError(Arc<str>),
1061    #[error("reqwest error: {0}")]
1062    Reqwest(Arc<str>),
1063    #[error("aws s3 request error: {0}")]
1064    AwsS3Request(String),
1065    #[error("aws s3 bytestream error: {0}")]
1066    AwsS3Bytes(Arc<str>),
1067    #[error("invalid reqwest header: {0}")]
1068    InvalidHeader(Arc<str>),
1069    #[error("failed to decode Row from a record batch: {0}")]
1070    InvalidRecordBatch(Arc<str>),
1071    #[error("programming error: {0}")]
1072    ProgrammingError(Arc<str>),
1073    #[error("failed to get the size of an object")]
1074    MissingSize,
1075    #[error("object is missing the required '{0}' field")]
1076    MissingField(Arc<str>),
1077    #[error("failed while evaluating the provided mfp: '{0}'")]
1078    MfpEvalError(Arc<str>),
1079    #[error("no matching files found at the given url")]
1080    NoMatchingFiles,
1081    #[error("server returned HTTP {0}; Redirects are not supported, use the final URL directly.")]
1082    Redirect(u16),
1083    #[error("something went wrong: {0}")]
1084    Generic(String),
1085}
1086
1087impl From<csv_async::Error> for StorageErrorXKind {
1088    fn from(err: csv_async::Error) -> Self {
1089        StorageErrorXKind::CsvDecoding(err.to_string().into())
1090    }
1091}
1092
1093impl From<reqwest::Error> for StorageErrorXKind {
1094    fn from(err: reqwest::Error) -> Self {
1095        // Walk the source chain so that inner causes (like the custom DNS
1096        // resolver's "Address resolved to a private IP" rejection) are
1097        // visible. reqwest's top-level Display only says "error sending
1098        // request for url ..." and hides the underlying cause.
1099        StorageErrorXKind::Reqwest(err.to_string_with_causes().into())
1100    }
1101}
1102
1103impl From<reqwest::header::ToStrError> for StorageErrorXKind {
1104    fn from(err: reqwest::header::ToStrError) -> Self {
1105        StorageErrorXKind::InvalidHeader(err.to_string().into())
1106    }
1107}
1108
1109impl From<aws_smithy_types::byte_stream::error::Error> for StorageErrorXKind {
1110    fn from(err: aws_smithy_types::byte_stream::error::Error) -> Self {
1111        StorageErrorXKind::AwsS3Bytes(DisplayErrorContext(err).to_string().into())
1112    }
1113}
1114
1115impl From<::parquet::errors::ParquetError> for StorageErrorXKind {
1116    fn from(err: ::parquet::errors::ParquetError) -> Self {
1117        StorageErrorXKind::ParquetError(err.to_string().into())
1118    }
1119}
1120
1121impl StorageErrorXKind {
1122    pub fn with_context<C: Display>(self, context: C) -> StorageErrorX {
1123        StorageErrorX {
1124            kind: self,
1125            context: LinkedList::from([context.to_string()]),
1126        }
1127    }
1128
1129    pub fn invalid_record_batch<S: Into<Arc<str>>>(error: S) -> StorageErrorXKind {
1130        StorageErrorXKind::InvalidRecordBatch(error.into())
1131    }
1132
1133    pub fn generic<C: Display>(error: C) -> StorageErrorXKind {
1134        StorageErrorXKind::Generic(error.to_string())
1135    }
1136
1137    pub fn programming_error<S: Into<Arc<str>>>(error: S) -> StorageErrorXKind {
1138        StorageErrorXKind::ProgrammingError(error.into())
1139    }
1140}
1141
1142impl<E> From<E> for StorageErrorX
1143where
1144    E: Into<StorageErrorXKind>,
1145{
1146    fn from(err: E) -> Self {
1147        StorageErrorX {
1148            kind: err.into(),
1149            context: LinkedList::new(),
1150        }
1151    }
1152}
1153
1154trait StorageErrorXContext<T> {
1155    fn context<C>(self, context: C) -> Result<T, StorageErrorX>
1156    where
1157        C: Display;
1158}
1159
1160impl<T, E> StorageErrorXContext<T> for Result<T, E>
1161where
1162    E: Into<StorageErrorXKind>,
1163{
1164    fn context<C>(self, context: C) -> Result<T, StorageErrorX>
1165    where
1166        C: Display,
1167    {
1168        match self {
1169            Ok(val) => Ok(val),
1170            Err(kind) => Err(StorageErrorX {
1171                kind: kind.into(),
1172                context: LinkedList::from([context.to_string()]),
1173            }),
1174        }
1175    }
1176}
1177
1178impl<T> StorageErrorXContext<T> for Result<T, StorageErrorX> {
1179    fn context<C>(self, context: C) -> Result<T, StorageErrorX>
1180    where
1181        C: Display,
1182    {
1183        match self {
1184            Ok(val) => Ok(val),
1185            Err(mut e) => {
1186                e.context.push_back(context.to_string());
1187                Err(e)
1188            }
1189        }
1190    }
1191}