risingwave_connector/connector_common/iceberg/
mod.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
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
// 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.

mod jni_catalog;
mod mock_catalog;
mod storage_catalog;
use std::collections::HashMap;
use std::sync::Arc;

use anyhow::Context;
use iceberg::io::{S3_ACCESS_KEY_ID, S3_ENDPOINT, S3_REGION, S3_SECRET_ACCESS_KEY};
use icelake::catalog::{
    load_iceberg_base_catalog_config, BaseCatalogConfig, CATALOG_NAME, CATALOG_TYPE,
};
use risingwave_common::bail;
use serde_derive::Deserialize;
use serde_with::serde_as;
use url::Url;
use with_options::WithOptions;

use crate::deserialize_optional_bool_from_string;
use crate::error::ConnectorResult;

#[serde_as]
#[derive(Debug, Clone, PartialEq, Eq, Deserialize, WithOptions)]
pub struct IcebergCommon {
    // Catalog type supported by iceberg, such as "storage", "rest".
    // If not set, we use "storage" as default.
    #[serde(rename = "catalog.type")]
    pub catalog_type: Option<String>,
    #[serde(rename = "s3.region")]
    pub region: Option<String>,
    #[serde(rename = "s3.endpoint")]
    pub endpoint: Option<String>,
    #[serde(rename = "s3.access.key")]
    pub access_key: String,
    #[serde(rename = "s3.secret.key")]
    pub secret_key: String,
    /// Path of iceberg warehouse, only applicable in storage catalog.
    #[serde(rename = "warehouse.path")]
    pub warehouse_path: Option<String>,
    /// Catalog name, can be omitted for storage catalog, but
    /// must be set for other catalogs.
    #[serde(rename = "catalog.name")]
    pub catalog_name: Option<String>,
    /// URI of iceberg catalog, only applicable in rest catalog.
    #[serde(rename = "catalog.uri")]
    pub catalog_uri: Option<String>,
    #[serde(rename = "database.name")]
    pub database_name: Option<String>,
    /// Full name of table, must include schema name.
    #[serde(rename = "table.name")]
    pub table_name: String,
    /// Credential for accessing iceberg catalog, only applicable in rest catalog.
    /// A credential to exchange for a token in the OAuth2 client credentials flow.
    #[serde(rename = "catalog.credential")]
    pub credential: Option<String>,
    /// token for accessing iceberg catalog, only applicable in rest catalog.
    /// A Bearer token which will be used for interaction with the server.
    #[serde(rename = "catalog.token")]
    pub token: Option<String>,
    /// `oauth2-server-uri` for accessing iceberg catalog, only applicable in rest catalog.
    /// Token endpoint URI to fetch token from if the Rest Catalog is not the authorization server.
    #[serde(rename = "catalog.oauth2-server-uri")]
    pub oauth2_server_uri: Option<String>,
    /// scope for accessing iceberg catalog, only applicable in rest catalog.
    /// Additional scope for OAuth2.
    #[serde(rename = "catalog.scope")]
    pub scope: Option<String>,

    #[serde(
        rename = "s3.path.style.access",
        default,
        deserialize_with = "deserialize_optional_bool_from_string"
    )]
    pub path_style_access: Option<bool>,
}

impl IcebergCommon {
    pub fn catalog_type(&self) -> &str {
        self.catalog_type.as_deref().unwrap_or("storage")
    }

    pub fn catalog_name(&self) -> String {
        self.catalog_name
            .as_ref()
            .map(|s| s.to_string())
            .unwrap_or_else(|| "risingwave".to_string())
    }

