-
-
Notifications
You must be signed in to change notification settings - Fork 162
Expand file tree
/
Copy pathgcs.rs
More file actions
853 lines (745 loc) · 28 KB
/
gcs.rs
File metadata and controls
853 lines (745 loc) · 28 KB
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
/*
* Parseable Server (C) 2022 - 2025 Parseable, Inc.
*
* This program is free software: you can redistribute it and/or modify
* it under the terms of the GNU Affero General Public License as
* published by the Free Software Foundation, either version 3 of the
* License, or (at your option) any later version.
*
* This program is distributed in the hope that it will be useful,
* but WITHOUT ANY WARRANTY; without even the implied warranty of
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
* GNU Affero General Public License for more details.
*
* You should have received a copy of the GNU Affero General Public License
* along with this program. If not, see <http://www.gnu.org/licenses/>.
*
*/
use std::{
collections::HashSet,
path::Path,
sync::{
Arc,
atomic::{AtomicU64, Ordering},
},
time::Duration,
};
use crate::{
metrics::{
increment_bytes_scanned_in_object_store_calls_by_date,
increment_files_scanned_in_object_store_calls_by_date,
increment_object_store_calls_by_date,
},
parseable::LogStream,
};
use async_trait::async_trait;
use bytes::Bytes;
use chrono::Utc;
use datafusion::{
config::{ConfigExtension, ExtensionOptions}, datasource::listing::ListingTableUrl, execution::{
object_store::{DefaultObjectStoreRegistry, ObjectStoreRegistry, ObjectStoreUrl},
runtime_env::RuntimeEnvBuilder,
}
};
use futures::{StreamExt, TryStreamExt, stream::FuturesUnordered};
use object_store::{
BackoffConfig, ClientOptions, ListResult, ObjectMeta, ObjectStore, PutPayload, RetryConfig,
buffered::BufReader,
gcp::{GoogleCloudStorage, GoogleCloudStorageBuilder},
limit::LimitStore,
path::Path as StorePath,
};
use relative_path::{RelativePath, RelativePathBuf};
use tokio::{fs::OpenOptions, io::AsyncReadExt};
use tracing::error;
use super::{
CONNECT_TIMEOUT_SECS, MIN_MULTIPART_UPLOAD_SIZE, ObjectStorage, ObjectStorageError,
ObjectStorageProvider, PARSEABLE_ROOT_DIRECTORY, REQUEST_TIMEOUT_SECS,
STREAM_METADATA_FILE_NAME, metrics_layer::MetricLayer, object_storage::parseable_json_path,
to_object_store_path,
};
#[derive(Debug, Clone, clap::Args)]
#[command(
name = "GCS config",
about = "Start Parseable with GCS or compatible as storage",
help_template = "\
{about-section}
{all-args}
"
)]
pub struct GcsConfig {
/// The endpoint to GCS or compatible object storage platform
#[arg(
long,
env = "P_GCS_URL",
value_name = "url",
default_value = "https://storage.googleapis.com",
required = false
)]
pub endpoint_url: String,
/// The GCS or compatible object storage bucket to be used for storage
#[arg(
long,
env = "P_GCS_BUCKET",
value_name = "bucket-name",
required = true
)]
pub bucket_name: String,
/// Set client to skip tls verification
#[arg(
long,
env = "P_GCS_TLS_SKIP_VERIFY",
value_name = "bool",
default_value = "false"
)]
pub skip_tls: bool,
}
impl GcsConfig {
fn get_default_builder(&self) -> GoogleCloudStorageBuilder {
let mut client_options = ClientOptions::default()
.with_allow_http(true)
.with_connect_timeout(Duration::from_secs(CONNECT_TIMEOUT_SECS))
.with_timeout(Duration::from_secs(REQUEST_TIMEOUT_SECS));
if self.skip_tls {
client_options = client_options.with_allow_invalid_certificates(true)
}
let retry_config = RetryConfig {
max_retries: 5,
retry_timeout: Duration::from_secs(30),
backoff: BackoffConfig::default(),
};
let builder = GoogleCloudStorageBuilder::from_env()
.with_bucket_name(&self.bucket_name)
.with_retry(retry_config);
builder.with_client_options(client_options)
}
}
impl ConfigExtension for GcsConfig {
const PREFIX: &'static str = "gcs";
}
impl ExtensionOptions for GcsConfig {
fn as_any(&self) -> &dyn std::any::Any {
todo!()
}
fn as_any_mut(&mut self) -> &mut dyn std::any::Any {
todo!()
}
fn cloned(&self) -> Box<dyn ExtensionOptions> {
todo!()
}
fn set(&mut self, _key: &str, _value: &str) -> datafusion::error::Result<()> {
todo!()
}
fn entries(&self) -> Vec<datafusion::config::ConfigEntry> {
todo!()
}
}
impl ObjectStorageProvider for GcsConfig {
fn name(&self) -> &'static str {
"gcs"
}
fn get_datafusion_runtime(&self) -> RuntimeEnvBuilder {
let gcs = self.get_default_builder().build().unwrap();
// limit objectstore to a concurrent request limit
let gcs = LimitStore::new(gcs, super::MAX_OBJECT_STORE_REQUESTS);
let gcs = MetricLayer::new(gcs, "gcs");
let object_store_registry = DefaultObjectStoreRegistry::new();
// Register GCS client under the "gs://" scheme so DataFusion can route
// object store calls to our GoogleCloudStorage implementation
let url = ObjectStoreUrl::parse(format!("gs://{}", &self.bucket_name)).unwrap();
object_store_registry.register_store(url.as_ref(), Arc::new(gcs));
RuntimeEnvBuilder::new().with_object_store_registry(Arc::new(object_store_registry))
}
fn construct_client(&self) -> Arc<dyn ObjectStorage> {
let gcs = self.get_default_builder().build().unwrap();
Arc::new(Gcs {
client: Arc::new(gcs),
bucket: self.bucket_name.clone(),
root: StorePath::from(""),
})
}
fn get_endpoint(&self) -> String {
format!("{}/{}", self.endpoint_url, self.bucket_name)
}
fn get_object_store(&self) -> Arc<dyn ObjectStorage> {
static STORE: once_cell::sync::OnceCell<Arc<dyn ObjectStorage>> =
once_cell::sync::OnceCell::new();
STORE.get_or_init(|| self.construct_client()).clone()
}
}
#[derive(Debug)]
pub struct Gcs {
client: Arc<GoogleCloudStorage>,
bucket: String,
root: StorePath,
}
impl Gcs {
async fn _get_object(&self, path: &RelativePath) -> Result<Bytes, ObjectStorageError> {
let resp = self.client.get(&to_object_store_path(path)).await;
increment_object_store_calls_by_date("GET", &Utc::now().date_naive().to_string());
match resp {
Ok(resp) => {
let body: Bytes = resp.bytes().await?;
increment_files_scanned_in_object_store_calls_by_date(
"GET",
1,
&Utc::now().date_naive().to_string(),
);
increment_bytes_scanned_in_object_store_calls_by_date(
"GET",
body.len() as u64,
&Utc::now().date_naive().to_string(),
);
Ok(body)
}
Err(err) => Err(err.into()),
}
}
async fn _put_object(
&self,
path: &RelativePath,
resource: PutPayload,
) -> Result<(), ObjectStorageError> {
let resp = self.client.put(&to_object_store_path(path), resource).await;
increment_object_store_calls_by_date("PUT", &Utc::now().date_naive().to_string());
match resp {
Ok(_) => {
increment_files_scanned_in_object_store_calls_by_date(
"PUT",
1,
&Utc::now().date_naive().to_string(),
);
Ok(())
}
Err(err) => Err(err.into()),
}
}
async fn _delete_prefix(&self, key: &str) -> Result<(), ObjectStorageError> {
let files_scanned = Arc::new(AtomicU64::new(0));
let files_deleted = Arc::new(AtomicU64::new(0));
// Track LIST operation
let object_stream = self.client.list(Some(&(key.into())));
increment_object_store_calls_by_date("LIST", &Utc::now().date_naive().to_string());
object_stream
.for_each_concurrent(None, |x| async {
files_scanned.fetch_add(1, Ordering::Relaxed);
match x {
Ok(obj) => {
files_deleted.fetch_add(1, Ordering::Relaxed);
let delete_resp = self.client.delete(&obj.location).await;
increment_object_store_calls_by_date(
"DELETE",
&Utc::now().date_naive().to_string(),
);
if delete_resp.is_err() {
error!(
"Failed to delete object during delete stream: {:?}",
delete_resp
);
}
}
Err(err) => {
error!("Failed to fetch object during delete stream: {:?}", err);
}
};
})
.await;
increment_files_scanned_in_object_store_calls_by_date(
"LIST",
files_scanned.load(Ordering::Relaxed),
&Utc::now().date_naive().to_string(),
);
increment_files_scanned_in_object_store_calls_by_date(
"DELETE",
files_deleted.load(Ordering::Relaxed),
&Utc::now().date_naive().to_string(),
);
Ok(())
}
async fn _list_dates(&self, stream: &str) -> Result<Vec<String>, ObjectStorageError> {
let resp: Result<object_store::ListResult, object_store::Error> = self
.client
.list_with_delimiter(Some(&(stream.into())))
.await;
increment_object_store_calls_by_date("LIST", &Utc::now().date_naive().to_string());
let resp = match resp {
Ok(resp) => resp,
Err(err) => {
return Err(err.into());
}
};
let common_prefixes = resp.common_prefixes;
increment_files_scanned_in_object_store_calls_by_date(
"LIST",
common_prefixes.len() as u64,
&Utc::now().date_naive().to_string(),
);
// return prefixes at the root level
let dates: Vec<_> = common_prefixes
.iter()
.filter_map(|path| path.as_ref().strip_prefix(&format!("{stream}/")))
.map(String::from)
.collect();
Ok(dates)
}
async fn _upload_file(&self, key: &str, path: &Path) -> Result<(), ObjectStorageError> {
let bytes = tokio::fs::read(path).await?;
let result = self.client.put(&key.into(), bytes.into()).await;
increment_object_store_calls_by_date("PUT", &Utc::now().date_naive().to_string());
match result {
Ok(_) => {
increment_files_scanned_in_object_store_calls_by_date(
"PUT",
1,
&Utc::now().date_naive().to_string(),
);
Ok(())
}
Err(err) => Err(err.into()),
}
}
async fn _upload_multipart(
&self,
key: &RelativePath,
path: &Path,
) -> Result<(), ObjectStorageError> {
let mut file = OpenOptions::new().read(true).open(path).await?;
let location = &to_object_store_path(key);
let async_writer = self.client.put_multipart(location).await;
let mut async_writer = match async_writer {
Ok(writer) => writer,
Err(err) => {
return Err(err.into());
}
};
let meta = file.metadata().await?;
let total_size = meta.len() as usize;
if total_size < MIN_MULTIPART_UPLOAD_SIZE {
let mut data = Vec::new();
file.read_to_end(&mut data).await?;
// Track single PUT operation for small files
let result = self.client.put(location, data.into()).await;
increment_object_store_calls_by_date("PUT", &Utc::now().date_naive().to_string());
match result {
Ok(_) => {
increment_files_scanned_in_object_store_calls_by_date(
"PUT",
1,
&Utc::now().date_naive().to_string(),
);
}
Err(err) => {
return Err(err.into());
}
}
return Ok(());
} else {
let mut data = Vec::new();
file.read_to_end(&mut data).await?;
let has_final_partial_part = !total_size.is_multiple_of(MIN_MULTIPART_UPLOAD_SIZE);
let num_full_parts = total_size / MIN_MULTIPART_UPLOAD_SIZE;
let total_parts = num_full_parts + if has_final_partial_part { 1 } else { 0 };
// Upload each part with metrics
for part_number in 0..(total_parts) {
let start_pos = part_number * MIN_MULTIPART_UPLOAD_SIZE;
let end_pos = if part_number == num_full_parts && has_final_partial_part {
// Last part might be smaller than 5MB (which is allowed)
total_size
} else {
// All other parts must be at least 5MB
start_pos + MIN_MULTIPART_UPLOAD_SIZE
};
// Extract this part's data
let part_data = data[start_pos..end_pos].to_vec();
// Track individual part upload
let result = async_writer.put_part(part_data.into()).await;
if result.is_err() {
return Err(result.err().unwrap().into());
}
increment_object_store_calls_by_date(
"PUT_MULTIPART",
&Utc::now().date_naive().to_string(),
);
}
// Track multipart completion
let complete_result = async_writer.complete().await;
if let Err(err) = complete_result {
if let Err(abort_err) = async_writer.abort().await {
error!(
"Failed to abort multipart upload after completion failure: {:?}",
abort_err
);
}
return Err(err.into());
}
}
Ok(())
}
}
#[async_trait]
impl ObjectStorage for Gcs {
async fn get_buffered_reader(
&self,
path: &RelativePath,
) -> Result<BufReader, ObjectStorageError> {
let path = &to_object_store_path(path);
let meta = self.client.head(path).await;
increment_object_store_calls_by_date("HEAD", &Utc::now().date_naive().to_string());
let meta = match meta {
Ok(meta) => {
increment_files_scanned_in_object_store_calls_by_date(
"HEAD",
1,
&Utc::now().date_naive().to_string(),
);
meta
}
Err(err) => {
return Err(err.into());
}
};
let store: Arc<dyn ObjectStore> = self.client.clone();
let buf = object_store::buffered::BufReader::new(store, &meta);
Ok(buf)
}
async fn upload_multipart(
&self,
key: &RelativePath,
path: &Path,
) -> Result<(), ObjectStorageError> {
self._upload_multipart(key, path).await
}
async fn head(&self, path: &RelativePath) -> Result<ObjectMeta, ObjectStorageError> {
let result = self.client.head(&to_object_store_path(path)).await;
increment_object_store_calls_by_date("HEAD", &Utc::now().date_naive().to_string());
if result.is_ok() {
increment_files_scanned_in_object_store_calls_by_date(
"HEAD",
1,
&Utc::now().date_naive().to_string(),
);
}
Ok(result?)
}
async fn get_object(&self, path: &RelativePath) -> Result<Bytes, ObjectStorageError> {
Ok(self._get_object(path).await?)
}
async fn get_objects(
&self,
base_path: Option<&RelativePath>,
filter_func: Box<dyn Fn(String) -> bool + Send>,
) -> Result<Vec<Bytes>, ObjectStorageError> {
let prefix = if let Some(base_path) = base_path {
to_object_store_path(base_path)
} else {
self.root.clone()
};
let mut list_stream = self.client.list(Some(&prefix));
let mut res = vec![];
let mut files_scanned = 0;
// Note: We track each streaming list item retrieval
while let Some(meta_result) = list_stream.next().await {
let meta = match meta_result {
Ok(meta) => meta,
Err(err) => {
return Err(err.into());
}
};
files_scanned += 1;
let ingestor_file = filter_func(meta.location.filename().unwrap().to_string());
if !ingestor_file {
continue;
}
let byts = self
.get_object(
RelativePath::from_path(meta.location.as_ref())
.map_err(ObjectStorageError::PathError)?,
)
.await?;
res.push(byts);
}
// Record total files scanned
increment_files_scanned_in_object_store_calls_by_date(
"LIST",
files_scanned as u64,
&Utc::now().date_naive().to_string(),
);
increment_object_store_calls_by_date("LIST", &Utc::now().date_naive().to_string());
Ok(res)
}
async fn get_ingestor_meta_file_paths(
&self,
) -> Result<Vec<RelativePathBuf>, ObjectStorageError> {
let mut path_arr = vec![];
let mut files_scanned = 0;
let mut object_stream = self.client.list(Some(&self.root));
increment_object_store_calls_by_date("LIST", &Utc::now().date_naive().to_string());
while let Some(meta_result) = object_stream.next().await {
let meta = match meta_result {
Ok(meta) => meta,
Err(err) => {
return Err(err.into());
}
};
files_scanned += 1;
let flag = meta.location.filename().unwrap().starts_with("ingestor");
if flag {
path_arr.push(RelativePathBuf::from(meta.location.as_ref()));
}
}
// Record total files scanned
increment_files_scanned_in_object_store_calls_by_date(
"LIST",
files_scanned as u64,
&Utc::now().date_naive().to_string(),
);
Ok(path_arr)
}
async fn put_object(
&self,
path: &RelativePath,
resource: Bytes,
) -> Result<(), ObjectStorageError> {
self._put_object(path, resource.into())
.await
.map_err(|err| ObjectStorageError::ConnectionError(Box::new(err)))?;
Ok(())
}
async fn delete_prefix(&self, path: &RelativePath) -> Result<(), ObjectStorageError> {
self._delete_prefix(path.as_ref()).await?;
Ok(())
}
async fn delete_object(&self, path: &RelativePath) -> Result<(), ObjectStorageError> {
let result = self.client.delete(&to_object_store_path(path)).await;
increment_object_store_calls_by_date("DELETE", &Utc::now().date_naive().to_string());
if result.is_ok() {
increment_files_scanned_in_object_store_calls_by_date(
"DELETE",
1,
&Utc::now().date_naive().to_string(),
);
}
Ok(result?)
}
async fn check(&self) -> Result<(), ObjectStorageError> {
let result = self
.client
.head(&to_object_store_path(&parseable_json_path()))
.await;
increment_object_store_calls_by_date("HEAD", &Utc::now().date_naive().to_string());
if result.is_ok() {
increment_files_scanned_in_object_store_calls_by_date(
"HEAD",
1,
&Utc::now().date_naive().to_string(),
);
}
Ok(result.map(|_| ())?)
}
async fn delete_stream(&self, stream_name: &str) -> Result<(), ObjectStorageError> {
self._delete_prefix(stream_name).await?;
Ok(())
}
async fn try_delete_node_meta(&self, node_filename: String) -> Result<(), ObjectStorageError> {
let file = RelativePathBuf::from(&node_filename);
let result = self.client.delete(&to_object_store_path(&file)).await;
increment_object_store_calls_by_date("DELETE", &Utc::now().date_naive().to_string());
match result {
Ok(_) => {
increment_files_scanned_in_object_store_calls_by_date(
"DELETE",
1,
&Utc::now().date_naive().to_string(),
);
Ok(())
}
Err(err) => Err(err.into()),
}
}
async fn list_streams(&self) -> Result<HashSet<LogStream>, ObjectStorageError> {
// self._list_streams().await
Err(ObjectStorageError::Custom(
"GCS doesn't implement list_streams".into(),
))
}
async fn list_old_streams(&self) -> Result<HashSet<LogStream>, ObjectStorageError> {
let resp = self.client.list_with_delimiter(None).await?;
let common_prefixes = resp.common_prefixes; // get all dirs
increment_files_scanned_in_object_store_calls_by_date(
"LIST",
common_prefixes.len() as u64,
&Utc::now().date_naive().to_string(),
);
increment_object_store_calls_by_date("LIST", &Utc::now().date_naive().to_string());
// return prefixes at the root level
let dirs: HashSet<_> = common_prefixes
.iter()
.filter_map(|path| path.parts().next())
.map(|name| name.as_ref().to_string())
.filter(|x| x != PARSEABLE_ROOT_DIRECTORY)
.collect();
let stream_json_check = FuturesUnordered::new();
for dir in &dirs {
let key = format!("{dir}/{STREAM_METADATA_FILE_NAME}");
let task = async move {
let result = self.client.head(&StorePath::from(key)).await;
increment_object_store_calls_by_date("HEAD", &Utc::now().date_naive().to_string());
result.map(|_| ())
};
stream_json_check.push(task);
}
increment_files_scanned_in_object_store_calls_by_date(
"HEAD",
dirs.len() as u64,
&Utc::now().date_naive().to_string(),
);
stream_json_check.try_collect::<()>().await?;
Ok(dirs)
}
async fn list_dates(&self, stream_name: &str) -> Result<Vec<String>, ObjectStorageError> {
let streams = self._list_dates(stream_name).await?;
Ok(streams)
}
async fn list_hours(
&self,
stream_name: &str,
date: &str,
) -> Result<Vec<String>, ObjectStorageError> {
let pre = object_store::path::Path::from(format!("{}/{}/", stream_name, date));
let resp = self.client.list_with_delimiter(Some(&pre)).await?;
increment_files_scanned_in_object_store_calls_by_date(
"LIST",
resp.common_prefixes.len() as u64,
&Utc::now().date_naive().to_string(),
);
increment_object_store_calls_by_date("LIST", &Utc::now().date_naive().to_string());
let hours: Vec<String> = resp
.common_prefixes
.iter()
.filter_map(|path| {
let path_str = path.as_ref();
if let Some(stripped) = path_str.strip_prefix(&format!("{}/{}/", stream_name, date))
{
// Remove trailing slash if present, otherwise use as is
let clean_path = stripped.strip_suffix('/').unwrap_or(stripped);
Some(clean_path.to_string())
} else {
None
}
})
.filter(|dir| dir.starts_with("hour="))
.collect();
Ok(hours)
}
async fn list_minutes(
&self,
stream_name: &str,
date: &str,
hour: &str,
) -> Result<Vec<String>, ObjectStorageError> {
let pre = object_store::path::Path::from(format!("{}/{}/{}/", stream_name, date, hour));
let resp = self.client.list_with_delimiter(Some(&pre)).await?;
increment_files_scanned_in_object_store_calls_by_date(
"LIST",
resp.common_prefixes.len() as u64,
&Utc::now().date_naive().to_string(),
);
increment_object_store_calls_by_date("LIST", &Utc::now().date_naive().to_string());
let minutes: Vec<String> = resp
.common_prefixes
.iter()
.filter_map(|path| {
let path_str = path.as_ref();
if let Some(stripped) =
path_str.strip_prefix(&format!("{}/{}/{}/", stream_name, date, hour))
{
// Remove trailing slash if present, otherwise use as is
let clean_path = stripped.strip_suffix('/').unwrap_or(stripped);
Some(clean_path.to_string())
} else {
None
}
})
.filter(|dir| dir.starts_with("minute="))
.collect();
Ok(minutes)
}
async fn upload_file(&self, key: &str, path: &Path) -> Result<(), ObjectStorageError> {
Ok(self._upload_file(key, path).await?)
}
fn absolute_url(&self, prefix: &RelativePath) -> object_store::path::Path {
object_store::path::Path::parse(prefix).unwrap()
}
fn query_prefixes(&self, prefixes: Vec<String>) -> Vec<ListingTableUrl> {
prefixes
.into_iter()
.map(|prefix| {
let path = format!("gs://{}/{}", &self.bucket, prefix);
ListingTableUrl::parse(path).unwrap()
})
.collect()
}
fn store_url(&self) -> url::Url {
url::Url::parse(&format!("gs://{}", self.bucket)).unwrap()
}
async fn list_dirs(&self) -> Result<Vec<String>, ObjectStorageError> {
let pre = object_store::path::Path::from("/");
let resp = self.client.list_with_delimiter(Some(&pre)).await;
increment_object_store_calls_by_date("LIST", &Utc::now().date_naive().to_string());
let resp = match resp {
Ok(resp) => {
increment_files_scanned_in_object_store_calls_by_date(
"LIST",
resp.common_prefixes.len() as u64,
&Utc::now().date_naive().to_string(),
);
resp
}
Err(err) => {
return Err(err.into());
}
};
Ok(resp
.common_prefixes
.iter()
.flat_map(|path| path.parts())
.map(|name| name.as_ref().to_string())
.collect::<Vec<_>>())
}
async fn list_dirs_relative(
&self,
relative_path: &RelativePath,
) -> Result<Vec<String>, ObjectStorageError> {
let prefix = object_store::path::Path::from(relative_path.as_str());
let resp = self.client.list_with_delimiter(Some(&prefix)).await;
increment_object_store_calls_by_date("LIST", &Utc::now().date_naive().to_string());
let resp = match resp {
Ok(resp) => {
increment_files_scanned_in_object_store_calls_by_date(
"LIST",
resp.common_prefixes.len() as u64,
&Utc::now().date_naive().to_string(),
);
resp
}
Err(err) => {
return Err(err.into());
}
};
Ok(resp
.common_prefixes
.iter()
.flat_map(|path| path.parts())
.map(|name| name.as_ref().to_string())
.collect::<Vec<_>>())
}
async fn list_with_delimiter(
&self,
prefix: Option<object_store::path::Path>,
) -> Result<ListResult, ObjectStorageError> {
Ok(self.client.list_with_delimiter(prefix.as_ref()).await?)
}
fn get_bucket_name(&self) -> String {
self.bucket.clone()
}
}