icepick 0.4.1

Experimental Rust client for Apache Iceberg with WASM support for AWS S3 Tables and Cloudflare R2
Documentation
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
use super::rest::IcebergRestCatalog;
use super::{map_catalog_error, AuthProvider, Catalog, CatalogError, CatalogOptions, RetryConfig};
use crate::error::{Error, Result};
use crate::io::FileIO;
use crate::spec::{NamespaceIdent, TableCreation, TableIdent};
use crate::table::Table;
use async_trait::async_trait;
use std::collections::HashMap;
use std::time::Duration;

/// Generic Iceberg REST catalog wrapper that can target any compliant REST endpoint.
///
/// This type exposes the shared REST implementation used by [`R2Catalog`] and
/// [`S3TablesCatalog`], but lets callers provide their own authentication logic,
/// endpoint, and `FileIO` configuration. Use this when connecting to Nessie,
/// Glue REST, or custom Iceberg catalog implementations.
#[derive(Debug)]
pub struct RestCatalog {
    inner: IcebergRestCatalog,
}

impl RestCatalog {
    /// Create a new builder using the provided catalog name and base endpoint.
    pub fn builder(name: impl Into<String>, endpoint: impl Into<String>) -> RestCatalogBuilder {
        RestCatalogBuilder::new(name, endpoint)
    }

    /// Convenience constructor for simple use cases that don't need builder customization.
    pub fn new(
        name: impl Into<String>,
        endpoint: impl Into<String>,
        auth_provider: impl RestAuthProvider + 'static,
        file_io: FileIO,
    ) -> Result<Self> {
        Self::builder(name, endpoint)
            .with_auth_provider(auth_provider)
            .with_file_io(file_io)
            .build()
    }
}

/// Builder for constructing [`RestCatalog`] instances with custom options.
pub struct RestCatalogBuilder {
    name: String,
    endpoint: String,
    prefix: String,
    options: CatalogOptions,
    file_io: Option<FileIO>,
    auth_provider: Option<Box<dyn RestAuthProvider>>,
}

impl RestCatalogBuilder {
    fn new(name: impl Into<String>, endpoint: impl Into<String>) -> Self {
        Self {
            name: name.into(),
            endpoint: endpoint.into(),
            prefix: String::new(),
            options: CatalogOptions::default(),
            file_io: None,
            auth_provider: None,
        }
    }

    /// Override the namespace/table prefix that is inserted between `/v1/` and the endpoint path.
    ///
    /// For example: `https://example.com/iceberg` with `prefix = "warehouse"` will produce
    /// requests like `https://example.com/iceberg/v1/warehouse/namespaces/...`.
    pub fn with_prefix(mut self, prefix: impl Into<String>) -> Self {
        self.prefix = prefix.into();
        self
    }

    /// Override the default [`CatalogOptions`], letting you change HTTP timeouts
    /// or use a non-`main` reference/branch.
    pub fn with_options(mut self, options: CatalogOptions) -> Self {
        self.options = options;
        self
    }

    /// Provide the [`FileIO`] implementation that resolves data files referenced by the catalog.
    ///
    /// Most callers will create a single-operator [`FileIO::new`] using an OpenDAL operator,
    /// or configure AWS credentials via [`FileIO::from_aws_credentials`].
    pub fn with_file_io(mut self, file_io: FileIO) -> Self {
        self.file_io = Some(file_io);
        self
    }

    /// Provide a custom authentication provider used to sign every HTTP request.
    pub fn with_auth_provider<A>(mut self, provider: A) -> Self
    where
        A: RestAuthProvider + 'static,
    {
        self.auth_provider = Some(Box::new(provider));
        self
    }

    /// Convenience helper for bearer token authentication.
    pub fn with_bearer_token(mut self, token: impl Into<String>) -> Self {
        self.auth_provider = Some(Box::new(crate::catalog::BearerTokenAuthProvider::new(
            token,
        )));
        self
    }

    /// Configure retry behavior for catalog operations.
    ///
    /// This sets application-level retries based on error types (Transient/Permanent/Timeout).
    /// For HTTP-level retries (connection errors, network issues), use [`CatalogOptions::with_http_config`].
    ///
    /// # Example
    ///
    /// ```no_run
    /// use icepick::catalog::{RestCatalog, RetryConfig, BackoffStrategy};
    /// use std::time::Duration;
    /// # use icepick::io::FileIO;
    /// # use opendal::Operator;
    ///
    /// # fn example() -> Result<(), Box<dyn std::error::Error>> {
    /// # let file_io = FileIO::new(Operator::via_iter(opendal::Scheme::Memory, [])?);
    /// let retry = RetryConfig::new(
    ///     5,
    ///     BackoffStrategy::Exponential {
    ///         initial_delay: Duration::from_millis(100),
    ///         max_delay: Duration::from_secs(30),
    ///         multiplier: 2.0,
    ///     }
    /// );
    ///
    /// let catalog = RestCatalog::builder("my-catalog", "https://api.example.com")
    ///     .with_bearer_token("my-token")
    ///     .with_retry_config(retry)
    ///     .with_file_io(file_io)
    ///     .build()?;
    /// # Ok(())
    /// # }
    /// ```
    pub fn with_retry_config(mut self, retry: RetryConfig) -> Self {
        self.options = self.options.with_retry_config(retry);
        self
    }