    /// For both V1 and V2.
    fn build_jni_catalog_configs(
        &self,
        java_catalog_props: &HashMap<String, String>,
    ) -> ConnectorResult<(BaseCatalogConfig, HashMap<String, String>)> {
        let mut iceberg_configs = HashMap::new();

        let base_catalog_config = {
            let catalog_type = self.catalog_type().to_string();

            iceberg_configs.insert(CATALOG_TYPE.to_string(), catalog_type.clone());
            iceberg_configs.insert(CATALOG_NAME.to_string(), self.catalog_name());

            if let Some(region) = &self.region {
                // icelake
                iceberg_configs.insert(
                    "iceberg.table.io.region".to_string(),
                    region.clone().to_string(),
                );
                // iceberg-rust
                iceberg_configs.insert(
                    ("iceberg.table.io.".to_string() + S3_REGION).to_string(),
                    region.clone().to_string(),
                );
            }

            if let Some(endpoint) = &self.endpoint {
                iceberg_configs.insert(
                    "iceberg.table.io.endpoint".to_string(),
                    endpoint.clone().to_string(),
                );

                // iceberg-rust
                iceberg_configs.insert(
                    ("iceberg.table.io.".to_string() + S3_ENDPOINT).to_string(),
                    endpoint.clone().to_string(),
                );
            }

            // icelake
            iceberg_configs.insert(
                "iceberg.table.io.access_key_id".to_string(),
                self.access_key.clone().to_string(),
            );
            iceberg_configs.insert(
                "iceberg.table.io.secret_access_key".to_string(),
                self.secret_key.clone().to_string(),
            );

            // iceberg-rust
            iceberg_configs.insert(
                ("iceberg.table.io.".to_string() + S3_ACCESS_KEY_ID).to_string(),
                self.access_key.clone().to_string(),
            );
            iceberg_configs.insert(
                ("iceberg.table.io.".to_string() + S3_SECRET_ACCESS_KEY).to_string(),
                self.secret_key.clone().to_string(),
            );

            match &self.warehouse_path {
                Some(warehouse_path) => {
                    let (bucket, _) = {
                        let url = Url::parse(warehouse_path);
                        if url.is_err() && catalog_type == "rest" {
                            // If the warehouse path is not a valid URL, it could be a warehouse name in rest catalog
                            // so we allow it to pass here.
                            (None, None)
                        } else {
                            let url = url.with_context(|| {
                                format!("Invalid warehouse path: {}", warehouse_path)
                            })?;
                            let bucket = url
                                .host_str()
                                .with_context(|| {
                                    format!(
                                        "Invalid s3 path: {}, bucket is missing",
                                        warehouse_path
                                    )
                                })?
                                .to_string();
                            let root = url.path().trim_start_matches('/').to_string();
                            (Some(bucket), Some(root))
                        }
                    };

                    if let Some(bucket) = bucket {
                        iceberg_configs.insert("iceberg.table.io.bucket".to_string(), bucket);
                    }
                }
                None => {
                    if catalog_type != "rest" {
                        bail!("`warehouse.path` must be set in {} catalog", &catalog_type);
                    }
                }
            }

            // #TODO
            // Support load config file
            iceberg_configs.insert(
                "iceberg.table.io.disable_config_load".to_string(),
                "true".to_string(),
            );

            load_iceberg_base_catalog_config(&iceberg_configs)?
        };

        // Prepare jni configs, for details please see https://iceberg.apache.org/docs/latest/aws/
        let mut java_catalog_configs = HashMap::new();
        {
            if let Some(uri) = self.catalog_uri.as_deref() {
                java_catalog_configs.insert("uri".to_string(), uri.to_string());
            }

            if let Some(warehouse_path) = &self.warehouse_path {
                java_catalog_configs.insert("warehouse".to_string(), warehouse_path.clone());
            }
            java_catalog_configs.extend(java_catalog_props.clone());

            // Currently we only support s3, so let's set it to s3
            java_catalog_configs.insert(
                "io-impl".to_string(),
                "org.apache.iceberg.aws.s3.S3FileIO".to_string(),
            );

            // suppress log of S3FileIO like: Unclosed S3FileIO instance created by...
            java_catalog_configs
                .insert("init-creation-stacktrace".to_string(), "false".to_string());

            if let Some(endpoint) = &self.endpoint {
                java_catalog_configs
                    .insert("s3.endpoint".to_string(), endpoint.clone().to_string());
            }

            java_catalog_configs.insert(
                "s3.access-key-id".to_string(),
                self.access_key.clone().to_string(),
            );
            java_catalog_configs.insert(
                "s3.secret-access-key".to_string(),
                self.secret_key.clone().to_string(),
            );

            if let Some(path_style_access) = self.path_style_access {
                java_catalog_configs.insert(
                    "s3.path-style-access".to_string(),
                    path_style_access.to_string(),
                );
            }

            match self.catalog_type.as_deref() {
                Some("rest") => {
                    if let Some(credential) = &self.credential {
                        java_catalog_configs.insert("credential".to_string(), credential.clone());
                    }
                    if let Some(token) = &self.token {
                        java_catalog_configs.insert("token".to_string(), token.clone());
                    }
                    if let Some(oauth2_server_uri) = &self.oauth2_server_uri {
                        java_catalog_configs
                            .insert("oauth2-server-uri".to_string(), oauth2_server_uri.clone());
                    }
                    if let Some(scope) = &self.scope {
                        java_catalog_configs.insert("scope".to_string(), scope.clone());
                    }
                }
                Some("glue") => {
                    java_catalog_configs.insert(
                        "client.credentials-provider".to_string(),
                        "com.risingwave.connector.catalog.GlueCredentialProvider".to_string(),
                    );
                    // Use S3 ak/sk and region as glue ak/sk and region by default.
                    // TODO: use different ak/sk and region for s3 and glue.
                    java_catalog_configs.insert(
                        "client.credentials-provider.glue.access-key-id".to_string(),
                        self.access_key.clone().to_string(),
                    );
                    java_catalog_configs.insert(
                        "client.credentials-provider.glue.secret-access-key".to_string(),
                        self.secret_key.clone().to_string(),
                    );
                    if let Some(region) = &self.region {
                        java_catalog_configs
                            .insert("client.region".to_string(), region.clone().to_string());
                        java_catalog_configs.insert(
                            "glue.endpoint".to_string(),
                            format!("https://glue.{}.amazonaws.com", region),
                        );
                    }
                }
                _ => {}
            }
        }

        Ok((base_catalog_config, java_catalog_configs))
    }
}

