diff --git a/catalogs/catalog-lakehouse-iceberg/build.gradle.kts b/catalogs/catalog-lakehouse-iceberg/build.gradle.kts index b8ed8c6b4be..76b5a6d06d9 100644 --- a/catalogs/catalog-lakehouse-iceberg/build.gradle.kts +++ b/catalogs/catalog-lakehouse-iceberg/build.gradle.kts @@ -38,6 +38,7 @@ dependencies { exclude("org.slf4j") } implementation(libs.iceberg.hive.metastore) + implementation(libs.sqlite.jdbc) implementation(libs.hive2.metastore) { exclude("org.apache.avro", "avro") diff --git a/catalogs/catalog-lakehouse-iceberg/src/main/java/com/datastrato/gravitino/catalog/lakehouse/iceberg/IcebergConfig.java b/catalogs/catalog-lakehouse-iceberg/src/main/java/com/datastrato/gravitino/catalog/lakehouse/iceberg/IcebergConfig.java index df97d407b93..56a2f8232cc 100644 --- a/catalogs/catalog-lakehouse-iceberg/src/main/java/com/datastrato/gravitino/catalog/lakehouse/iceberg/IcebergConfig.java +++ b/catalogs/catalog-lakehouse-iceberg/src/main/java/com/datastrato/gravitino/catalog/lakehouse/iceberg/IcebergConfig.java @@ -6,6 +6,9 @@ package com.datastrato.gravitino.catalog.lakehouse.iceberg; import static com.datastrato.gravitino.catalog.lakehouse.iceberg.IcebergCatalogPropertiesMetadata.CATALOG_BACKEND_NAME; +import static com.datastrato.gravitino.catalog.lakehouse.iceberg.IcebergCatalogPropertiesMetadata.ICEBERG_JDBC_INITIALIZE; +import static com.datastrato.gravitino.catalog.lakehouse.iceberg.IcebergCatalogPropertiesMetadata.ICEBERG_JDBC_PASSWORD; +import static com.datastrato.gravitino.catalog.lakehouse.iceberg.IcebergCatalogPropertiesMetadata.ICEBERG_JDBC_USER; import static com.datastrato.gravitino.catalog.lakehouse.iceberg.IcebergCatalogPropertiesMetadata.URI; import static com.datastrato.gravitino.catalog.lakehouse.iceberg.IcebergCatalogPropertiesMetadata.WAREHOUSE; @@ -36,6 +39,27 @@ public class IcebergConfig extends Config { .stringConf() .createWithDefault(null); + public static final ConfigEntry JDBC_USER = + new ConfigBuilder(ICEBERG_JDBC_USER) + .doc("The username of the Jdbc connection") + .version(DEFAULT_VERSION) + .stringConf() + .createWithDefault(null); + + public static final ConfigEntry JDBC_PASSWORD = + new ConfigBuilder(ICEBERG_JDBC_PASSWORD) + .doc("The password of the Jdbc connection") + .version(DEFAULT_VERSION) + .stringConf() + .createWithDefault(null); + + public static final ConfigEntry JDBC_INIT_TABLES = + new ConfigBuilder(ICEBERG_JDBC_INITIALIZE) + .doc("Whether to initialize meta tables when create Jdbc catalog") + .version(DEFAULT_VERSION) + .booleanConf() + .createWithDefault(true); + public IcebergConfig() { super(false); } diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 27ef2349d35..3c4f526048a 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -29,6 +29,7 @@ iceberg = '1.3.1' trino = '426' spark = "3.4.1" scala-collection-compat = "2.7.0" +sqlite-jdbc = "3.42.0.0" protobuf-plugin = "0.9.2" @@ -93,6 +94,7 @@ trino-testing= { group = "io.trino", name = "trino-testing", version.ref = "trin iceberg-spark-runtime = { group = "org.apache.iceberg", name = "iceberg-spark-runtime-3.4_2.13", version.ref = "iceberg" } spark-sql = { group = "org.apache.spark", name = "spark-sql_2.13", version.ref = "spark" } scala-collection-compat = { group = "org.scala-lang.modules", name = "scala-collection-compat_2.13", version.ref = "scala-collection-compat" } +sqlite-jdbc = { group = "org.xerial", name = "sqlite-jdbc", version.ref = "sqlite-jdbc" } [bundles] diff --git a/integration-test/build.gradle.kts b/integration-test/build.gradle.kts index daf098b965d..59917dda581 100644 --- a/integration-test/build.gradle.kts +++ b/integration-test/build.gradle.kts @@ -107,6 +107,7 @@ dependencies { } testImplementation(libs.scala.collection.compat) testImplementation(libs.slf4j.jdk14) + testImplementation(libs.sqlite.jdbc) } /* Optimizing integration test execution conditions */ diff --git a/integration-test/src/test/java/com/datastrato/gravitino/integration/test/catalog/lakehouse/iceberg/IcebergRESTJdbcCatalogIT.java b/integration-test/src/test/java/com/datastrato/gravitino/integration/test/catalog/lakehouse/iceberg/IcebergRESTJdbcCatalogIT.java new file mode 100644 index 00000000000..dca629dca7b --- /dev/null +++ b/integration-test/src/test/java/com/datastrato/gravitino/integration/test/catalog/lakehouse/iceberg/IcebergRESTJdbcCatalogIT.java @@ -0,0 +1,17 @@ +/* + * Copyright 2023 Datastrato. + * This software is licensed under the Apache License version 2. + */ + +package com.datastrato.gravitino.integration.test.catalog.lakehouse.iceberg; + +import com.datastrato.gravitino.catalog.lakehouse.iceberg.IcebergCatalogBackend; +import org.junit.jupiter.api.TestInstance; +import org.junit.jupiter.api.TestInstance.Lifecycle; + +@TestInstance(Lifecycle.PER_CLASS) +public class IcebergRESTJdbcCatalogIT extends IcebergRESTServiceIT { + public IcebergRESTJdbcCatalogIT() { + catalogType = IcebergCatalogBackend.JDBC; + } +} diff --git a/integration-test/src/test/java/com/datastrato/gravitino/integration/test/catalog/lakehouse/iceberg/IcebergRESTServiceBaseIT.java b/integration-test/src/test/java/com/datastrato/gravitino/integration/test/catalog/lakehouse/iceberg/IcebergRESTServiceBaseIT.java index 57f2b4590cb..28b9c24ad6c 100644 --- a/integration-test/src/test/java/com/datastrato/gravitino/integration/test/catalog/lakehouse/iceberg/IcebergRESTServiceBaseIT.java +++ b/integration-test/src/test/java/com/datastrato/gravitino/integration/test/catalog/lakehouse/iceberg/IcebergRESTServiceBaseIT.java @@ -112,6 +112,49 @@ private static Map getIcebergMemoryCatalogConfigs() { private static Map getIcebergJdbcCatalogConfigs() { Map configMap = new HashMap<>(); + + configMap.put( + AuxiliaryServiceManager.GRAVITINO_AUX_SERVICE_PREFIX + + IcebergRESTService.SERVICE_NAME + + "." + + IcebergConfig.CATALOG_BACKEND.getKey(), + IcebergCatalogBackend.JDBC.toString().toLowerCase()); + + configMap.put( + AuxiliaryServiceManager.GRAVITINO_AUX_SERVICE_PREFIX + + IcebergRESTService.SERVICE_NAME + + "." + + IcebergConfig.CATALOG_URI.getKey(), + "jdbc:sqlite:/tmp/iceberg-rest-sqlite"); + + configMap.put( + AuxiliaryServiceManager.GRAVITINO_AUX_SERVICE_PREFIX + + IcebergRESTService.SERVICE_NAME + + "." + + IcebergConfig.JDBC_USER.getKey(), + "iceberg"); + + configMap.put( + AuxiliaryServiceManager.GRAVITINO_AUX_SERVICE_PREFIX + + IcebergRESTService.SERVICE_NAME + + "." + + IcebergConfig.JDBC_PASSWORD.getKey(), + "iceberg"); + + configMap.put( + AuxiliaryServiceManager.GRAVITINO_AUX_SERVICE_PREFIX + + IcebergRESTService.SERVICE_NAME + + "." + + IcebergConfig.JDBC_INIT_TABLES.getKey(), + "true"); + + configMap.put( + AuxiliaryServiceManager.GRAVITINO_AUX_SERVICE_PREFIX + + IcebergRESTService.SERVICE_NAME + + "." + + IcebergConfig.CATALOG_WAREHOUSE.getKey(), + "file:///tmp/user/hive/warehouse-jdbc-sqlite/"); + return configMap; }