    /// Set the request timeout for catalog operations.
    ///
    /// This is a convenience method for configuring HTTP-level timeouts.
    ///
    /// # Example
    ///
    /// ```no_run
    /// use icepick::catalog::RestCatalog;
    /// use std::time::Duration;
    /// # use icepick::io::FileIO;
    /// # use opendal::Operator;
    ///
    /// # fn example() -> Result<(), Box<dyn std::error::Error>> {
    /// # let file_io = FileIO::new(Operator::via_iter(opendal::Scheme::Memory, [])?);
    /// let catalog = RestCatalog::builder("my-catalog", "https://api.example.com")
    ///     .with_bearer_token("my-token")
    ///     .with_timeout(Duration::from_secs(60))
    ///     .with_file_io(file_io)
    ///     .build()?;
    /// # Ok(())
    /// # }
    /// ```
    pub fn with_timeout(mut self, timeout: Duration) -> Self {
        let http = self.options.http().clone().with_timeout(timeout);
        self.options = self.options.with_http_config(http);
        self
    }

    /// Construct the [`RestCatalog`], validating that all required components are set.
    pub fn build(self) -> Result<RestCatalog> {
        let file_io = self.file_io.ok_or_else(|| {
            Error::invalid_config("RestCatalog requires a FileIO. Call with_file_io first.")
        })?;
        let auth_provider = self.auth_provider.ok_or_else(|| {
            Error::invalid_config("RestCatalog requires an auth provider. Call with_auth_provider.")
        })?;

        let adapter: Box<dyn AuthProvider> = Box::new(ExternalAuthProvider {
            inner: auth_provider,
        });

        let inner = IcebergRestCatalog::from_components(
            self.name,
            self.endpoint,
            self.prefix,
            adapter,
            file_io,
            self.options,
        )
        .map_err(map_catalog_error)?;

        Ok(RestCatalog { inner })
    }
}

/// Trait for supplying custom authentication logic to the REST catalog.
#[cfg_attr(not(target_family = "wasm"), async_trait)]
#[cfg_attr(target_family = "wasm", async_trait(?Send))]
pub trait RestAuthProvider: Send + Sync + std::fmt::Debug {
    /// Sign or otherwise modify a request before it is sent to the REST endpoint.
    async fn sign_request(&self, request: reqwest::Request) -> Result<reqwest::Request>;
}

struct ExternalAuthProvider {
    inner: Box<dyn RestAuthProvider>,
}

impl std::fmt::Debug for ExternalAuthProvider {
    fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
        f.debug_struct("ExternalAuthProvider")
            .finish_non_exhaustive()
    }
}

#[cfg_attr(not(target_family = "wasm"), async_trait)]
#[cfg_attr(target_family = "wasm", async_trait(?Send))]
impl AuthProvider for ExternalAuthProvider {
    async fn sign_request(&self, request: reqwest::Request) -> super::Result<reqwest::Request> {
        self.inner
            .sign_request(request)
            .await
            .map_err(map_auth_error)
    }
}

fn map_auth_error(err: Error) -> CatalogError {
    match err {
        Error::Unauthorized { provider } => CatalogError::AuthError(provider),
        Error::NetworkError { source } => CatalogError::Network(source),
        Error::InvalidRequest { message } => CatalogError::InvalidRequest(message),
        Error::ServerError { status, message } => CatalogError::ServerError { status, message },
        _ => CatalogError::AuthError(err.to_string()),
    }
}

// Implement Catalog trait by delegating to inner Rest implementation (native)
#[cfg(not(target_family = "wasm"))]
#[async_trait]
impl Catalog for RestCatalog {
    fn file_io(&self) -> &crate::io::FileIO {
        self.inner.file_io()
    }

    async fn create_namespace(
        &self,
        namespace: &NamespaceIdent,
        properties: HashMap<String, String>,
    ) -> Result<()> {
        self.inner.create_namespace(namespace, properties).await
    }

    async fn namespace_exists(&self, namespace: &NamespaceIdent) -> Result<bool> {
        self.inner.namespace_exists(namespace).await
    }

    async fn list_namespaces(&self) -> Result<Vec<NamespaceIdent>> {
        self.inner.list_namespaces().await
    }

    async fn list_tables(&self, namespace: &NamespaceIdent) -> Result<Vec<TableIdent>> {
        self.inner.list_tables(namespace).await
    }

    async fn table_exists(&self, identifier: &TableIdent) -> Result<bool> {
        self.inner.table_exists(identifier).await
    }

    async fn create_table(
        &self,
        namespace: &NamespaceIdent,
        creation: TableCreation,
    ) -> Result<Table> {
        self.inner.create_table(namespace, creation).await
    }