/// icelake
mod v1 {
    use anyhow::anyhow;
    use icelake::catalog::{load_catalog, CatalogRef};
    use icelake::{Table, TableIdentifier};

    use super::*;

    impl IcebergCommon {
        pub fn full_table_name(&self) -> ConnectorResult<TableIdentifier> {
            let ret = if let Some(database_name) = &self.database_name {
                TableIdentifier::new(vec![database_name, &self.table_name])
            } else {
                TableIdentifier::new(vec![&self.table_name])
            };

            Ok(ret.context("Failed to create table identifier")?)
        }

        fn build_iceberg_configs(&self) -> ConnectorResult<HashMap<String, String>> {
            let mut iceberg_configs = HashMap::new();

            let catalog_type = self.catalog_type().to_string();

            iceberg_configs.insert(CATALOG_NAME.to_string(), self.catalog_name());

            match catalog_type.as_str() {
                "storage" => {
                    iceberg_configs.insert(
                        format!("iceberg.catalog.{}.warehouse", self.catalog_name()),
                        self.warehouse_path.clone().ok_or_else(|| {
                            anyhow!("`warehouse.path` must be set in storage catalog")
                        })?,
                    );
                    iceberg_configs.insert(CATALOG_TYPE.to_string(), "storage".into());
                }
                "rest_rust" => {
                    let uri = self
                        .catalog_uri
                        .clone()
                        .with_context(|| "`catalog.uri` must be set in rest catalog".to_string())?;
                    iceberg_configs
                        .insert(format!("iceberg.catalog.{}.uri", self.catalog_name()), uri);
                    iceberg_configs.insert(CATALOG_TYPE.to_string(), "rest".into());
                }
                _ => {
                    bail!(
                        "Unsupported catalog type: {}, only support `storage` and `rest`",
                        catalog_type
                    );
                }
            }

            if let Some(region) = &self.region {
                iceberg_configs.insert(
                    "iceberg.table.io.region".to_string(),
                    region.clone().to_string(),
                );
            }

            if let Some(endpoint) = &self.endpoint {
                iceberg_configs.insert(
                    "iceberg.table.io.endpoint".to_string(),
                    endpoint.clone().to_string(),
                );
            }

            iceberg_configs.insert(
                "iceberg.table.io.access_key_id".to_string(),
                self.access_key.clone().to_string(),
            );
            iceberg_configs.insert(
                "iceberg.table.io.secret_access_key".to_string(),
                self.secret_key.clone().to_string(),
            );
            if let Some(path_style_access) = self.path_style_access {
                iceberg_configs.insert(
                    "iceberg.table.io.enable_virtual_host_style".to_string(),
                    (!path_style_access).to_string(),
                );
            }

            match &self.warehouse_path {
                Some(warehouse_path) => {
                    let (bucket, root) = {
                        let url = Url::parse(warehouse_path).with_context(|| {
                            format!("Invalid warehouse path: {}", warehouse_path)
                        })?;
                        let bucket = url
                            .host_str()
                            .with_context(|| {
                                format!("Invalid s3 path: {}, bucket is missing", warehouse_path)
                            })?
                            .to_string();
                        let root = url.path().trim_start_matches('/').to_string();
                        (bucket, root)
                    };

                    iceberg_configs.insert("iceberg.table.io.bucket".to_string(), bucket);

                    // Only storage catalog should set this.
                    if catalog_type == "storage" {
                        iceberg_configs.insert("iceberg.table.io.root".to_string(), root);
                    }
                }
                None => {
                    if catalog_type == "storage" {
                        bail!("`warehouse.path` must be set in storage catalog");
                    }
                }
            }

            // #TODO
            // Support load config file
            iceberg_configs.insert(
                "iceberg.table.io.disable_config_load".to_string(),
                "true".to_string(),
            );

            Ok(iceberg_configs)
        }

