1use std::fmt;
66use std::sync::Arc;
67
68use bytes::Bytes;
69use differential_dataflow::Hashable;
70use futures::stream::BoxStream;
71use futures::{StreamExt, TryStreamExt};
72use mz_expr::SafeMfpPlan;
73use mz_ore::cast::CastFrom;
74use mz_persist_client::Diagnostics;
75use mz_persist_client::batch::ProtoBatch;
76use mz_persist_client::cache::PersistClientCache;
77use mz_persist_types::codec_impls::UnitSchema;
78use mz_repr::{DatumVec, GlobalId, Row, RowArena, Timestamp};
79use mz_storage_types::StorageDiff;
80use mz_storage_types::connections::ConnectionContext;
81use mz_storage_types::controller::CollectionMetadata;
82use mz_storage_types::oneshot_sources::{
83 ContentFilter, ContentFormat, ContentSource, OneshotIngestionRequest,
84};
85use mz_storage_types::sources::SourceData;
86use mz_timely_util::builder_async::{
87 Event as AsyncEvent, OperatorBuilder as AsyncOperatorBuilder, PressOnDropButton,
88};
89use mz_timely_util::pact::Distribute;
90use serde::de::DeserializeOwned;
91use serde::{Deserialize, Serialize};
92use std::collections::{BTreeSet, LinkedList};
93use std::fmt::{Debug, Display};
94use std::future::Future;
95use timely::container::CapacityContainerBuilder;
96use timely::dataflow::channels::pact::Pipeline;
97use timely::dataflow::{Scope, Stream as TimelyStream};
98use timely::progress::Antichain;
99use tracing::info;
100
101use crate::oneshot_source::aws_source::{AwsS3Source, S3Checksum, S3Object};
102use crate::oneshot_source::csv::{CsvDecoder, CsvRecord, CsvWorkRequest};
103use crate::oneshot_source::http_source::{HttpChecksum, HttpObject, HttpOneshotSource};
104use crate::oneshot_source::parquet::{ParquetFormat, ParquetRowGroup, ParquetWorkRequest};
105
106pub mod csv;
107pub mod parquet;
108
109pub mod aws_source;
110pub mod http_source;
111
112mod util;
113
114pub fn render<G, F>(
132 scope: G,
133 persist_clients: Arc<PersistClientCache>,
134 connection_context: ConnectionContext,
135 collection_id: GlobalId,
136 collection_meta: CollectionMetadata,
137 request: OneshotIngestionRequest,
138 worker_callback: F,
139) -> Vec<PressOnDropButton>
140where
141 G: Scope<Timestamp = Timestamp>,
142 F: FnOnce(Result<Option<ProtoBatch>, String>) -> () + 'static,
143{
144 let OneshotIngestionRequest {
145 source,
146 format,
147 filter,
148 shape,
149 } = request;
150
151 let source = match source {
152 ContentSource::Http { url } => {
153 let source = HttpOneshotSource::new(reqwest::Client::default(), url);
154 SourceKind::Http(source)
155 }
156 ContentSource::AwsS3 {
157 connection,
158 connection_id,
159 uri,
160 } => {
161 let source = AwsS3Source::new(connection, connection_id, connection_context, uri);
162 SourceKind::AwsS3(source)
163 }
164 };
165 tracing::info!(?source, "created oneshot source");
166
167 let format = match format {
168 ContentFormat::Csv(params) => {
169 let format = CsvDecoder::new(params, &shape.source_desc);
170 FormatKind::Csv(format)
171 }
172 ContentFormat::Parquet => {
173 let format = ParquetFormat::new(shape.source_desc);
174 FormatKind::Parquet(format)
175 }
176 };
177
178 let (objects_stream, discover_token) =
180 render_discover_objects(scope.clone(), collection_id, source.clone(), filter);
181 let (work_stream, split_token) = render_split_work(
183 scope.clone(),
184 collection_id,
185 &objects_stream,
186 source.clone(),
187 format.clone(),
188 );
189 let (records_stream, fetch_token) = render_fetch_work(
191 scope.clone(),
192 collection_id,
193 source.clone(),
194 format.clone(),
195 &work_stream,
196 );
197 let (rows_stream, decode_token) = render_decode_chunk(
199 scope.clone(),
200 format.clone(),
201 &records_stream,
202 shape.source_mfp,
203 );
204 let (batch_stream, batch_token) = render_stage_batches_operator(
206 scope.clone(),
207 collection_id,
208 &collection_meta,
209 persist_clients,
210 &rows_stream,
211 );
212
213 render_completion_operator(scope, &batch_stream, worker_callback);
215
216 let tokens = vec![
217 discover_token,
218 split_token,
219 fetch_token,
220 decode_token,
221 batch_token,
222 ];
223
224 tokens
225}
226
227pub fn render_discover_objects<G, S>(
230 scope: G,
231 collection_id: GlobalId,
232 source: S,
233 filter: ContentFilter,
234) -> (
235 TimelyStream<G, Result<(S::Object, S::Checksum), StorageErrorX>>,
236 PressOnDropButton,
237)
238where
239 G: Scope<Timestamp = Timestamp>,
240 S: OneshotSource + 'static,
241{
242 let worker_id = scope.index();
244 let num_workers = scope.peers();
245 let active_worker_id = usize::cast_from((collection_id, "discover").hashed()) % num_workers;
246 let is_active_worker = worker_id == active_worker_id;
247
248 let mut builder = AsyncOperatorBuilder::new("CopyFrom-discover".to_string(), scope.clone());
249
250 let (start_handle, start_stream) = builder.new_output::<CapacityContainerBuilder<_>>();
251
252 let shutdown = builder.build(move |caps| async move {
253 let [start_cap] = caps.try_into().unwrap();
254
255 if !is_active_worker {
256 return;
257 }
258
259 let filter = match ObjectFilter::try_new(filter) {
260 Ok(filter) => filter,
261 Err(err) => {
262 tracing::warn!(?err, "failed to create filter");
263 start_handle.give(&start_cap, Err(StorageErrorXKind::generic(err).into()));
264 return;
265 }
266 };
267
268 let work = source.list().await.context("list");
269 match work {
270 Ok(objects) => {
271 let (include, exclude): (Vec<_>, Vec<_>) = objects
272 .into_iter()
273 .partition(|(o, _checksum)| filter.filter::<S>(o));
274 tracing::info!(%worker_id, ?include, ?exclude, "listed objects");
275
276 include
277 .into_iter()
278 .for_each(|object| start_handle.give(&start_cap, Ok(object)))
279 }
280 Err(err) => {
281 tracing::warn!(?err, "failed to list oneshot source");
282 start_handle.give(&start_cap, Err(err))
283 }
284 }
285 });
286
287 (start_stream, shutdown.press_on_drop())
288}
289
290pub fn render_split_work<G, S, F>(
293 scope: G,
294 collection_id: GlobalId,
295 objects: &TimelyStream<G, Result<(S::Object, S::Checksum), StorageErrorX>>,
296 source: S,
297 format: F,
298) -> (
299 TimelyStream<G, Result<F::WorkRequest<S>, StorageErrorX>>,
300 PressOnDropButton,
301)
302where
303 G: Scope,
304 S: OneshotSource + Send + Sync + 'static,
305 F: OneshotFormat + Send + Sync + 'static,
306{
307 let worker_id = scope.index();
308 let mut builder = AsyncOperatorBuilder::new("CopyFrom-split_work".to_string(), scope.clone());
309
310 let (request_handle, request_stream) = builder.new_output::<CapacityContainerBuilder<_>>();
311 let mut objects_handle = builder.new_input_for(objects, Distribute, &request_handle);
312
313 let shutdown = builder.build(move |caps| async move {
314 let [_objects_cap] = caps.try_into().unwrap();
315
316 info!(%collection_id, %worker_id, "CopyFrom Split Work");
317
318 while let Some(event) = objects_handle.next().await {
319 let (capability, maybe_objects) = match event {
320 AsyncEvent::Data(cap, req) => (cap, req),
321 AsyncEvent::Progress(_) => continue,
322 };
323
324 let result = async {
327 let mut requests = Vec::new();
328
329 for maybe_object in maybe_objects {
330 let (object, checksum) = maybe_object?;
332
333 let format_ = format.clone();
334 let source_ = source.clone();
335 let work_requests = mz_ore::task::spawn(|| "split-work", async move {
336 info!(%worker_id, object = %object.name(), "splitting object");
337 format_.split_work(source_.clone(), object, checksum).await
338 })
339 .await
340 .expect("failed to spawn task")?;
341
342 requests.extend(work_requests);
343 }
344
345 Ok::<_, StorageErrorX>(requests)
346 }
347 .await
348 .context("split");
349
350 match result {
351 Ok(requests) => requests
352 .into_iter()
353 .for_each(|req| request_handle.give(&capability, Ok(req))),
354 Err(err) => request_handle.give(&capability, Err(err)),
355 }
356 }
357 });
358
359 (request_stream, shutdown.press_on_drop())
360}
361
362pub fn render_fetch_work<G, S, F>(
366 scope: G,
367 collection_id: GlobalId,
368 source: S,
369 format: F,
370 work_requests: &TimelyStream<G, Result<F::WorkRequest<S>, StorageErrorX>>,
371) -> (
372 TimelyStream<G, Result<F::RecordChunk, StorageErrorX>>,
373 PressOnDropButton,
374)
375where
376 G: Scope,
377 S: OneshotSource + Sync + 'static,
378 F: OneshotFormat + Sync + 'static,
379{
380 let worker_id = scope.index();
381 let mut builder = AsyncOperatorBuilder::new("CopyFrom-fetch_work".to_string(), scope.clone());
382
383 let (record_handle, record_stream) = builder.new_output::<CapacityContainerBuilder<_>>();
384 let mut work_requests_handle = builder.new_input_for(work_requests, Distribute, &record_handle);
385
386 let shutdown = builder.build(move |caps| async move {
387 let [_work_cap] = caps.try_into().unwrap();
388
389 info!(%collection_id, %worker_id, "CopyFrom Fetch Work");
390
391 while let Some(event) = work_requests_handle.next().await {
392 let (capability, maybe_requests) = match event {
393 AsyncEvent::Data(cap, req) => (cap, req),
394 AsyncEvent::Progress(_) => continue,
395 };
396
397 let result = async {
399 for maybe_request in maybe_requests {
401 let request = maybe_request?;
402
403 let mut work_stream = format.fetch_work(&source, request);
404 while let Some(result) = work_stream.next().await {
405 let record_chunk = result.context("fetch worker")?;
407
408 record_handle.give(&capability, Ok(record_chunk));
413 }
414 }
415
416 Ok::<_, StorageErrorX>(())
417 }
418 .await
419 .context("fetch work");
420
421 if let Err(err) = result {
422 tracing::warn!(?err, "failed to fetch");
423 record_handle.give(&capability, Err(err))
424 }
425 }
426 });
427
428 (record_stream, shutdown.press_on_drop())
429}
430
431pub fn render_decode_chunk<G, F>(
434 scope: G,
435 format: F,
436 record_chunks: &TimelyStream<G, Result<F::RecordChunk, StorageErrorX>>,
437 mfp: SafeMfpPlan,
438) -> (
439 TimelyStream<G, Result<Row, StorageErrorX>>,
440 PressOnDropButton,
441)
442where
443 G: Scope,
444 F: OneshotFormat + 'static,
445{
446 let mut builder = AsyncOperatorBuilder::new("CopyFrom-decode_chunk".to_string(), scope.clone());
447
448 let (row_handle, row_stream) = builder.new_output::<CapacityContainerBuilder<_>>();
449 let mut record_chunk_handle = builder.new_input_for(record_chunks, Distribute, &row_handle);
450
451 let shutdown = builder.build(move |caps| async move {
452 let [_row_cap] = caps.try_into().unwrap();
453
454 let mut datum_vec = DatumVec::default();
455 let row_arena = RowArena::default();
456 let mut row_buf = Row::default();
457
458 while let Some(event) = record_chunk_handle.next().await {
459 let (capability, maybe_chunks) = match event {
460 AsyncEvent::Data(cap, data) => (cap, data),
461 AsyncEvent::Progress(_) => continue,
462 };
463
464 let result = async {
465 let mut rows = Vec::new();
466 for maybe_chunk in maybe_chunks {
467 let chunk = maybe_chunk?;
468 format.decode_chunk(chunk, &mut rows)?;
469 }
470 Ok::<_, StorageErrorX>(rows)
471 }
472 .await
473 .context("decode chunk");
474
475 match result {
476 Ok(rows) => {
477 for row in rows {
480 let mut datums = datum_vec.borrow_with(&row);
481 let result = mfp
482 .evaluate_into(&mut *datums, &row_arena, &mut row_buf)
483 .map(|row| row.cloned());
484
485 match result {
486 Ok(Some(row)) => row_handle.give(&capability, Ok(row)),
487 Ok(None) => {
488 mz_ore::soft_panic_or_log!("oneshot source MFP filtered out data!");
491 }
492 Err(e) => {
493 let err = StorageErrorXKind::MfpEvalError(e.to_string().into())
494 .with_context("decode");
495 row_handle.give(&capability, Err(err))
496 }
497 }
498 }
499 }
500 Err(err) => row_handle.give(&capability, Err(err)),
501 }
502 }
503 });
504
505 (row_stream, shutdown.press_on_drop())
506}
507
508pub fn render_stage_batches_operator<G>(
511 scope: G,
512 collection_id: GlobalId,
513 collection_meta: &CollectionMetadata,
514 persist_clients: Arc<PersistClientCache>,
515 rows_stream: &TimelyStream<G, Result<Row, StorageErrorX>>,
516) -> (
517 TimelyStream<G, Result<ProtoBatch, StorageErrorX>>,
518 PressOnDropButton,
519)
520where
521 G: Scope,
522{
523 let persist_location = collection_meta.persist_location.clone();
524 let shard_id = collection_meta.data_shard;
525 let collection_desc = collection_meta.relation_desc.clone();
526
527 let mut builder =
528 AsyncOperatorBuilder::new("CopyFrom-stage_batches".to_string(), scope.clone());
529
530 let (proto_batch_handle, proto_batch_stream) =
531 builder.new_output::<CapacityContainerBuilder<_>>();
532 let mut rows_handle = builder.new_input_for(rows_stream, Pipeline, &proto_batch_handle);
533
534 let shutdown = builder.build(move |caps| async move {
535 let [proto_batch_cap] = caps.try_into().unwrap();
536
537 let persist_client = persist_clients
539 .open(persist_location)
540 .await
541 .expect("failed to open Persist client");
542 let persist_diagnostics = Diagnostics {
543 shard_name: collection_id.to_string(),
544 handle_purpose: "CopyFrom::stage_batches".to_string(),
545 };
546 let write_handle = persist_client
547 .open_writer::<SourceData, (), mz_repr::Timestamp, StorageDiff>(
548 shard_id,
549 Arc::new(collection_desc),
550 Arc::new(UnitSchema),
551 persist_diagnostics,
552 )
553 .await
554 .expect("could not open Persist shard");
555
556 let lower = mz_repr::Timestamp::MIN;
560 let upper = Antichain::from_elem(lower.step_forward());
561
562 let mut batch_builder = write_handle.builder(Antichain::from_elem(lower));
563
564 while let Some(event) = rows_handle.next().await {
565 let AsyncEvent::Data(_, row_batch) = event else {
566 continue;
567 };
568
569 for maybe_row in row_batch {
571 match maybe_row {
572 Ok(row) => {
574 let data = SourceData(Ok(row));
575 batch_builder
576 .add(&data, &(), &lower, &1)
577 .await
578 .expect("failed to add Row to batch");
579 }
580 Err(err) => {
582 let batch = batch_builder
584 .finish(upper)
585 .await
586 .expect("failed to cleanup batch");
587 batch.delete().await;
588
589 proto_batch_handle
591 .give(&proto_batch_cap, Err(err).context("stage batches"));
592 return;
593 }
594 }
595 }
596 }
597
598 let batch = batch_builder
599 .finish(upper)
600 .await
601 .expect("failed to create Batch");
602
603 let proto_batch = batch.into_transmittable_batch();
612 proto_batch_handle.give(&proto_batch_cap, Ok(proto_batch));
613 });
614
615 (proto_batch_stream, shutdown.press_on_drop())
616}
617
618pub fn render_completion_operator<G, F>(
621 scope: G,
622 results_stream: &TimelyStream<G, Result<ProtoBatch, StorageErrorX>>,
623 worker_callback: F,
624) where
625 G: Scope,
626 F: FnOnce(Result<Option<ProtoBatch>, String>) -> () + 'static,
627{
628 let mut builder = AsyncOperatorBuilder::new("CopyFrom-completion".to_string(), scope.clone());
629 let mut results_input = builder.new_disconnected_input(results_stream, Pipeline);
630
631 builder.build(move |_| async move {
632 let result = async move {
633 let mut maybe_payload: Option<ProtoBatch> = None;
634
635 while let Some(event) = results_input.next().await {
636 if let AsyncEvent::Data(_cap, results) = event {
637 let [result] = results
638 .try_into()
639 .expect("only 1 event on the result stream");
640
641 if maybe_payload.is_some() {
643 panic!("expected only one batch!");
644 }
645
646 maybe_payload = Some(result.map_err(|e| e.to_string())?);
647 }
648 }
649
650 Ok(maybe_payload)
651 }
652 .await;
653
654 worker_callback(result);
656 });
657}
658
659pub trait OneshotObject {
661 fn name(&self) -> &str;
663
664 fn path(&self) -> &str;
666
667 fn size(&self) -> usize;
669
670 fn encodings(&self) -> &[Encoding];
676}
677
678#[derive(Debug, Copy, Clone, Serialize, Deserialize)]
680pub enum Encoding {
681 Bzip2,
682 Gzip,
683 Xz,
684 Zstd,
685}
686
687pub trait OneshotSource: Clone + Send + Unpin {
689 type Object: OneshotObject
691 + Debug
692 + Clone
693 + Send
694 + Unpin
695 + Serialize
696 + DeserializeOwned
697 + 'static;
698 type Checksum: Debug + Clone + Send + Unpin + Serialize + DeserializeOwned + 'static;
700
701 fn list<'a>(
703 &'a self,
704 ) -> impl Future<Output = Result<Vec<(Self::Object, Self::Checksum)>, StorageErrorX>> + Send;
705
706 fn get<'s>(
708 &'s self,
709 object: Self::Object,
710 checksum: Self::Checksum,
711 range: Option<std::ops::RangeInclusive<usize>>,
712 ) -> BoxStream<'s, Result<Bytes, StorageErrorX>>;
713}
714
715#[derive(Clone, Debug)]
722pub(crate) enum SourceKind {
723 Http(HttpOneshotSource),
724 AwsS3(AwsS3Source),
725}
726
727impl OneshotSource for SourceKind {
728 type Object = ObjectKind;
729 type Checksum = ChecksumKind;
730
731 async fn list<'a>(&'a self) -> Result<Vec<(Self::Object, Self::Checksum)>, StorageErrorX> {
732 match self {
733 SourceKind::Http(http) => {
734 let objects = http.list().await.context("http")?;
735 let objects = objects
736 .into_iter()
737 .map(|(object, checksum)| {
738 (ObjectKind::Http(object), ChecksumKind::Http(checksum))
739 })
740 .collect();
741 Ok(objects)
742 }
743 SourceKind::AwsS3(s3) => {
744 let objects = s3.list().await.context("s3")?;
745 let objects = objects
746 .into_iter()
747 .map(|(object, checksum)| {
748 (ObjectKind::AwsS3(object), ChecksumKind::AwsS3(checksum))
749 })
750 .collect();
751 Ok(objects)
752 }
753 }
754 }
755
756 fn get<'s>(
757 &'s self,
758 object: Self::Object,
759 checksum: Self::Checksum,
760 range: Option<std::ops::RangeInclusive<usize>>,
761 ) -> BoxStream<'s, Result<Bytes, StorageErrorX>> {
762 match (self, object, checksum) {
763 (SourceKind::Http(http), ObjectKind::Http(object), ChecksumKind::Http(checksum)) => {
764 http.get(object, checksum, range)
765 .map(|result| result.context("http"))
766 .boxed()
767 }
768 (SourceKind::AwsS3(s3), ObjectKind::AwsS3(object), ChecksumKind::AwsS3(checksum)) => s3
769 .get(object, checksum, range)
770 .map(|result| result.context("aws_s3"))
771 .boxed(),
772 (SourceKind::AwsS3(_) | SourceKind::Http(_), _, _) => {
773 unreachable!("programming error! wrong source, object, and checksum kind");
774 }
775 }
776 }
777}
778
779#[derive(Debug, Clone, Serialize, Deserialize)]
781pub(crate) enum ObjectKind {
782 Http(HttpObject),
783 AwsS3(S3Object),
784}
785
786impl OneshotObject for ObjectKind {
787 fn name(&self) -> &str {
788 match self {
789 ObjectKind::Http(object) => object.name(),
790 ObjectKind::AwsS3(object) => object.name(),
791 }
792 }
793
794 fn path(&self) -> &str {
795 match self {
796 ObjectKind::Http(object) => object.path(),
797 ObjectKind::AwsS3(object) => object.path(),
798 }
799 }
800
801 fn size(&self) -> usize {
802 match self {
803 ObjectKind::Http(object) => object.size(),
804 ObjectKind::AwsS3(object) => object.size(),
805 }
806 }
807
808 fn encodings(&self) -> &[Encoding] {
809 match self {
810 ObjectKind::Http(object) => object.encodings(),
811 ObjectKind::AwsS3(object) => object.encodings(),
812 }
813 }
814}
815
816#[derive(Debug, Clone, Serialize, Deserialize)]
818pub(crate) enum ChecksumKind {
819 Http(HttpChecksum),
820 AwsS3(S3Checksum),
821}
822
823pub trait OneshotFormat: Clone {
825 type WorkRequest<S>: Debug + Clone + Send + Serialize + DeserializeOwned + 'static
827 where
828 S: OneshotSource;
829 type RecordChunk: Debug + Clone + Send + Serialize + DeserializeOwned + 'static;
831
832 fn split_work<S: OneshotSource + Send>(
837 &self,
838 source: S,
839 object: S::Object,
840 checksum: S::Checksum,
841 ) -> impl Future<Output = Result<Vec<Self::WorkRequest<S>>, StorageErrorX>> + Send;
842
843 fn fetch_work<'a, S: OneshotSource + Sync + 'static>(
846 &'a self,
847 source: &'a S,
848 request: Self::WorkRequest<S>,
849 ) -> BoxStream<'a, Result<Self::RecordChunk, StorageErrorX>>;
850
851 fn decode_chunk(
853 &self,
854 chunk: Self::RecordChunk,
855 rows: &mut Vec<Row>,
856 ) -> Result<usize, StorageErrorX>;
857}
858
859#[derive(Clone, Debug)]
866pub(crate) enum FormatKind {
867 Csv(CsvDecoder),
868 Parquet(ParquetFormat),
869}
870
871impl OneshotFormat for FormatKind {
872 type WorkRequest<S>
873 = RequestKind<S::Object, S::Checksum>
874 where
875 S: OneshotSource;
876 type RecordChunk = RecordChunkKind;
877
878 async fn split_work<S: OneshotSource + Send>(
879 &self,
880 source: S,
881 object: S::Object,
882 checksum: S::Checksum,
883 ) -> Result<Vec<Self::WorkRequest<S>>, StorageErrorX> {
884 match self {
885 FormatKind::Csv(csv) => {
886 let work = csv
887 .split_work(source, object, checksum)
888 .await
889 .context("csv")?
890 .into_iter()
891 .map(RequestKind::Csv)
892 .collect();
893 Ok(work)
894 }
895 FormatKind::Parquet(parquet) => {
896 let work = parquet
897 .split_work(source, object, checksum)
898 .await
899 .context("parquet")?
900 .into_iter()
901 .map(RequestKind::Parquet)
902 .collect();
903 Ok(work)
904 }
905 }
906 }
907
908 fn fetch_work<'a, S: OneshotSource + Sync + 'static>(
909 &'a self,
910 source: &'a S,
911 request: Self::WorkRequest<S>,
912 ) -> BoxStream<'a, Result<Self::RecordChunk, StorageErrorX>> {
913 match (self, request) {
914 (FormatKind::Csv(csv), RequestKind::Csv(request)) => csv
915 .fetch_work(source, request)
916 .map_ok(RecordChunkKind::Csv)
917 .map(|result| result.context("csv"))
918 .boxed(),
919 (FormatKind::Parquet(parquet), RequestKind::Parquet(request)) => parquet
920 .fetch_work(source, request)
921 .map_ok(RecordChunkKind::Parquet)
922 .map(|result| result.context("parquet"))
923 .boxed(),
924 (FormatKind::Parquet(_), RequestKind::Csv(_))
925 | (FormatKind::Csv(_), RequestKind::Parquet(_)) => {
926 unreachable!("programming error, {self:?}")
927 }
928 }
929 }
930
931 fn decode_chunk(
932 &self,
933 chunk: Self::RecordChunk,
934 rows: &mut Vec<Row>,
935 ) -> Result<usize, StorageErrorX> {
936 match (self, chunk) {
937 (FormatKind::Csv(csv), RecordChunkKind::Csv(chunk)) => {
938 csv.decode_chunk(chunk, rows).context("csv")
939 }
940 (FormatKind::Parquet(parquet), RecordChunkKind::Parquet(chunk)) => {
941 parquet.decode_chunk(chunk, rows).context("parquet")
942 }
943 (FormatKind::Parquet(_), RecordChunkKind::Csv(_))
944 | (FormatKind::Csv(_), RecordChunkKind::Parquet(_)) => {
945 unreachable!("programming error, {self:?}")
946 }
947 }
948 }
949}
950
951#[derive(Clone, Debug, Serialize, Deserialize)]
952pub(crate) enum RequestKind<O, C> {
953 Csv(CsvWorkRequest<O, C>),
954 Parquet(ParquetWorkRequest<O, C>),
955}
956
957#[derive(Clone, Debug, Serialize, Deserialize)]
958pub(crate) enum RecordChunkKind {
959 Csv(CsvRecord),
960 Parquet(ParquetRowGroup),
961}
962
963pub(crate) enum ObjectFilter {
964 None,
965 Files(BTreeSet<Box<str>>),
966 Pattern(glob::Pattern),
967}
968
969impl ObjectFilter {
970 pub fn try_new(filter: ContentFilter) -> Result<Self, anyhow::Error> {
971 match filter {
972 ContentFilter::None => Ok(ObjectFilter::None),
973 ContentFilter::Files(files) => {
974 let files = files.into_iter().map(|f| f.into()).collect();
975 Ok(ObjectFilter::Files(files))
976 }
977 ContentFilter::Pattern(pattern) => {
978 let pattern = glob::Pattern::new(&pattern)?;
979 Ok(ObjectFilter::Pattern(pattern))
980 }
981 }
982 }
983
984 pub fn filter<S: OneshotSource>(&self, object: &S::Object) -> bool {
986 match self {
987 ObjectFilter::None => true,
988 ObjectFilter::Files(files) => files.contains(object.path()),
989 ObjectFilter::Pattern(pattern) => pattern.matches(object.path()),
990 }
991 }
992}
993
994#[derive(Debug, Clone, Deserialize, Serialize)]
1001pub struct StorageErrorX {
1002 kind: StorageErrorXKind,
1003 context: LinkedList<String>,
1004}
1005
1006impl fmt::Display for StorageErrorX {
1007 fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
1008 writeln!(f, "error: {}", self.kind)?;
1009 writeln!(f, "causes: {:?}", self.context)?;
1010 Ok(())
1011 }
1012}
1013
1014#[derive(Debug, Clone, Serialize, Deserialize, thiserror::Error)]
1016pub enum StorageErrorXKind {
1017 #[error("csv decoding error: {0}")]
1018 CsvDecoding(Arc<str>),
1019 #[error("parquet error: {0}")]
1020 ParquetError(Arc<str>),
1021 #[error("reqwest error: {0}")]
1022 Reqwest(Arc<str>),
1023 #[error("aws s3 request error: {0}")]
1024 AwsS3Request(String),
1025 #[error("aws s3 bytestream error: {0}")]
1026 AwsS3Bytes(Arc<str>),
1027 #[error("invalid reqwest header: {0}")]
1028 InvalidHeader(Arc<str>),
1029 #[error("failed to decode Row from a record batch: {0}")]
1030 InvalidRecordBatch(Arc<str>),
1031 #[error("programming error: {0}")]
1032 ProgrammingError(Arc<str>),
1033 #[error("failed to get the size of an object")]
1034 MissingSize,
1035 #[error("object is missing the required '{0}' field")]
1036 MissingField(Arc<str>),
1037 #[error("failed while evaluating the provided mfp: '{0}'")]
1038 MfpEvalError(Arc<str>),
1039 #[error("something went wrong: {0}")]
1040 Generic(String),
1041}
1042
1043impl From<csv_async::Error> for StorageErrorXKind {
1044 fn from(err: csv_async::Error) -> Self {
1045 StorageErrorXKind::CsvDecoding(err.to_string().into())
1046 }
1047}
1048
1049impl From<reqwest::Error> for StorageErrorXKind {
1050 fn from(err: reqwest::Error) -> Self {
1051 StorageErrorXKind::Reqwest(err.to_string().into())
1052 }
1053}
1054
1055impl From<reqwest::header::ToStrError> for StorageErrorXKind {
1056 fn from(err: reqwest::header::ToStrError) -> Self {
1057 StorageErrorXKind::InvalidHeader(err.to_string().into())
1058 }
1059}
1060
1061impl From<aws_smithy_types::byte_stream::error::Error> for StorageErrorXKind {
1062 fn from(err: aws_smithy_types::byte_stream::error::Error) -> Self {
1063 StorageErrorXKind::AwsS3Request(err.to_string())
1064 }
1065}
1066
1067impl From<::parquet::errors::ParquetError> for StorageErrorXKind {
1068 fn from(err: ::parquet::errors::ParquetError) -> Self {
1069 StorageErrorXKind::ParquetError(err.to_string().into())
1070 }
1071}
1072
1073impl StorageErrorXKind {
1074 pub fn with_context<C: Display>(self, context: C) -> StorageErrorX {
1075 StorageErrorX {
1076 kind: self,
1077 context: LinkedList::from([context.to_string()]),
1078 }
1079 }
1080
1081 pub fn invalid_record_batch<S: Into<Arc<str>>>(error: S) -> StorageErrorXKind {
1082 StorageErrorXKind::InvalidRecordBatch(error.into())
1083 }
1084
1085 pub fn generic<C: Display>(error: C) -> StorageErrorXKind {
1086 StorageErrorXKind::Generic(error.to_string())
1087 }
1088
1089 pub fn programming_error<S: Into<Arc<str>>>(error: S) -> StorageErrorXKind {
1090 StorageErrorXKind::ProgrammingError(error.into())
1091 }
1092}
1093
1094impl<E> From<E> for StorageErrorX
1095where
1096 E: Into<StorageErrorXKind>,
1097{
1098 fn from(err: E) -> Self {
1099 StorageErrorX {
1100 kind: err.into(),
1101 context: LinkedList::new(),
1102 }
1103 }
1104}
1105
1106trait StorageErrorXContext<T> {
1107 fn context<C>(self, context: C) -> Result<T, StorageErrorX>
1108 where
1109 C: Display;
1110}
1111
1112impl<T, E> StorageErrorXContext<T> for Result<T, E>
1113where
1114 E: Into<StorageErrorXKind>,
1115{
1116 fn context<C>(self, context: C) -> Result<T, StorageErrorX>
1117 where
1118 C: Display,
1119 {
1120 match self {
1121 Ok(val) => Ok(val),
1122 Err(kind) => Err(StorageErrorX {
1123 kind: kind.into(),
1124 context: LinkedList::from([context.to_string()]),
1125 }),
1126 }
1127 }
1128}
1129
1130impl<T> StorageErrorXContext<T> for Result<T, StorageErrorX> {
1131 fn context<C>(self, context: C) -> Result<T, StorageErrorX>
1132 where
1133 C: Display,
1134 {
1135 match self {
1136 Ok(val) => Ok(val),
1137 Err(mut e) => {
1138 e.context.push_back(context.to_string());
1139 Err(e)
1140 }
1141 }
1142 }
1143}