risingwave_object_store/object/opendal_engine/
opendal_object_store.rs

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
// Copyright 2024 RisingWave Labs
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
//     http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.

use std::ops::Range;
use std::sync::Arc;
use std::time::Duration;

use bytes::Bytes;
use fail::fail_point;
use futures::{stream, StreamExt};
use opendal::layers::{RetryLayer, TimeoutLayer};
use opendal::raw::BoxedStaticFuture;
use opendal::services::Memory;
use opendal::{Execute, Executor, Metakey, Operator, Writer};
use risingwave_common::config::ObjectStoreConfig;
use risingwave_common::range::RangeBoundsExt;
use thiserror_ext::AsReport;

use crate::object::object_metrics::ObjectStoreMetrics;
use crate::object::{
    prefix, ObjectDataStream, ObjectError, ObjectMetadata, ObjectMetadataIter, ObjectRangeBounds,
    ObjectResult, ObjectStore, OperationType, StreamingUploader,
};

/// Opendal object storage.
#[derive(Clone)]
pub struct OpendalObjectStore {
    pub(crate) op: Operator,
    pub(crate) media_type: MediaType,

    pub(crate) config: Arc<ObjectStoreConfig>,
    pub(crate) metrics: Arc<ObjectStoreMetrics>,
}

#[derive(Clone)]
pub enum MediaType {
    Memory,
    Hdfs,
    Gcs,
    Minio,
    S3,
    Obs,
    Oss,
    Webhdfs,
    Azblob,
    Fs,
}

impl MediaType {
    pub fn as_str(&self) -> &'static str {
        match self {
            MediaType::Memory => "Memory",
            MediaType::Hdfs => "Hdfs",
            MediaType::Gcs => "Gcs",
            MediaType::Minio => "Minio",
            MediaType::S3 => "S3",
            MediaType::Obs => "Obs",
            MediaType::Oss => "Oss",
            MediaType::Webhdfs => "Webhdfs",
            MediaType::Azblob => "Azblob",
            MediaType::Fs => "Fs",
        }
    }
}

impl OpendalObjectStore {
    /// create opendal memory engine, used for unit tests.
    pub fn test_new_memory_engine() -> ObjectResult<Self> {
        // Create memory backend builder.
        let builder = Memory::default();
        let op: Operator = Operator::new(builder)?.finish();
        Ok(Self {
            op,
            media_type: MediaType::Memory,
            config: Arc::new(ObjectStoreConfig::default()),
            metrics: Arc::new(ObjectStoreMetrics::unused()),
        })
    }
}

#[async_trait::async_trait]
impl ObjectStore for OpendalObjectStore {
    type StreamingUploader = OpendalStreamingUploader;

    fn get_object_prefix(&self, obj_id: u64, use_new_object_prefix_strategy: bool) -> String {
        match self.media_type {
            MediaType::S3 => prefix::s3::get_object_prefix(obj_id),
            MediaType::Minio => prefix::s3::get_object_prefix(obj_id),
            MediaType::Memory => String::default(),
            MediaType::Hdfs
            | MediaType::Gcs
            | MediaType::Obs
            | MediaType::Oss
            | MediaType::Webhdfs
            | MediaType::Azblob
            | MediaType::Fs => {
                prefix::opendal_engine::get_object_prefix(obj_id, use_new_object_prefix_strategy)
            }
        }
    }

    async fn upload(&self, path: &str, obj: Bytes) -> ObjectResult<()> {
        if obj.is_empty() {
            Err(ObjectError::internal("upload empty object"))
        } else {
            self.op.write(path, obj).await?;
            Ok(())
        }
    }

    async fn streaming_upload(&self, path: &str) -> ObjectResult<Self::StreamingUploader> {
        Ok(OpendalStreamingUploader::new(
            self.op.clone(),
            path.to_string(),
            self.config.clone(),
            self.metrics.clone(),
            self.store_media_type(),
        )
        .await?)
    }