        /// TODO: remove the arguments and put them into `IcebergCommon`. Currently the handling in source and sink are different, so pass them separately to be safer.
        pub async fn create_catalog(
            &self,
            java_catalog_props: &HashMap<String, String>,
        ) -> ConnectorResult<CatalogRef> {
            match self.catalog_type() {
                "storage" | "rest_rust" => {
                    let iceberg_configs = self.build_iceberg_configs()?;
                    let catalog = load_catalog(&iceberg_configs).await?;
                    Ok(catalog)
                }
                catalog_type
                    if catalog_type == "hive"
                        || catalog_type == "jdbc"
                        || catalog_type == "glue"
                        || catalog_type == "rest" =>
                {
                    // Create java catalog
                    let (base_catalog_config, java_catalog_props) =
                        self.build_jni_catalog_configs(java_catalog_props)?;
                    let catalog_impl = match catalog_type {
                        "hive" => "org.apache.iceberg.hive.HiveCatalog",
                        "jdbc" => "org.apache.iceberg.jdbc.JdbcCatalog",
                        "glue" => "org.apache.iceberg.aws.glue.GlueCatalog",
                        "rest" => "org.apache.iceberg.rest.RESTCatalog",
                        _ => unreachable!(),
                    };

                    jni_catalog::JniCatalog::build_catalog(
                        base_catalog_config,
                        self.catalog_name(),
                        catalog_impl,
                        java_catalog_props,
                    )
                }
                "mock" => Ok(Arc::new(mock_catalog::MockCatalog {})),
                _ => {
                    bail!(
                    "Unsupported catalog type: {}, only support `storage`, `rest`, `hive`, `jdbc`, `glue`",
                    self.catalog_type()
                )
                }
            }
        }

        /// TODO: remove the arguments and put them into `IcebergCommon`. Currently the handling in source and sink are different, so pass them separately to be safer.
        pub async fn load_table(
            &self,
            java_catalog_props: &HashMap<String, String>,
        ) -> ConnectorResult<Table> {
            let catalog = self
                .create_catalog(java_catalog_props)
                .await
                .context("Unable to load iceberg catalog")?;

            let table_id = self
                .full_table_name()
                .context("Unable to parse table name")?;

            catalog.load_table(&table_id).await.map_err(Into::into)
        }
    }
}

