This is an automated email from the ASF dual-hosted git repository.
fanng pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/gravitino.git
The following commit(s) were added to refs/heads/main by this push:
new f753afa96 [#4938]feat(lakehouse-paimon): Support S3 filesystem for
Paimon catalog. (#4939)
f753afa96 is described below
commit f753afa96be1db5a599ecc2d69b39f76ab52a936
Author: Qi Yu <[email protected]>
AuthorDate: Thu Oct 10 10:37:56 2024 +0800
[#4938]feat(lakehouse-paimon): Support S3 filesystem for Paimon catalog.
(#4939)
### What changes were proposed in this pull request?
Add support for Paimon S3 filesystem.
Note: related documents will be added in another PR.
### Why are the changes needed?
for better user experience.
Fix: #4938
### Does this PR introduce _any_ user-facing change?
N/A
### How was this patch tested?
Test locally and IT
---
catalogs/catalog-common/build.gradle.kts | 3 +-
catalogs/catalog-lakehouse-paimon/build.gradle.kts | 6 +
.../paimon/PaimonCatalogPropertiesMetadata.java | 19 +++
.../lakehouse/paimon/storage/S3StorageConfig.java | 105 +++++++++++++++++
.../lakehouse/paimon/utils/CatalogUtils.java | 10 ++
.../integration/test/CatalogPaimonBaseIT.java | 17 ++-
.../paimon/integration/test/CatalogPaimonS3IT.java | 127 +++++++++++++++++++++
docs/lakehouse-paimon-catalog.md | 5 +-
gradle/libs.versions.toml | 3 +
.../util/IcebergRESTServerManagerForDeploy.java | 4 +-
.../integration/test/util/AbstractIT.java | 11 +-
...cDriverDownloader.java => DownloaderUtils.java} | 12 +-
12 files changed, 301 insertions(+), 21 deletions(-)
diff --git a/catalogs/catalog-common/build.gradle.kts
b/catalogs/catalog-common/build.gradle.kts
index c9a76ca90..4cae8c530 100644
--- a/catalogs/catalog-common/build.gradle.kts
+++ b/catalogs/catalog-common/build.gradle.kts
@@ -23,8 +23,9 @@ plugins {
// try to avoid adding extra dependencies because it is used by catalogs and
connectors.
dependencies {
- implementation(libs.slf4j.api)
+ implementation(libs.commons.lang3)
implementation(libs.guava)
+ implementation(libs.slf4j.api)
testImplementation(libs.junit.jupiter.api)
testImplementation(libs.junit.jupiter.params)
diff --git a/catalogs/catalog-lakehouse-paimon/build.gradle.kts
b/catalogs/catalog-lakehouse-paimon/build.gradle.kts
index f30fd8880..c81959ee7 100644
--- a/catalogs/catalog-lakehouse-paimon/build.gradle.kts
+++ b/catalogs/catalog-lakehouse-paimon/build.gradle.kts
@@ -33,6 +33,9 @@ dependencies {
implementation(project(":api")) {
exclude("*")
}
+ implementation(project(":catalogs:catalog-common")) {
+ exclude("*")
+ }
implementation(project(":common")) {
exclude("*")
}
@@ -121,7 +124,10 @@ dependencies {
testImplementation(libs.postgresql.driver)
testImplementation(libs.bundles.log4j)
testImplementation(libs.junit.jupiter.params)
+ testImplementation(libs.paimon.s3)
+ testImplementation(libs.paimon.spark)
testImplementation(libs.testcontainers)
+ testImplementation(libs.testcontainers.localstack)
testRuntimeOnly(libs.junit.jupiter.engine)
}
diff --git
a/catalogs/catalog-lakehouse-paimon/src/main/java/org/apache/gravitino/catalog/lakehouse/paimon/PaimonCatalogPropertiesMetadata.java
b/catalogs/catalog-lakehouse-paimon/src/main/java/org/apache/gravitino/catalog/lakehouse/paimon/PaimonCatalogPropertiesMetadata.java
index 9b7d45c77..589d79672 100644
---
a/catalogs/catalog-lakehouse-paimon/src/main/java/org/apache/gravitino/catalog/lakehouse/paimon/PaimonCatalogPropertiesMetadata.java
+++
b/catalogs/catalog-lakehouse-paimon/src/main/java/org/apache/gravitino/catalog/lakehouse/paimon/PaimonCatalogPropertiesMetadata.java
@@ -31,9 +31,11 @@ import java.util.List;
import java.util.Map;
import
org.apache.gravitino.catalog.lakehouse.paimon.authentication.AuthenticationConfig;
import
org.apache.gravitino.catalog.lakehouse.paimon.authentication.kerberos.KerberosConfig;
+import org.apache.gravitino.catalog.lakehouse.paimon.storage.S3StorageConfig;
import org.apache.gravitino.connector.BaseCatalogPropertiesMetadata;
import org.apache.gravitino.connector.PropertiesMetadata;
import org.apache.gravitino.connector.PropertyEntry;
+import org.apache.gravitino.storage.S3Properties;
/**
* Implementation of {@link PropertiesMetadata} that represents Paimon catalog
properties metadata.
@@ -45,6 +47,11 @@ public class PaimonCatalogPropertiesMetadata extends
BaseCatalogPropertiesMetada
public static final String WAREHOUSE = "warehouse";
public static final String URI = "uri";
+ // S3 properties needed by Paimon
+ public static final String S3_ENDPOINT = "s3.endpoint";
+ public static final String S3_ACCESS_KEY = "s3.access-key";
+ public static final String S3_SECRET_KEY = "s3.secret-key";
+
public static final Map<String, String> GRAVITINO_CONFIG_TO_PAIMON =
ImmutableMap.of(GRAVITINO_CATALOG_BACKEND, PAIMON_METASTORE, WAREHOUSE,
WAREHOUSE, URI, URI);
private static final Map<String, PropertyEntry<?>> PROPERTIES_METADATA;
@@ -61,6 +68,12 @@ public class PaimonCatalogPropertiesMetadata extends
BaseCatalogPropertiesMetada
AuthenticationConfig.AUTH_TYPE_KEY,
AuthenticationConfig.AUTH_TYPE_KEY);
+ public static final Map<String, String> S3_CONFIGURATION =
+ ImmutableMap.of(
+ S3Properties.GRAVITINO_S3_ACCESS_KEY_ID, S3_ACCESS_KEY,
+ S3Properties.GRAVITINO_S3_SECRET_ACCESS_KEY, S3_SECRET_KEY,
+ S3Properties.GRAVITINO_S3_ENDPOINT, S3_ENDPOINT);
+
static {
List<PropertyEntry<?>> propertyEntries =
ImmutableList.of(
@@ -88,6 +101,8 @@ public class PaimonCatalogPropertiesMetadata extends
BaseCatalogPropertiesMetada
result.putAll(Maps.uniqueIndex(propertyEntries, PropertyEntry::getName));
result.putAll(KerberosConfig.KERBEROS_PROPERTY_ENTRIES);
result.putAll(AuthenticationConfig.AUTHENTICATION_PROPERTY_ENTRIES);
+ result.putAll(S3StorageConfig.S3_FILESYSTEM_PROPERTY_ENTRIES);
+
PROPERTIES_METADATA = ImmutableMap.copyOf(result);
}
@@ -107,6 +122,10 @@ public class PaimonCatalogPropertiesMetadata extends
BaseCatalogPropertiesMetada
if (KERBEROS_CONFIGURATION.containsKey(key)) {
gravitinoConfig.put(KERBEROS_CONFIGURATION.get(key), value);
}
+
+ if (S3_CONFIGURATION.containsKey(key)) {
+ gravitinoConfig.put(S3_CONFIGURATION.get(key), value);
+ }
});
return gravitinoConfig;
}
diff --git
a/catalogs/catalog-lakehouse-paimon/src/main/java/org/apache/gravitino/catalog/lakehouse/paimon/storage/S3StorageConfig.java
b/catalogs/catalog-lakehouse-paimon/src/main/java/org/apache/gravitino/catalog/lakehouse/paimon/storage/S3StorageConfig.java
new file mode 100644
index 000000000..213589d78
--- /dev/null
+++
b/catalogs/catalog-lakehouse-paimon/src/main/java/org/apache/gravitino/catalog/lakehouse/paimon/storage/S3StorageConfig.java
@@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+package org.apache.gravitino.catalog.lakehouse.paimon.storage;
+
+import static
org.apache.gravitino.storage.S3Properties.GRAVITINO_S3_ACCESS_KEY_ID;
+import static org.apache.gravitino.storage.S3Properties.GRAVITINO_S3_ENDPOINT;
+import static
org.apache.gravitino.storage.S3Properties.GRAVITINO_S3_SECRET_ACCESS_KEY;
+
+import com.google.common.collect.ImmutableMap;
+import java.util.Map;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.gravitino.Config;
+import org.apache.gravitino.config.ConfigBuilder;
+import org.apache.gravitino.config.ConfigConstants;
+import org.apache.gravitino.config.ConfigEntry;
+import org.apache.gravitino.connector.PropertyEntry;
+
+public class S3StorageConfig extends Config {
+
+ public S3StorageConfig(Map<String, String> properties) {
+ super(false);
+ loadFromMap(properties, k -> true);
+ }
+
+ // Unified S3
+ public static final ConfigEntry<String> PAIMON_S3_ENDPOINT_ENTRY =
+ new ConfigBuilder(GRAVITINO_S3_ENDPOINT)
+ .doc("The endpoint of the AWS s3")
+ .version(ConfigConstants.VERSION_0_7_0)
+ .stringConf()
+ .checkValue(StringUtils::isNotBlank,
ConfigConstants.NOT_BLANK_ERROR_MSG)
+ .create();
+
+ public static final ConfigEntry<String> PAIMON_S3_ACCESS_KEY_ENTRY =
+ new ConfigBuilder(GRAVITINO_S3_ACCESS_KEY_ID)
+ .doc("The access key of the AWS s3")
+ .version(ConfigConstants.VERSION_0_7_0)
+ .stringConf()
+ .checkValue(StringUtils::isNotBlank,
ConfigConstants.NOT_BLANK_ERROR_MSG)
+ .create();
+
+ public static final ConfigEntry<String> PAIMON_S3_SECRET_KEY_ENTRY =
+ new ConfigBuilder(GRAVITINO_S3_SECRET_ACCESS_KEY)
+ .doc("The secret key of the AWS s3")
+ .version(ConfigConstants.VERSION_0_7_0)
+ .stringConf()
+ .checkValue(StringUtils::isNotBlank,
ConfigConstants.NOT_BLANK_ERROR_MSG)
+ .create();
+
+ public String getS3Endpoint() {
+ return get(PAIMON_S3_ENDPOINT_ENTRY);
+ }
+
+ public String getS3AccessKey() {
+ return get(PAIMON_S3_ACCESS_KEY_ENTRY);
+ }
+
+ public String getS3SecretKey() {
+ return get(PAIMON_S3_SECRET_KEY_ENTRY);
+ }
+
+ public static final Map<String, PropertyEntry<?>>
S3_FILESYSTEM_PROPERTY_ENTRIES =
+ new ImmutableMap.Builder<String, PropertyEntry<?>>()
+ .put(
+ GRAVITINO_S3_ENDPOINT,
+ PropertyEntry.stringOptionalPropertyEntry(
+ GRAVITINO_S3_ENDPOINT,
+ "The endpoint of the AWS s3",
+ false /* immutable */,
+ null /* defaultValue */,
+ false /* hidden */))
+ .put(
+ GRAVITINO_S3_ACCESS_KEY_ID,
+ PropertyEntry.stringOptionalPropertyEntry(
+ GRAVITINO_S3_ACCESS_KEY_ID,
+ "The access key of the AWS s3",
+ false /* immutable */,
+ null /* defaultValue */,
+ false /* hidden */))
+ .put(
+ GRAVITINO_S3_SECRET_ACCESS_KEY,
+ PropertyEntry.stringOptionalPropertyEntry(
+ GRAVITINO_S3_SECRET_ACCESS_KEY,
+ "The secret key of the AWS s3",
+ false /* immutable */,
+ null /* defaultValue */,
+ false /* hidden */))
+ .build();
+}
diff --git
a/catalogs/catalog-lakehouse-paimon/src/main/java/org/apache/gravitino/catalog/lakehouse/paimon/utils/CatalogUtils.java
b/catalogs/catalog-lakehouse-paimon/src/main/java/org/apache/gravitino/catalog/lakehouse/paimon/utils/CatalogUtils.java
index 848063904..ffce417a0 100644
---
a/catalogs/catalog-lakehouse-paimon/src/main/java/org/apache/gravitino/catalog/lakehouse/paimon/utils/CatalogUtils.java
+++
b/catalogs/catalog-lakehouse-paimon/src/main/java/org/apache/gravitino/catalog/lakehouse/paimon/utils/CatalogUtils.java
@@ -18,6 +18,7 @@
*/
package org.apache.gravitino.catalog.lakehouse.paimon.utils;
+import static
org.apache.gravitino.catalog.lakehouse.paimon.PaimonCatalogPropertiesMetadata.S3_CONFIGURATION;
import static
org.apache.gravitino.catalog.lakehouse.paimon.PaimonConfig.CATALOG_BACKEND;
import static
org.apache.gravitino.catalog.lakehouse.paimon.PaimonConfig.CATALOG_URI;
import static
org.apache.gravitino.catalog.lakehouse.paimon.PaimonConfig.CATALOG_WAREHOUSE;
@@ -26,6 +27,7 @@ import static
org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY
import com.google.common.base.Preconditions;
import java.io.File;
+import java.util.HashMap;
import java.util.Map;
import java.util.UUID;
import org.apache.commons.lang3.StringUtils;
@@ -120,4 +122,12 @@ public class CatalogUtils {
StringUtils.isNotBlank(uri), "Paimon Catalog uri can not be null or
empty.");
}
}
+
+ public static Map<String, String> toPaimonCatalogProperties(
+ Map<String, String> gravitinoProperties) {
+ Map<String, String> paimonProperties = new HashMap<>();
+ gravitinoProperties.forEach(
+ (key, value) ->
paimonProperties.put(S3_CONFIGURATION.getOrDefault(key, key), value));
+ return paimonProperties;
+ }
}
diff --git
a/catalogs/catalog-lakehouse-paimon/src/test/java/org/apache/gravitino/catalog/lakehouse/paimon/integration/test/CatalogPaimonBaseIT.java
b/catalogs/catalog-lakehouse-paimon/src/test/java/org/apache/gravitino/catalog/lakehouse/paimon/integration/test/CatalogPaimonBaseIT.java
index e2cb6d6e1..bd907d06d 100644
---
a/catalogs/catalog-lakehouse-paimon/src/test/java/org/apache/gravitino/catalog/lakehouse/paimon/integration/test/CatalogPaimonBaseIT.java
+++
b/catalogs/catalog-lakehouse-paimon/src/test/java/org/apache/gravitino/catalog/lakehouse/paimon/integration/test/CatalogPaimonBaseIT.java
@@ -109,12 +109,12 @@ public abstract class CatalogPaimonBaseIT extends
AbstractIT {
private GravitinoMetalake metalake;
private Catalog catalog;
private org.apache.paimon.catalog.Catalog paimonCatalog;
- private SparkSession spark;
+ protected SparkSession spark;
private Map<String, String> catalogProperties;
@BeforeAll
public void startup() {
- containerSuite.startHiveContainer();
+ startNecessaryContainers();
catalogProperties = initPaimonCatalogProperties();
createMetalake();
createCatalog();
@@ -122,6 +122,10 @@ public abstract class CatalogPaimonBaseIT extends
AbstractIT {
initSparkEnv();
}
+ protected void startNecessaryContainers() {
+ containerSuite.startHiveContainer();
+ }
+
@AfterAll
public void stop() {
clearTableAndSchema();
@@ -882,8 +886,13 @@ public abstract class CatalogPaimonBaseIT extends
AbstractIT {
Preconditions.checkArgument(
StringUtils.isNotBlank(type), "Paimon Catalog backend type can not be
null or empty.");
catalogProperties.put(PaimonCatalogPropertiesMetadata.PAIMON_METASTORE,
type);
+
+ // Why needs this conversion? Because PaimonCatalogOperations#initialize
will try to convert
+ // Gravitino general S3 properties to Paimon specific S3 properties.
+ Map<String, String> copy =
CatalogUtils.toPaimonCatalogProperties(catalogProperties);
+
PaimonBackendCatalogWrapper paimonBackendCatalogWrapper =
- CatalogUtils.loadCatalogBackend(new PaimonConfig(catalogProperties));
+ CatalogUtils.loadCatalogBackend(new PaimonConfig(copy));
paimonCatalog = paimonBackendCatalogWrapper.getCatalog();
}
@@ -926,7 +935,7 @@ public abstract class CatalogPaimonBaseIT extends
AbstractIT {
return properties;
}
- private void initSparkEnv() {
+ protected void initSparkEnv() {
spark =
SparkSession.builder()
.master("local[1]")
diff --git
a/catalogs/catalog-lakehouse-paimon/src/test/java/org/apache/gravitino/catalog/lakehouse/paimon/integration/test/CatalogPaimonS3IT.java
b/catalogs/catalog-lakehouse-paimon/src/test/java/org/apache/gravitino/catalog/lakehouse/paimon/integration/test/CatalogPaimonS3IT.java
new file mode 100644
index 000000000..2030bbd19
--- /dev/null
+++
b/catalogs/catalog-lakehouse-paimon/src/test/java/org/apache/gravitino/catalog/lakehouse/paimon/integration/test/CatalogPaimonS3IT.java
@@ -0,0 +1,127 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+package org.apache.gravitino.catalog.lakehouse.paimon.integration.test;
+
+import com.google.common.collect.Maps;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import
org.apache.gravitino.catalog.lakehouse.paimon.PaimonCatalogPropertiesMetadata;
+import
org.apache.gravitino.integration.test.container.GravitinoLocalStackContainer;
+import org.apache.gravitino.integration.test.util.DownloaderUtils;
+import org.apache.gravitino.integration.test.util.ITUtils;
+import org.apache.gravitino.storage.S3Properties;
+import org.apache.spark.sql.SparkSession;
+import org.junit.jupiter.api.Tag;
+import org.junit.jupiter.api.TestInstance;
+import org.testcontainers.containers.Container;
+import org.testcontainers.shaded.org.awaitility.Awaitility;
+
+@Tag("gravitino-docker-test")
+@TestInstance(TestInstance.Lifecycle.PER_CLASS)
+public class CatalogPaimonS3IT extends CatalogPaimonBaseIT {
+
+ private static final String S3_BUCKET_NAME = "my-test-bucket";
+ private static GravitinoLocalStackContainer localStackContainer;
+ private String accessKey;
+ private String secretKey;
+ private String endpoint;
+
+ private static final String PAIMON_S3_JAR_URL =
+
"https://repo1.maven.org/maven2/org/apache/paimon/paimon-s3/0.8.0/paimon-s3-0.8.0.jar";
+
+ @Override
+ protected Map<String, String> initPaimonCatalogProperties() {
+
+ Map<String, String> catalogProperties = Maps.newHashMap();
+ catalogProperties.put("key1", "val1");
+ catalogProperties.put("key2", "val2");
+
+ TYPE = "filesystem";
+ WAREHOUSE = "s3://" + S3_BUCKET_NAME + "/";
+
+ accessKey = "accessKey";
+ secretKey = "secretKey";
+ endpoint = String.format("http://%s:%d",
localStackContainer.getContainerIpAddress(), 4566);
+
+
catalogProperties.put(PaimonCatalogPropertiesMetadata.GRAVITINO_CATALOG_BACKEND,
TYPE);
+ catalogProperties.put(PaimonCatalogPropertiesMetadata.WAREHOUSE,
WAREHOUSE);
+ catalogProperties.put(S3Properties.GRAVITINO_S3_ACCESS_KEY_ID, accessKey);
+ catalogProperties.put(S3Properties.GRAVITINO_S3_SECRET_ACCESS_KEY,
secretKey);
+ catalogProperties.put(S3Properties.GRAVITINO_S3_ENDPOINT, endpoint);
+
+ // Need to download the S3 dependency in the deploy mode.
+ downloadS3Dependency();
+
+ return catalogProperties;
+ }
+
+ private void downloadS3Dependency() {
+ String gravitinoHome = System.getenv("GRAVITINO_HOME");
+ try {
+ if (!ITUtils.EMBEDDED_TEST_MODE.equals(testMode)) {
+ String serverPath = ITUtils.joinPath(gravitinoHome, "libs");
+ String paimonCatalogPath =
+ ITUtils.joinPath(gravitinoHome, "catalogs", "lakehouse-paimon",
"libs");
+ DownloaderUtils.downloadFile(PAIMON_S3_JAR_URL, serverPath,
paimonCatalogPath);
+ }
+ } catch (Exception e) {
+ throw new RuntimeException("Failed to download the S3 dependency", e);
+ }
+ }
+
+ @Override
+ protected void startNecessaryContainers() {
+ containerSuite.startLocalStackContainer();
+ localStackContainer = containerSuite.getLocalStackContainer();
+
+ Awaitility.await()
+ .atMost(60, TimeUnit.SECONDS)
+ .pollInterval(1, TimeUnit.SECONDS)
+ .until(
+ () -> {
+ try {
+ Container.ExecResult result =
+ localStackContainer.executeInContainer(
+ "awslocal", "s3", "mb", "s3://" + S3_BUCKET_NAME);
+ return result.getExitCode() == 0;
+ } catch (Exception e) {
+ return false;
+ }
+ });
+ }
+
+ protected void initSparkEnv() {
+ spark =
+ SparkSession.builder()
+ .master("local[1]")
+ .appName("Paimon Catalog integration test")
+ .config("spark.sql.warehouse.dir", WAREHOUSE)
+ .config("spark.sql.catalog.paimon",
"org.apache.paimon.spark.SparkCatalog")
+ .config("spark.sql.catalog.paimon.warehouse", WAREHOUSE)
+ .config(
+ "spark.sql.extensions",
+
"org.apache.paimon.spark.extensions.PaimonSparkSessionExtensions")
+ .config("spark.sql.catalog.paimon.s3.access-key", accessKey)
+ .config("spark.sql.catalog.paimon.s3.secret-key", secretKey)
+ .config("spark.sql.catalog.paimon.s3.endpoint", endpoint)
+ .enableHiveSupport()
+ .getOrCreate();
+ }
+}
diff --git a/docs/lakehouse-paimon-catalog.md b/docs/lakehouse-paimon-catalog.md
index fa1e04523..03a36408e 100644
--- a/docs/lakehouse-paimon-catalog.md
+++ b/docs/lakehouse-paimon-catalog.md
@@ -34,12 +34,15 @@ Builds with Apache Paimon `0.8.0`.
|----------------------------------------------------|-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|------------------------|-----------------------------------------------------------------|---------------|
| `catalog-backend` | Catalog backend of
Gravitino Paimon catalog. Only supports `filesystem` now.
| (none) | Yes
| 0.6.0 |
| `uri` | The URI configuration
of the Paimon catalog. `thrift://127.0.0.1:9083` or
`jdbc:postgresql://127.0.0.1:5432/db_name` or
`jdbc:mysql://127.0.0.1:3306/metastore_db`. It is optional for
`FilesystemCatalog`. | (none) | required if the value of
`catalog-backend` is not `filesystem`. | 0.6.0 |
-| `warehouse` | Warehouse directory of
catalog. `file:///user/hive/warehouse-paimon/` for local fs or
`hdfs://namespace/hdfs/path` for HDFS.
| (none) | Yes
| 0.6.0 |
+| `warehouse` | Warehouse directory of
catalog. `file:///user/hive/warehouse-paimon/` for local fs,
`hdfs://namespace/hdfs/path` for HDFS or `s3://{bucket-name}/path/` for S3
| (none) | Yes
| 0.6.0 |
| `authentication.type` | The type of
authentication for Paimon catalog backend, currently Gravitino only supports
`Kerberos` and `simple`.
| `simple` | No
| 0.6.0 |
| `authentication.kerberos.principal` | The principal of the
Kerberos authentication.
| (none) | required if the value of
`authentication.type` is Kerberos. | 0.6.0 |
| `authentication.kerberos.keytab-uri` | The URI of The keytab
for the Kerberos authentication.
| (none) | required if the value of
`authentication.type` is Kerberos. | 0.6.0 |
| `authentication.kerberos.check-interval-sec` | The check interval of
Kerberos credential for Paimon catalog.
| 60 | No
| 0.6.0 |
| `authentication.kerberos.keytab-fetch-timeout-sec` | The fetch timeout of
retrieving Kerberos keytab from `authentication.kerberos.keytab-uri`.
| 60 | No
| 0.6.0 |
+| `s3-endpoint` | The endpoint of the AWS
s3.
| (none) | required if the value of
`warehouse` is a S3 path | 0.7.0 |
+| `s3-access-key-id` | The access key of the
AWS s3.
| (none) | required if the value of
`warehouse` is a S3 path | 0.7.0 |
+| `s3-secret-access-key` | The secret key of the
AWS s3.
| (none) | required if the value of
`warehouse` is a S3 path | 0.7.0 |
Any properties not defined by Gravitino with `gravitino.bypass.` prefix will
pass to Paimon catalog properties and HDFS configuration. For example, if
specify `gravitino.bypass.table.type`, `table.type` will pass to Paimon catalog
properties.
diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml
index 255306c98..5e1446a8f 100644
--- a/gradle/libs.versions.toml
+++ b/gradle/libs.versions.toml
@@ -173,6 +173,9 @@ iceberg-gcp = { group = "org.apache.iceberg", name =
"iceberg-gcp", version.ref
paimon-core = { group = "org.apache.paimon", name = "paimon-core", version.ref
= "paimon" }
paimon-format = { group = "org.apache.paimon", name = "paimon-format",
version.ref = "paimon" }
paimon-hive-catalog = { group = "org.apache.paimon", name =
"paimon-hive-catalog", version.ref = "paimon" }
+paimon-s3 = { group = "org.apache.paimon", name = "paimon-s3", version.ref =
"paimon" }
+paimon-spark = { group = "org.apache.paimon", name = "paimon-spark",
version.ref = "paimon" }
+
trino-spi= { group = "io.trino", name = "trino-spi", version.ref = "trino" }
trino-testing= { group = "io.trino", name = "trino-testing", version.ref =
"trino" }
trino-memory= { group = "io.trino", name = "trino-memory", version.ref =
"trino" }
diff --git
a/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/integration/test/util/IcebergRESTServerManagerForDeploy.java
b/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/integration/test/util/IcebergRESTServerManagerForDeploy.java
index 135713223..ca4ba0cc4 100644
---
a/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/integration/test/util/IcebergRESTServerManagerForDeploy.java
+++
b/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/integration/test/util/IcebergRESTServerManagerForDeploy.java
@@ -27,7 +27,7 @@ import java.util.UUID;
import java.util.concurrent.Future;
import org.apache.commons.io.FileUtils;
import org.apache.gravitino.integration.test.util.CommandExecutor;
-import org.apache.gravitino.integration.test.util.JdbcDriverDownloader;
+import org.apache.gravitino.integration.test.util.DownloaderUtils;
import org.apache.gravitino.integration.test.util.ProcessData;
import org.apache.gravitino.integration.test.util.ProcessData.TypesOfData;
import org.testcontainers.shaded.com.google.common.collect.ImmutableMap;
@@ -51,7 +51,7 @@ public class IcebergRESTServerManagerForDeploy extends
IcebergRESTServerManager
@Override
public Optional<Future<?>> doStartIcebergRESTServer() throws Exception {
- JdbcDriverDownloader.downloadJdbcDriver(
+ DownloaderUtils.downloadFile(
SQLITE_DRIVER_DOWNLOAD_URL,
Paths.get(icebergRESTServerHome.toString(), "iceberg-rest-server",
"libs").toString());
diff --git
a/integration-test-common/src/test/java/org/apache/gravitino/integration/test/util/AbstractIT.java
b/integration-test-common/src/test/java/org/apache/gravitino/integration/test/util/AbstractIT.java
index 71d92d6c9..0ef754d5e 100644
---
a/integration-test-common/src/test/java/org/apache/gravitino/integration/test/util/AbstractIT.java
+++
b/integration-test-common/src/test/java/org/apache/gravitino/integration/test/util/AbstractIT.java
@@ -145,18 +145,15 @@ public class AbstractIT {
String serverPath = ITUtils.joinPath(gravitinoHome, "libs");
String icebergCatalogPath =
ITUtils.joinPath(gravitinoHome, "catalogs", "lakehouse-iceberg",
"libs");
- JdbcDriverDownloader.downloadJdbcDriver(
- DOWNLOAD_MYSQL_JDBC_DRIVER_URL, serverPath, icebergCatalogPath);
- JdbcDriverDownloader.downloadJdbcDriver(
+ DownloaderUtils.downloadFile(DOWNLOAD_MYSQL_JDBC_DRIVER_URL, serverPath,
icebergCatalogPath);
+ DownloaderUtils.downloadFile(
DOWNLOAD_POSTGRESQL_JDBC_DRIVER_URL, serverPath, icebergCatalogPath);
} else {
Path icebergLibsPath =
Paths.get(gravitinoHome, "catalogs", "catalog-lakehouse-iceberg",
"build", "libs");
- JdbcDriverDownloader.downloadJdbcDriver(
- DOWNLOAD_MYSQL_JDBC_DRIVER_URL, icebergLibsPath.toString());
+ DownloaderUtils.downloadFile(DOWNLOAD_MYSQL_JDBC_DRIVER_URL,
icebergLibsPath.toString());
- JdbcDriverDownloader.downloadJdbcDriver(
- DOWNLOAD_POSTGRESQL_JDBC_DRIVER_URL, icebergLibsPath.toString());
+ DownloaderUtils.downloadFile(DOWNLOAD_POSTGRESQL_JDBC_DRIVER_URL,
icebergLibsPath.toString());
}
}
diff --git
a/integration-test-common/src/test/java/org/apache/gravitino/integration/test/util/JdbcDriverDownloader.java
b/integration-test-common/src/test/java/org/apache/gravitino/integration/test/util/DownloaderUtils.java
similarity index 84%
rename from
integration-test-common/src/test/java/org/apache/gravitino/integration/test/util/JdbcDriverDownloader.java
rename to
integration-test-common/src/test/java/org/apache/gravitino/integration/test/util/DownloaderUtils.java
index f53417248..0d4e12ee0 100644
---
a/integration-test-common/src/test/java/org/apache/gravitino/integration/test/util/JdbcDriverDownloader.java
+++
b/integration-test-common/src/test/java/org/apache/gravitino/integration/test/util/DownloaderUtils.java
@@ -30,20 +30,20 @@ import org.junit.jupiter.api.Assertions;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-public class JdbcDriverDownloader {
+public class DownloaderUtils {
- public static final Logger LOG =
LoggerFactory.getLogger(JdbcDriverDownloader.class);
+ public static final Logger LOG =
LoggerFactory.getLogger(DownloaderUtils.class);
- public static void downloadJdbcDriver(String jdbcDriverUrl, String...
destinationDirectories)
+ public static void downloadFile(String fileUrl, String...
destinationDirectories)
throws IOException {
- URL url = new URL(jdbcDriverUrl);
+ URL url = new URL(fileUrl);
URLConnection connection = url.openConnection();
String fileName = getFileName(url);
String destinationDirectory = destinationDirectories[0];
Path destinationPath = Paths.get(destinationDirectory, fileName);
File file = new File(destinationPath.toString());
if (!file.exists()) {
- LOG.info("Start download jdbc-driver from:{}", jdbcDriverUrl);
+ LOG.info("Start download file from:{}", fileUrl);
try (InputStream in = connection.getInputStream()) {
if (!Files.exists(Paths.get(destinationDirectory))) {
@@ -52,7 +52,7 @@ public class JdbcDriverDownloader {
Files.copy(in, destinationPath,
java.nio.file.StandardCopyOption.REPLACE_EXISTING);
Assertions.assertTrue(new File(destinationPath.toString()).exists());
- LOG.info("Download jdbc-driver:{} success. path:{}", fileName,
destinationPath);
+ LOG.info("Download file:{} success. path:{}", fileName,
destinationPath);
}
}
for (int i = 1; i < destinationDirectories.length; i++) {