    async fn load_table(&self, identifier: &TableIdent) -> Result<Table> {
        self.inner.load_table(identifier).await
    }

    async fn drop_table(&self, identifier: &TableIdent) -> Result<()> {
        self.inner.drop_table(identifier).await
    }

    async fn update_table_metadata(
        &self,
        identifier: &TableIdent,
        old_metadata_location: &str,
        new_metadata_location: &str,
    ) -> Result<()> {
        self.inner
            .update_table_metadata(identifier, old_metadata_location, new_metadata_location)
            .await
    }

    async fn expire_snapshots(&self, identifier: &TableIdent, snapshot_ids: &[i64]) -> Result<()> {
        self.inner.expire_snapshots(identifier, snapshot_ids).await
    }
}

// Implement Catalog trait for WASM targets without Send requirement.
#[cfg(target_family = "wasm")]
#[async_trait(?Send)]
impl Catalog for RestCatalog {
    fn file_io(&self) -> &crate::io::FileIO {
        self.inner.file_io()
    }

    async fn create_namespace(
        &self,
        namespace: &NamespaceIdent,
        properties: HashMap<String, String>,
    ) -> Result<()> {
        self.inner.create_namespace(namespace, properties).await
    }

    async fn namespace_exists(&self, namespace: &NamespaceIdent) -> Result<bool> {
        self.inner.namespace_exists(namespace).await
    }

    async fn list_namespaces(&self) -> Result<Vec<NamespaceIdent>> {
        self.inner.list_namespaces().await
    }

    async fn list_tables(&self, namespace: &NamespaceIdent) -> Result<Vec<TableIdent>> {
        self.inner.list_tables(namespace).await
    }

    async fn table_exists(&self, identifier: &TableIdent) -> Result<bool> {
        self.inner.table_exists(identifier).await
    }

    async fn create_table(
        &self,
        namespace: &NamespaceIdent,
        creation: TableCreation,
    ) -> Result<Table> {
        self.inner.create_table(namespace, creation).await
    }

    async fn load_table(&self, identifier: &TableIdent) -> Result<Table> {
        self.inner.load_table(identifier).await
    }

    async fn drop_table(&self, identifier: &TableIdent) -> Result<()> {
        self.inner.drop_table(identifier).await
    }

    async fn update_table_metadata(
        &self,
        identifier: &TableIdent,
        old_metadata_location: &str,
        new_metadata_location: &str,
    ) -> Result<()> {
        self.inner
            .update_table_metadata(identifier, old_metadata_location, new_metadata_location)
            .await
    }

    async fn expire_snapshots(&self, identifier: &TableIdent, snapshot_ids: &[i64]) -> Result<()> {
        self.inner.expire_snapshots(identifier, snapshot_ids).await
    }
}

#[cfg_attr(not(target_family = "wasm"), async_trait)]
#[cfg_attr(target_family = "wasm", async_trait(?Send))]
impl RestAuthProvider for crate::catalog::BearerTokenAuthProvider {
    async fn sign_request(&self, request: reqwest::Request) -> Result<reqwest::Request> {
        <Self as AuthProvider>::sign_request(self, request)
            .await
            .map_err(map_catalog_error)
    }
}

#[cfg(not(target_family = "wasm"))]
#[async_trait]
impl RestAuthProvider for crate::catalog::SigV4AuthProvider {
    async fn sign_request(&self, request: reqwest::Request) -> Result<reqwest::Request> {
        <Self as AuthProvider>::sign_request(self, request)
            .await
            .map_err(map_catalog_error)
    }
}

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

    #[derive(Debug)]
    struct NoopAuth;

    #[cfg_attr(not(target_family = "wasm"), async_trait)]
    #[cfg_attr(target_family = "wasm", async_trait(?Send))]
    impl RestAuthProvider for NoopAuth {
        async fn sign_request(&self, request: reqwest::Request) -> Result<reqwest::Request> {
            Ok(request)
        }
    }

    fn memory_file_io() -> FileIO {
        let operator =
            Operator::via_iter(opendal::Scheme::Memory, []).expect("memory operator should build");
        FileIO::new(operator)
    }

    #[test]
    fn builder_requires_file_io() {
        let err = RestCatalog::builder("test", "https://example.com/iceberg")
            .with_auth_provider(NoopAuth)
            .build()
            .expect_err("missing FileIO should error");

        assert!(matches!(err, Error::InvalidConfig { .. }));
    }

    #[test]
    fn builder_requires_auth() {
        let err = RestCatalog::builder("test", "https://example.com/iceberg")
            .with_file_io(memory_file_io())
            .build()
            .expect_err("missing auth provider should error");

        assert!(matches!(err, Error::InvalidConfig { .. }));
    }

    #[test]
    fn builder_accepts_components() {
        let result = RestCatalog::builder("test", "https://example.com/iceberg")
            .with_prefix("warehouse")
            .with_file_io(memory_file_io())
            .with_auth_provider(NoopAuth)
            .build();

        assert!(result.is_ok());
    }
}