    async fn read(&self, path: &str, range: impl ObjectRangeBounds) -> ObjectResult<Bytes> {
        let data = if range.is_full() {
            self.op.read(path).await?
        } else {
            self.op
                .read_with(path)
                .range(range.map(|v| *v as u64))
                .await?
        };

        if let Some(len) = range.len()
            && len != data.len()
        {
            return Err(ObjectError::internal(format!(
                "mismatched size: expected {}, found {} when reading {} at {:?}",
                len,
                data.len(),
                path,
                range,
            )));
        }

        Ok(data.to_bytes())
    }

    /// Returns a stream reading the object specified in `path`. If given, the stream starts at the
    /// byte with index `start_pos` (0-based). As far as possible, the stream only loads the amount
    /// of data into memory that is read from the stream.
    async fn streaming_read(
        &self,
        path: &str,
        range: Range<usize>,
    ) -> ObjectResult<ObjectDataStream> {
        fail_point!("opendal_streaming_read_err", |_| Err(
            ObjectError::internal("opendal streaming read error")
        ));
        let range: Range<u64> = (range.start as u64)..(range.end as u64);

        // The layer specified first will be executed first.
        // `TimeoutLayer` must be specified before `RetryLayer`.
        // Otherwise, it will lead to bad state inside OpenDAL and panic.
        // See https://docs.rs/opendal/latest/opendal/layers/struct.RetryLayer.html#panics
        let reader = self
            .op
            .clone()
            .layer(TimeoutLayer::new().with_io_timeout(Duration::from_millis(
                self.config.retry.streaming_read_attempt_timeout_ms,
            )))
            .layer(
                RetryLayer::new()
                    .with_min_delay(Duration::from_millis(
                        self.config.retry.req_backoff_interval_ms,
                    ))
                    .with_max_delay(Duration::from_millis(
                        self.config.retry.req_backoff_max_delay_ms,
                    ))
                    .with_max_times(self.config.retry.streaming_read_retry_attempts)
                    .with_factor(self.config.retry.req_backoff_factor as f32)
                    .with_jitter(),
            )
            .reader_with(path)
            .await?;
        let stream = reader.into_bytes_stream(range).await?.map(|item| {
            item.map(|b| Bytes::copy_from_slice(b.as_ref()))
                .map_err(|e| {
                    ObjectError::internal(format!("reader into_stream fail {}", e.as_report()))
                })
        });

        Ok(Box::pin(stream))
    }

    async fn metadata(&self, path: &str) -> ObjectResult<ObjectMetadata> {
        let opendal_metadata = self.op.stat(path).await?;
        let key = path.to_string();
        let last_modified = match opendal_metadata.last_modified() {
            Some(t) => t.timestamp() as f64,
            None => 0_f64,
        };

        let total_size = opendal_metadata.content_length() as usize;
        let metadata = ObjectMetadata {
            key,
            last_modified,
            total_size,
        };
        Ok(metadata)
    }

    async fn delete(&self, path: &str) -> ObjectResult<()> {
        self.op.delete(path).await?;
        Ok(())
    }

    /// Deletes the objects with the given paths permanently from the storage. If an object
    /// specified in the request is not found, it will be considered as successfully deleted.
    async fn delete_objects(&self, paths: &[String]) -> ObjectResult<()> {
        self.op.remove(paths.to_vec()).await?;
        Ok(())
    }

    async fn list(
        &self,
        prefix: &str,
        start_after: Option<String>,
        limit: Option<usize>,
    ) -> ObjectResult<ObjectMetadataIter> {
        let mut object_lister = self
            .op
            .lister_with(prefix)
            .recursive(true)
            .metakey(Metakey::ContentLength);
        if let Some(start_after) = start_after {
            object_lister = object_lister.start_after(&start_after);
        }
        let object_lister = object_lister.await?;

        let stream = stream::unfold(object_lister, |mut object_lister| async move {
            match object_lister.next().await {
                Some(Ok(object)) => {
                    let key = object.path().to_string();
                    let om = object.metadata();
                    let last_modified = match om.last_modified() {
                        Some(t) => t.timestamp() as f64,
                        None => 0_f64,
                    };
                    let total_size = om.content_length() as usize;
                    let metadata = ObjectMetadata {
                        key,
                        last_modified,
                        total_size,
                    };
                    Some((Ok(metadata), object_lister))
                }
                Some(Err(err)) => Some((Err(err.into()), object_lister)),
                None => None,
            }
        });

        Ok(stream.take(limit.unwrap_or(usize::MAX)).boxed())
    }