/// iceberg-rust
mod v2 {
    use anyhow::anyhow;
    use iceberg::spec::TableMetadata;
    use iceberg::table::Table as TableV2;
    use iceberg::{Catalog as CatalogV2, TableIdent};
    use iceberg_catalog_glue::{AWS_ACCESS_KEY_ID, AWS_REGION_NAME, AWS_SECRET_ACCESS_KEY};

    use super::*;

    impl IcebergCommon {
        pub fn full_table_name_v2(&self) -> ConnectorResult<TableIdent> {
            let ret = if let Some(database_name) = &self.database_name {
                TableIdent::from_strs(vec![database_name, &self.table_name])
            } else {
                TableIdent::from_strs(vec![&self.table_name])
            };

            Ok(ret.context("Failed to create table identifier")?)
        }

        /// TODO: remove the arguments and put them into `IcebergCommon`. Currently the handling in source and sink are different, so pass them separately to be safer.
        pub async fn create_catalog_v2(
            &self,
            java_catalog_props: &HashMap<String, String>,
        ) -> ConnectorResult<Arc<dyn CatalogV2>> {
            match self.catalog_type() {
                "storage" => {
                    let config = storage_catalog::StorageCatalogConfig::builder()
                        .warehouse(self.warehouse_path.clone().ok_or_else(|| {
                            anyhow!("`warehouse.path` must be set in storage catalog")
                        })?)
                        .access_key(self.access_key.clone())
                        .secret_key(self.secret_key.clone())
                        .region(self.region.clone())
                        .endpoint(self.endpoint.clone())
                        .build();
                    let catalog = storage_catalog::StorageCatalog::new(config)?;
                    Ok(Arc::new(catalog))
                }
                "rest_rust" => {
                    let mut iceberg_configs = HashMap::new();
                    if let Some(region) = &self.region {
                        iceberg_configs.insert(S3_REGION.to_string(), region.clone().to_string());
                    }
                    if let Some(endpoint) = &self.endpoint {
                        iceberg_configs
                            .insert(S3_ENDPOINT.to_string(), endpoint.clone().to_string());
                    }
                    iceberg_configs.insert(
                        S3_ACCESS_KEY_ID.to_string(),
                        self.access_key.clone().to_string(),
                    );
                    iceberg_configs.insert(
                        S3_SECRET_ACCESS_KEY.to_string(),
                        self.secret_key.clone().to_string(),
                    );
                    if let Some(credential) = &self.credential {
                        iceberg_configs.insert("credential".to_string(), credential.clone());
                    }
                    if let Some(token) = &self.token {
                        iceberg_configs.insert("token".to_string(), token.clone());
                    }
                    if let Some(oauth2_server_uri) = &self.oauth2_server_uri {
                        iceberg_configs
                            .insert("oauth2-server-uri".to_string(), oauth2_server_uri.clone());
                    }
                    if let Some(scope) = &self.scope {
                        iceberg_configs.insert("scope".to_string(), scope.clone());
                    }

                    let config_builder = iceberg_catalog_rest::RestCatalogConfig::builder()
                        .uri(self.catalog_uri.clone().with_context(|| {
                            "`catalog.uri` must be set in rest catalog".to_string()
                        })?)
                        .props(iceberg_configs);

                    let config = match &self.warehouse_path {
                        Some(warehouse_path) => {
                            config_builder.warehouse(warehouse_path.clone()).build()
                        }
                        None => config_builder.build(),
                    };
                    let catalog = iceberg_catalog_rest::RestCatalog::new(config);
                    Ok(Arc::new(catalog))
                }
                "glue" => {
                    let mut iceberg_configs = HashMap::new();
                    // glue
                    if let Some(region) = &self.region {
                        iceberg_configs
                            .insert(AWS_REGION_NAME.to_string(), region.clone().to_string());
                    }
                    iceberg_configs.insert(
                        AWS_ACCESS_KEY_ID.to_string(),
                        self.access_key.clone().to_string(),
                    );
                    iceberg_configs.insert(
                        AWS_SECRET_ACCESS_KEY.to_string(),
                        self.secret_key.clone().to_string(),
                    );
                    // s3
                    if let Some(region) = &self.region {
                        iceberg_configs.insert(S3_REGION.to_string(), region.clone().to_string());
                    }
                    if let Some(endpoint) = &self.endpoint {
                        iceberg_configs
                            .insert(S3_ENDPOINT.to_string(), endpoint.clone().to_string());
                    }
                    iceberg_configs.insert(
                        S3_ACCESS_KEY_ID.to_string(),
                        self.access_key.clone().to_string(),
                    );
                    iceberg_configs.insert(
                        S3_SECRET_ACCESS_KEY.to_string(),
                        self.secret_key.clone().to_string(),
                    );
                    let config_builder = iceberg_catalog_glue::GlueCatalogConfig::builder()
                        .warehouse(self.warehouse_path.clone().ok_or_else(|| {
                            anyhow!("`warehouse.path` must be set in glue catalog")
                        })?)
                        .props(iceberg_configs);
                    let config = if let Some(uri) = self.catalog_uri.as_deref() {
                        config_builder.uri(uri.to_string()).build()
                    } else {
                        config_builder.build()
                    };
                    let catalog = iceberg_catalog_glue::GlueCatalog::new(config).await?;
                    Ok(Arc::new(catalog))
                }
                catalog_type
                    if catalog_type == "hive"
                        || catalog_type == "jdbc"
                        || catalog_type == "rest" =>
                {
                    // Create java catalog
                    let (base_catalog_config, java_catalog_props) =
                        self.build_jni_catalog_configs(java_catalog_props)?;
                    let catalog_impl = match catalog_type {
                        "hive" => "org.apache.iceberg.hive.HiveCatalog",
                        "jdbc" => "org.apache.iceberg.jdbc.JdbcCatalog",
                        "rest" => "org.apache.iceberg.rest.RESTCatalog",
                        _ => unreachable!(),
                    };

                    jni_catalog::JniCatalog::build_catalog_v2(
                        base_catalog_config,
                        self.catalog_name(),
                        catalog_impl,
                        java_catalog_props,
                    )
                }
                _ => {
                    bail!(
                    "Unsupported catalog type: {}, only support `storage`, `rest`, `hive`, `jdbc`, `glue`",
                    self.catalog_type()
                )
                }
            }
        }

        /// TODO: remove the arguments and put them into `IcebergCommon`. Currently the handling in source and sink are different, so pass them separately to be safer.
        pub async fn load_table_v2(
            &self,
            java_catalog_props: &HashMap<String, String>,
        ) -> ConnectorResult<TableV2> {
            let catalog = self
                .create_catalog_v2(java_catalog_props)
                .await
                .context("Unable to load iceberg catalog")?;

            let table_id = self
                .full_table_name_v2()
                .context("Unable to parse table name")?;

            catalog.load_table(&table_id).await.map_err(Into::into)
        }

        pub async fn load_table_v2_with_metadata(
            &self,
            metadata: TableMetadata,
            java_catalog_props: &HashMap<String, String>,
        ) -> ConnectorResult<TableV2> {
            match self.catalog_type() {
                "storage" => {
                    let config = storage_catalog::StorageCatalogConfig::builder()
                        .warehouse(self.warehouse_path.clone().ok_or_else(|| {
                            anyhow!("`warehouse.path` must be set in storage catalog")
                        })?)
                        .access_key(self.access_key.clone())
                        .secret_key(self.secret_key.clone())
                        .region(self.region.clone())
                        .endpoint(self.endpoint.clone())
                        .build();
                    let storage_catalog = storage_catalog::StorageCatalog::new(config)?;

                    let table_id = self
                        .full_table_name_v2()
                        .context("Unable to parse table name")?;

                    Ok(iceberg::table::Table::builder()
                        .metadata(metadata)
                        .identifier(table_id)
                        .file_io(storage_catalog.file_io().clone())
                        // Only support readonly table for storage catalog now.
                        .readonly(true)
                        .build()?)
                }
                _ => self.load_table_v2(java_catalog_props).await,
            }
        }
    }
}