    fn store_media_type(&self) -> &'static str {
        self.media_type.as_str()
    }

    fn support_streaming_upload(&self) -> bool {
        self.op.info().native_capability().write_can_multi
    }
}

struct OpendalStreamingUploaderExecute {
    /// To record metrics for uploading part.
    metrics: Arc<ObjectStoreMetrics>,
    media_type: &'static str,
}

impl OpendalStreamingUploaderExecute {
    const STREAMING_UPLOAD_TYPE: OperationType = OperationType::StreamingUpload;

    fn new(metrics: Arc<ObjectStoreMetrics>, media_type: &'static str) -> Self {
        Self {
            metrics,
            media_type,
        }
    }
}

impl Execute for OpendalStreamingUploaderExecute {
    fn execute(&self, f: BoxedStaticFuture<()>) {
        let operation_type_str = Self::STREAMING_UPLOAD_TYPE.as_str();
        let media_type = self.media_type;

        let metrics = self.metrics.clone();
        let _handle = tokio::spawn(async move {
            let _timer = metrics
                .operation_latency
                .with_label_values(&[media_type, operation_type_str])
                .start_timer();

            f.await
        });
    }
}

/// Store multiple parts in a map, and concatenate them on finish.
pub struct OpendalStreamingUploader {
    writer: Writer,
    /// Buffer for data. It will store at least `UPLOAD_BUFFER_SIZE` bytes of data before wrapping itself
    /// into a stream and upload to object store as a part.
    buf: Vec<Bytes>,
    /// Length of the data that have not been uploaded to object store.
    not_uploaded_len: usize,
    /// Whether the writer is valid. The writer is invalid after abort/close.
    is_valid: bool,

    abort_on_err: bool,
}

impl OpendalStreamingUploader {
    const UPLOAD_BUFFER_SIZE: usize = 16 * 1024 * 1024;

    pub async fn new(
        op: Operator,
        path: String,
        config: Arc<ObjectStoreConfig>,
        metrics: Arc<ObjectStoreMetrics>,
        media_type: &'static str,
    ) -> ObjectResult<Self> {
        let monitored_execute = OpendalStreamingUploaderExecute::new(metrics, media_type);

        // The layer specified first will be executed first.
        // `TimeoutLayer` must be specified before `RetryLayer`.
        // Otherwise, it will lead to bad state inside OpenDAL and panic.
        // See https://docs.rs/opendal/latest/opendal/layers/struct.RetryLayer.html#panics
        let writer = op
            .clone()
            .layer(TimeoutLayer::new().with_io_timeout(Duration::from_millis(
                config.retry.streaming_upload_attempt_timeout_ms,
            )))
            .layer(
                RetryLayer::new()
                    .with_min_delay(Duration::from_millis(config.retry.req_backoff_interval_ms))
                    .with_max_delay(Duration::from_millis(config.retry.req_backoff_max_delay_ms))
                    .with_max_times(config.retry.streaming_upload_retry_attempts)
                    .with_factor(config.retry.req_backoff_factor as f32)
                    .with_jitter(),
            )
            .writer_with(&path)
            .concurrent(config.opendal_upload_concurrency)
            .executor(Executor::with(monitored_execute))
            .await?;
        Ok(Self {
            writer,
            buf: vec![],
            not_uploaded_len: 0,
            is_valid: true,
            abort_on_err: config.opendal_writer_abort_on_err,
        })
    }

    async fn flush(&mut self) -> ObjectResult<()> {
        let data: Vec<Bytes> = self.buf.drain(..).collect();
        debug_assert_eq!(
            data.iter().map(|b| b.len()).sum::<usize>(),
            self.not_uploaded_len
        );
        if let Err(err) = self.writer.write(data).await {
            self.is_valid = false;
            if self.abort_on_err {
                self.writer.abort().await?;
            }
            return Err(err.into());
        }
        self.not_uploaded_len = 0;
        Ok(())
    }
}

impl StreamingUploader for OpendalStreamingUploader {
    async fn write_bytes(&mut self, data: Bytes) -> ObjectResult<()> {
        assert!(self.is_valid);
        self.not_uploaded_len += data.len();
        self.buf.push(data);
        if self.not_uploaded_len >= Self::UPLOAD_BUFFER_SIZE {
            self.flush().await?;
        }
        Ok(())
    }

    async fn finish(mut self) -> ObjectResult<()> {
        assert!(self.is_valid);
        if self.not_uploaded_len > 0 {
            self.flush().await?;
        }

        assert!(self.buf.is_empty());
        assert_eq!(self.not_uploaded_len, 0);

        self.is_valid = false;
        match self.writer.close().await {
            Ok(_) => (),
            Err(err) => {
                if self.abort_on_err {
                    self.writer.abort().await?;
                }
                return Err(err.into());
            }
        };

        Ok(())
    }

    fn get_memory_usage(&self) -> u64 {
        Self::UPLOAD_BUFFER_SIZE as u64
    }
}

#[cfg(test)]
mod tests {
    use stream::TryStreamExt;

    use super::*;

    async fn list_all(prefix: &str, store: &OpendalObjectStore) -> Vec<ObjectMetadata> {
        store
            .list(prefix, None, None)
            .await
            .unwrap()
            .try_collect::<Vec<_>>()
            .await
            .unwrap()
    }

    #[tokio::test]
    async fn test_memory_upload() {
        let block = Bytes::from("123456");
        let store = OpendalObjectStore::test_new_memory_engine().unwrap();
        store.upload("/abc", block).await.unwrap();

        // No such object.
        store.read("/ab", 0..3).await.unwrap_err();

        let bytes = store.read("/abc", 4..6).await.unwrap();
        assert_eq!(String::from_utf8(bytes.to_vec()).unwrap(), "56".to_string());

        store.delete("/abc").await.unwrap();

        // No such object.
        store.read("/abc", 0..3).await.unwrap_err();
    }

    #[tokio::test]
    #[should_panic]
    async fn test_memory_read_overflow() {
        let block = Bytes::from("123456");
        let store = OpendalObjectStore::test_new_memory_engine().unwrap();
        store.upload("/abc", block).await.unwrap();

        // Overflow.
        store.read("/abc", 4..44).await.unwrap_err();
    }

    #[tokio::test]
    async fn test_memory_metadata() {
        let block = Bytes::from("123456");
        let path = "/abc".to_string();
        let obj_store = OpendalObjectStore::test_new_memory_engine().unwrap();
        obj_store.upload("/abc", block).await.unwrap();

        let err = obj_store.metadata("/not_exist").await.unwrap_err();
        assert!(err.is_object_not_found_error());
        let metadata = obj_store.metadata("/abc").await.unwrap();
        assert_eq!(metadata.total_size, 6);
        obj_store.delete(&path).await.unwrap();
    }

    #[tokio::test]
    async fn test_memory_delete_objects_and_list_object() {
        let block1 = Bytes::from("123456");
        let block2 = Bytes::from("987654");
        let store = OpendalObjectStore::test_new_memory_engine().unwrap();
        store.upload("abc", Bytes::from("123456")).await.unwrap();
        store.upload("prefix/abc", block1).await.unwrap();

        store.upload("prefix/xyz", block2).await.unwrap();

        assert_eq!(list_all("", &store).await.len(), 3);
        assert_eq!(list_all("prefix/", &store).await.len(), 2);
        let str_list = [String::from("prefix/abc"), String::from("prefix/xyz")];

        store.delete_objects(&str_list).await.unwrap();

        assert!(store.read("prefix/abc/", ..).await.is_err());
        assert!(store.read("prefix/xyz/", ..).await.is_err());
        assert_eq!(list_all("", &store).await.len(), 1);
        assert_eq!(list_all("prefix/", &store).await.len(), 0);
    }
}