From 6180ed6f7fbd7651619d18f77f763510e3284f6f Mon Sep 17 00:00:00 2001 From: yuqi Date: Fri, 13 Sep 2024 19:29:00 +0800 Subject: [PATCH 01/15] Support S3 filesystem for Paimon catalog. --- .../catalog-lakehouse-paimon/build.gradle.kts | 4 + .../paimon/PaimonCatalogOperations.java | 5 +- .../PaimonCatalogPropertiesMetadata.java | 12 ++ .../paimon/filesystem/FileSystemType.java | 47 ++++++++ .../s3/PaimonS3FileSystemConfig.java | 104 ++++++++++++++++++ .../lakehouse/paimon/utils/CatalogUtils.java | 34 ++++++ .../integration/test/CatalogPaimonBaseIT.java | 6 +- .../integration/test/CatalogPaimonS3IT.java | 102 +++++++++++++++++ gradle/libs.versions.toml | 6 + 9 files changed, 318 insertions(+), 2 deletions(-) create mode 100644 catalogs/catalog-lakehouse-paimon/src/main/java/org/apache/gravitino/catalog/lakehouse/paimon/filesystem/FileSystemType.java create mode 100644 catalogs/catalog-lakehouse-paimon/src/main/java/org/apache/gravitino/catalog/lakehouse/paimon/filesystem/s3/PaimonS3FileSystemConfig.java create mode 100644 catalogs/catalog-lakehouse-paimon/src/test/java/org/apache/gravitino/catalog/lakehouse/paimon/integration/test/CatalogPaimonS3IT.java diff --git a/catalogs/catalog-lakehouse-paimon/build.gradle.kts b/catalogs/catalog-lakehouse-paimon/build.gradle.kts index 3974fba61f3..0d6c75907fa 100644 --- a/catalogs/catalog-lakehouse-paimon/build.gradle.kts +++ b/catalogs/catalog-lakehouse-paimon/build.gradle.kts @@ -82,6 +82,10 @@ dependencies { testImplementation(libs.bundles.log4j) testImplementation(libs.junit.jupiter.params) testImplementation(libs.testcontainers) + testImplementation(libs.paimon.s3) + testImplementation(libs.paimon.spark) + testImplementation(libs.hadoop.aws) + 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/PaimonCatalogOperations.java b/catalogs/catalog-lakehouse-paimon/src/main/java/org/apache/gravitino/catalog/lakehouse/paimon/PaimonCatalogOperations.java index fbe6f4be49d..3188cd62cbe 100644 --- a/catalogs/catalog-lakehouse-paimon/src/main/java/org/apache/gravitino/catalog/lakehouse/paimon/PaimonCatalogOperations.java +++ b/catalogs/catalog-lakehouse-paimon/src/main/java/org/apache/gravitino/catalog/lakehouse/paimon/PaimonCatalogOperations.java @@ -39,6 +39,7 @@ import org.apache.gravitino.Namespace; import org.apache.gravitino.SchemaChange; import org.apache.gravitino.catalog.lakehouse.paimon.ops.PaimonCatalogOps; +import org.apache.gravitino.catalog.lakehouse.paimon.utils.CatalogUtils; import org.apache.gravitino.catalog.lakehouse.paimon.utils.TableOpsUtils; import org.apache.gravitino.connector.CatalogInfo; import org.apache.gravitino.connector.CatalogOperations; @@ -114,7 +115,9 @@ public void initialize( Map resultConf = Maps.newHashMap(prefixMap); resultConf.putAll(gravitinoConfig); - this.paimonCatalogOps = new PaimonCatalogOps(new PaimonConfig(resultConf)); + PaimonConfig paimonConfig = new PaimonConfig(resultConf); + CatalogUtils.checkWarehouseConfig(paimonConfig, resultConf); + this.paimonCatalogOps = new PaimonCatalogOps(paimonConfig); } /** 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 9b7d45c77bd..5d2ba9e753f 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,6 +31,7 @@ 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.filesystem.s3.PaimonS3FileSystemConfig; import org.apache.gravitino.connector.BaseCatalogPropertiesMetadata; import org.apache.gravitino.connector.PropertiesMetadata; import org.apache.gravitino.connector.PropertyEntry; @@ -61,6 +62,12 @@ public class PaimonCatalogPropertiesMetadata extends BaseCatalogPropertiesMetada AuthenticationConfig.AUTH_TYPE_KEY, AuthenticationConfig.AUTH_TYPE_KEY); + private static final Map S3_CONFIGURATION = + ImmutableMap.of( + PaimonS3FileSystemConfig.S3_ACCESS_KEY, PaimonS3FileSystemConfig.S3_ACCESS_KEY, + PaimonS3FileSystemConfig.S3_SECRET_KEY, PaimonS3FileSystemConfig.S3_SECRET_KEY, + PaimonS3FileSystemConfig.S3_ENDPOINT, PaimonS3FileSystemConfig.S3_ENDPOINT); + static { List> propertyEntries = ImmutableList.of( @@ -88,6 +95,7 @@ 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(PaimonS3FileSystemConfig.S3_FILESYSTEM_PROPERTY_ENTRIES); PROPERTIES_METADATA = ImmutableMap.copyOf(result); } @@ -107,6 +115,10 @@ protected Map transformProperties(Map properties 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/filesystem/FileSystemType.java b/catalogs/catalog-lakehouse-paimon/src/main/java/org/apache/gravitino/catalog/lakehouse/paimon/filesystem/FileSystemType.java new file mode 100644 index 00000000000..eb531a31dd0 --- /dev/null +++ b/catalogs/catalog-lakehouse-paimon/src/main/java/org/apache/gravitino/catalog/lakehouse/paimon/filesystem/FileSystemType.java @@ -0,0 +1,47 @@ +/* + * 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.filesystem; + +public enum FileSystemType { + HDFS, + S3, + OSS; + + public static FileSystemType fromString(String type) { + for (FileSystemType fileSystemType : FileSystemType.values()) { + if (fileSystemType.name().equalsIgnoreCase(type)) { + return fileSystemType; + } + } + + throw new IllegalArgumentException("Unsupported file system type: " + type); + } + + public static FileSystemType fromStoragePath(String storagePath) { + if (storagePath.startsWith("s3://")) { + return S3; + } else if (storagePath.startsWith("oss://")) { + return OSS; + } else if (storagePath.startsWith("hdfs://")) { + return HDFS; + } + + throw new IllegalArgumentException("Unsupported storage path: " + storagePath); + } +} diff --git a/catalogs/catalog-lakehouse-paimon/src/main/java/org/apache/gravitino/catalog/lakehouse/paimon/filesystem/s3/PaimonS3FileSystemConfig.java b/catalogs/catalog-lakehouse-paimon/src/main/java/org/apache/gravitino/catalog/lakehouse/paimon/filesystem/s3/PaimonS3FileSystemConfig.java new file mode 100644 index 00000000000..0b47a703908 --- /dev/null +++ b/catalogs/catalog-lakehouse-paimon/src/main/java/org/apache/gravitino/catalog/lakehouse/paimon/filesystem/s3/PaimonS3FileSystemConfig.java @@ -0,0 +1,104 @@ +/* + * 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.filesystem.s3; + +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 PaimonS3FileSystemConfig extends Config { + // S3 related properties + 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 PaimonS3FileSystemConfig(Map properties) { + super(false); + loadFromMap(properties, k -> true); + } + + public static final ConfigEntry PAIMON_S3_ENDPOINT_ENTRY = + new ConfigBuilder(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 PAIMON_S3_ACCESS_KEY_ENTRY = + new ConfigBuilder(S3_ACCESS_KEY) + .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 PAIMON_S3_SECRET_KEY_ENTRY = + new ConfigBuilder(S3_SECRET_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> S3_FILESYSTEM_PROPERTY_ENTRIES = + new ImmutableMap.Builder>() + .put( + S3_ENDPOINT, + PropertyEntry.stringOptionalPropertyEntry( + S3_ENDPOINT, + "The endpoint of the AWS s3", + true /* immutable */, + null /* defaultValue */, + false /* hidden */)) + .put( + S3_ACCESS_KEY, + PropertyEntry.stringOptionalPropertyEntry( + S3_ACCESS_KEY, + "The access key of the AWS s3", + true /* immutable */, + null /* defaultValue */, + false /* hidden */)) + .put( + S3_SECRET_KEY, + PropertyEntry.stringOptionalPropertyEntry( + S3_SECRET_KEY, + "The secret key of the AWS s3", + true /* 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 848063904f4..87f28a4f6af 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 @@ -33,6 +33,8 @@ import org.apache.gravitino.catalog.lakehouse.paimon.PaimonConfig; import org.apache.gravitino.catalog.lakehouse.paimon.authentication.AuthenticationConfig; import org.apache.gravitino.catalog.lakehouse.paimon.authentication.kerberos.KerberosClient; +import org.apache.gravitino.catalog.lakehouse.paimon.filesystem.FileSystemType; +import org.apache.gravitino.catalog.lakehouse.paimon.filesystem.s3.PaimonS3FileSystemConfig; import org.apache.gravitino.catalog.lakehouse.paimon.ops.PaimonBackendCatalogWrapper; import org.apache.hadoop.conf.Configuration; import org.apache.paimon.catalog.Catalog; @@ -120,4 +122,36 @@ private static void checkPaimonConfig(PaimonConfig paimonConfig) { StringUtils.isNotBlank(uri), "Paimon Catalog uri can not be null or empty."); } } + + public static void checkWarehouseConfig( + PaimonConfig paimonConfig, Map resultConf) { + String warehouse = paimonConfig.get(CATALOG_WAREHOUSE); + Preconditions.checkArgument( + StringUtils.isNotBlank(warehouse), "Paimon Catalog warehouse can not be null or empty."); + + FileSystemType fileSystemType = FileSystemType.fromStoragePath(warehouse); + switch (fileSystemType) { + case S3: + checkS3FileSystemConfig(resultConf); + break; + case HDFS: + case OSS: + break; + default: + throw new IllegalArgumentException("Unsupported file system type: " + fileSystemType); + } + } + + private static void checkS3FileSystemConfig(Map resultConf) { + PaimonS3FileSystemConfig s3FileSystemConfig = new PaimonS3FileSystemConfig(resultConf); + Preconditions.checkArgument( + StringUtils.isNotBlank(s3FileSystemConfig.getS3AccessKey()), + "S3 access key can not be null or empty."); + Preconditions.checkArgument( + StringUtils.isNotBlank(s3FileSystemConfig.getS3SecretKey()), + "S3 secret key can not be null or empty."); + Preconditions.checkArgument( + StringUtils.isNotBlank(s3FileSystemConfig.getS3Endpoint()), + "S3 endpoint can not be null or empty."); + } } 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 e2cb6d6e168..07a52735ee0 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 @@ -114,7 +114,7 @@ public abstract class CatalogPaimonBaseIT extends AbstractIT { @BeforeAll public void startup() { - containerSuite.startHiveContainer(); + startNecessaryContainers(); catalogProperties = initPaimonCatalogProperties(); createMetalake(); createCatalog(); @@ -122,6 +122,10 @@ public void startup() { initSparkEnv(); } + protected void startNecessaryContainers() { + containerSuite.startHiveContainer(); + } + @AfterAll public void stop() { clearTableAndSchema(); 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 00000000000..933ca241734 --- /dev/null +++ b/catalogs/catalog-lakehouse-paimon/src/test/java/org/apache/gravitino/catalog/lakehouse/paimon/integration/test/CatalogPaimonS3IT.java @@ -0,0 +1,102 @@ +/* + * 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 static org.testcontainers.containers.localstack.LocalStackContainer.Service.S3; + +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.catalog.lakehouse.paimon.filesystem.s3.PaimonS3FileSystemConfig; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInstance; +import org.testcontainers.containers.Container; +import org.testcontainers.containers.localstack.LocalStackContainer; +import org.testcontainers.shaded.org.awaitility.Awaitility; +import org.testcontainers.utility.DockerImageName; + +@Tag("gravitino-docker-test") +@TestInstance(TestInstance.Lifecycle.PER_CLASS) +public class CatalogPaimonS3IT extends CatalogPaimonBaseIT { + + // private static final String S3_BUCKET_NAME = + // GravitinoITUtils.genRandomName("paimon-s3-bucket-"); + private static final String S3_BUCKET_NAME = "my-test-bucket"; + private static LocalStackContainer localStackContainer; + + @Override + protected Map initPaimonCatalogProperties() { + + Map catalogProperties = Maps.newHashMap(); + catalogProperties.put("key1", "val1"); + catalogProperties.put("key2", "val2"); + + TYPE = "filesystem"; + WAREHOUSE = "s3://" + S3_BUCKET_NAME + "/"; + + String accessKey = localStackContainer.getAccessKey(); + String secretKey = localStackContainer.getSecretKey(); + String endpoint = localStackContainer.getEndpointOverride(S3).toString(); + + catalogProperties.put(PaimonCatalogPropertiesMetadata.GRAVITINO_CATALOG_BACKEND, TYPE); + catalogProperties.put(PaimonCatalogPropertiesMetadata.WAREHOUSE, WAREHOUSE); + catalogProperties.put(PaimonS3FileSystemConfig.S3_ACCESS_KEY, accessKey); + catalogProperties.put(PaimonS3FileSystemConfig.S3_SECRET_KEY, secretKey); + catalogProperties.put(PaimonS3FileSystemConfig.S3_ENDPOINT, endpoint); + + return catalogProperties; + } + + @Override + protected void startNecessaryContainers() { + localStackContainer = + new LocalStackContainer(DockerImageName.parse("localstack/localstack")).withServices(S3); + localStackContainer.start(); + + Awaitility.await() + .atMost(60, TimeUnit.SECONDS) + .pollInterval(1, TimeUnit.SECONDS) + .until( + () -> { + try { + Container.ExecResult result = + localStackContainer.execInContainer( + "awslocal", "s3", "mb", "s3://" + S3_BUCKET_NAME); + return result.getExitCode() == 0; + } catch (Exception e) { + return false; + } + }); + } + + @AfterAll + public void stop() { + super.stop(); + localStackContainer.stop(); + } + + @Test + void testOperationDataOfPaimonTable() { + // Something wrong to use spark to read data from paimon with s3 + } +} diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 4efb10eb220..1be30a02e50 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -97,6 +97,7 @@ mail = "1.4.1" rome = "1.0" jettison = "1.1" thrift = "0.12.0" +hadoop-s3 = "3.3.6" [libraries] protobuf-java = { group = "com.google.protobuf", name = "protobuf-java", version.ref = "protoc" } @@ -144,6 +145,7 @@ hadoop3-hdfs = { group = "org.apache.hadoop", name = "hadoop-hdfs", version.ref hadoop3-common = { group = "org.apache.hadoop", name = "hadoop-common", version.ref = "hadoop3"} hadoop3-client = { group = "org.apache.hadoop", name = "hadoop-client", version.ref = "hadoop3"} hadoop3-minicluster = { group = "org.apache.hadoop", name = "hadoop-minicluster", version.ref = "hadoop-minikdc"} +hadoop-aws = { group = "org.apache.hadoop", name = "hadoop-aws", version.ref = "hadoop-s3"} htrace-core4 = { group = "org.apache.htrace", name = "htrace-core4", version.ref = "htrace-core4" } airlift-json = { group = "io.airlift", name = "json", version.ref = "airlift-json"} airlift-resolver = { group = "io.airlift.resolver", name = "resolver", version.ref = "airlift-resolver"} @@ -166,6 +168,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" } @@ -177,6 +182,7 @@ testcontainers = { group = "org.testcontainers", name = "testcontainers", versio testcontainers-mysql = { group = "org.testcontainers", name = "mysql", version.ref = "testcontainers" } testcontainers-postgresql = { group = "org.testcontainers", name = "postgresql", version.ref = "testcontainers" } testcontainers-junit-jupiter = { group = "org.testcontainers", name = "junit-jupiter", version.ref = "testcontainers" } +testcontainers-localstack = { group = "org.testcontainers", name = "localstack", version.ref = "testcontainers" } trino-jdbc = { group = "io.trino", name = "trino-jdbc", version.ref = "trino" } jwt-api = { group = "io.jsonwebtoken", name = "jjwt-api", version.ref = "jwt"} jwt-impl = { group = "io.jsonwebtoken", name = "jjwt-impl", version.ref = "jwt"} From 69e8b2840600f633ce6af09f59ee88888b2c4d68 Mon Sep 17 00:00:00 2001 From: yuqi Date: Fri, 13 Sep 2024 19:31:45 +0800 Subject: [PATCH 02/15] Fix --- .../paimon/filesystem/s3/PaimonS3FileSystemConfig.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/catalogs/catalog-lakehouse-paimon/src/main/java/org/apache/gravitino/catalog/lakehouse/paimon/filesystem/s3/PaimonS3FileSystemConfig.java b/catalogs/catalog-lakehouse-paimon/src/main/java/org/apache/gravitino/catalog/lakehouse/paimon/filesystem/s3/PaimonS3FileSystemConfig.java index 0b47a703908..0190d6730be 100644 --- a/catalogs/catalog-lakehouse-paimon/src/main/java/org/apache/gravitino/catalog/lakehouse/paimon/filesystem/s3/PaimonS3FileSystemConfig.java +++ b/catalogs/catalog-lakehouse-paimon/src/main/java/org/apache/gravitino/catalog/lakehouse/paimon/filesystem/s3/PaimonS3FileSystemConfig.java @@ -81,7 +81,7 @@ public String getS3SecretKey() { PropertyEntry.stringOptionalPropertyEntry( S3_ENDPOINT, "The endpoint of the AWS s3", - true /* immutable */, + false/* immutable */, null /* defaultValue */, false /* hidden */)) .put( @@ -89,7 +89,7 @@ public String getS3SecretKey() { PropertyEntry.stringOptionalPropertyEntry( S3_ACCESS_KEY, "The access key of the AWS s3", - true /* immutable */, + false /* immutable */, null /* defaultValue */, false /* hidden */)) .put( @@ -97,7 +97,7 @@ public String getS3SecretKey() { PropertyEntry.stringOptionalPropertyEntry( S3_SECRET_KEY, "The secret key of the AWS s3", - true /* immutable */, + false /* immutable */, null /* defaultValue */, false /* hidden */)) .build(); From 1ff547af433408a14649ac4f1085c41476bb2260 Mon Sep 17 00:00:00 2001 From: yuqi Date: Fri, 13 Sep 2024 19:55:53 +0800 Subject: [PATCH 03/15] Fix --- .../catalog-lakehouse-paimon/build.gradle.kts | 4 +-- .../s3/PaimonS3FileSystemConfig.java | 2 +- .../integration/test/CatalogPaimonBaseIT.java | 4 +-- .../integration/test/CatalogPaimonS3IT.java | 30 ++++++++++++++----- gradle/libs.versions.toml | 2 -- 5 files changed, 28 insertions(+), 14 deletions(-) diff --git a/catalogs/catalog-lakehouse-paimon/build.gradle.kts b/catalogs/catalog-lakehouse-paimon/build.gradle.kts index 0d6c75907fa..d1846361db7 100644 --- a/catalogs/catalog-lakehouse-paimon/build.gradle.kts +++ b/catalogs/catalog-lakehouse-paimon/build.gradle.kts @@ -81,11 +81,11 @@ dependencies { testImplementation(libs.postgresql.driver) testImplementation(libs.bundles.log4j) testImplementation(libs.junit.jupiter.params) - testImplementation(libs.testcontainers) testImplementation(libs.paimon.s3) testImplementation(libs.paimon.spark) - testImplementation(libs.hadoop.aws) + testImplementation(libs.testcontainers) testImplementation(libs.testcontainers.localstack) +// testImplementation(libs.hadoop.aws) testRuntimeOnly(libs.junit.jupiter.engine) } diff --git a/catalogs/catalog-lakehouse-paimon/src/main/java/org/apache/gravitino/catalog/lakehouse/paimon/filesystem/s3/PaimonS3FileSystemConfig.java b/catalogs/catalog-lakehouse-paimon/src/main/java/org/apache/gravitino/catalog/lakehouse/paimon/filesystem/s3/PaimonS3FileSystemConfig.java index 0190d6730be..1f6c61f69f1 100644 --- a/catalogs/catalog-lakehouse-paimon/src/main/java/org/apache/gravitino/catalog/lakehouse/paimon/filesystem/s3/PaimonS3FileSystemConfig.java +++ b/catalogs/catalog-lakehouse-paimon/src/main/java/org/apache/gravitino/catalog/lakehouse/paimon/filesystem/s3/PaimonS3FileSystemConfig.java @@ -81,7 +81,7 @@ public String getS3SecretKey() { PropertyEntry.stringOptionalPropertyEntry( S3_ENDPOINT, "The endpoint of the AWS s3", - false/* immutable */, + false /* immutable */, null /* defaultValue */, false /* hidden */)) .put( 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 07a52735ee0..cb0a2223d8d 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,7 +109,7 @@ 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 catalogProperties; @BeforeAll @@ -930,7 +930,7 @@ private Map createProperties() { 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 index 933ca241734..d898edd510c 100644 --- 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 @@ -26,9 +26,9 @@ import java.util.concurrent.TimeUnit; import org.apache.gravitino.catalog.lakehouse.paimon.PaimonCatalogPropertiesMetadata; import org.apache.gravitino.catalog.lakehouse.paimon.filesystem.s3.PaimonS3FileSystemConfig; +import org.apache.spark.sql.SparkSession; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.Tag; -import org.junit.jupiter.api.Test; import org.junit.jupiter.api.TestInstance; import org.testcontainers.containers.Container; import org.testcontainers.containers.localstack.LocalStackContainer; @@ -43,6 +43,9 @@ public class CatalogPaimonS3IT extends CatalogPaimonBaseIT { // GravitinoITUtils.genRandomName("paimon-s3-bucket-"); private static final String S3_BUCKET_NAME = "my-test-bucket"; private static LocalStackContainer localStackContainer; + private String accessKey; + private String secretKey; + private String endpoint; @Override protected Map initPaimonCatalogProperties() { @@ -54,9 +57,9 @@ protected Map initPaimonCatalogProperties() { TYPE = "filesystem"; WAREHOUSE = "s3://" + S3_BUCKET_NAME + "/"; - String accessKey = localStackContainer.getAccessKey(); - String secretKey = localStackContainer.getSecretKey(); - String endpoint = localStackContainer.getEndpointOverride(S3).toString(); + accessKey = localStackContainer.getAccessKey(); + secretKey = localStackContainer.getSecretKey(); + endpoint = localStackContainer.getEndpointOverride(S3).toString(); catalogProperties.put(PaimonCatalogPropertiesMetadata.GRAVITINO_CATALOG_BACKEND, TYPE); catalogProperties.put(PaimonCatalogPropertiesMetadata.WAREHOUSE, WAREHOUSE); @@ -95,8 +98,21 @@ public void stop() { localStackContainer.stop(); } - @Test - void testOperationDataOfPaimonTable() { - // Something wrong to use spark to read data from paimon with s3 + 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/gradle/libs.versions.toml b/gradle/libs.versions.toml index 1be30a02e50..0fbe92e8328 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -97,7 +97,6 @@ mail = "1.4.1" rome = "1.0" jettison = "1.1" thrift = "0.12.0" -hadoop-s3 = "3.3.6" [libraries] protobuf-java = { group = "com.google.protobuf", name = "protobuf-java", version.ref = "protoc" } @@ -145,7 +144,6 @@ hadoop3-hdfs = { group = "org.apache.hadoop", name = "hadoop-hdfs", version.ref hadoop3-common = { group = "org.apache.hadoop", name = "hadoop-common", version.ref = "hadoop3"} hadoop3-client = { group = "org.apache.hadoop", name = "hadoop-client", version.ref = "hadoop3"} hadoop3-minicluster = { group = "org.apache.hadoop", name = "hadoop-minicluster", version.ref = "hadoop-minikdc"} -hadoop-aws = { group = "org.apache.hadoop", name = "hadoop-aws", version.ref = "hadoop-s3"} htrace-core4 = { group = "org.apache.htrace", name = "htrace-core4", version.ref = "htrace-core4" } airlift-json = { group = "io.airlift", name = "json", version.ref = "airlift-json"} airlift-resolver = { group = "io.airlift.resolver", name = "resolver", version.ref = "airlift-resolver"} From f9bf3c1189f4e8c18c77b2139cc1064d337cd695 Mon Sep 17 00:00:00 2001 From: yuqi Date: Fri, 13 Sep 2024 20:05:22 +0800 Subject: [PATCH 04/15] Add some document. --- docs/lakehouse-paimon-catalog.md | 23 +++++++++++++---------- 1 file changed, 13 insertions(+), 10 deletions(-) diff --git a/docs/lakehouse-paimon-catalog.md b/docs/lakehouse-paimon-catalog.md index 6eabd3e8fcd..0cf3942684c 100644 --- a/docs/lakehouse-paimon-catalog.md +++ b/docs/lakehouse-paimon-catalog.md @@ -30,16 +30,19 @@ Builds with Apache Paimon `0.8.0`. ### Catalog properties -| Property name | Description | Default value | Required | Since Version | -|----------------------------------------------------|-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|------------------------|-----------------------------------------------------------------|---------------| -| `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 | -| `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 | +| Property name | Description | Default value | Required | Since Version | +|-----------------------------------------------------|-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|------------------------|-----------------------------------------------------------------|---------------| +| `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, `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` | The access key of the AWS s3. | (none) | required if the value of `warehouse` is a S3 path | 0.7.0 | +| `s3.secret-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. From fe2cc8fc730bb8bd29bcfd27c38c4ea683a01278 Mon Sep 17 00:00:00 2001 From: yuqi Date: Fri, 13 Sep 2024 20:10:38 +0800 Subject: [PATCH 05/15] fix --- catalogs/catalog-lakehouse-paimon/build.gradle.kts | 1 - 1 file changed, 1 deletion(-) diff --git a/catalogs/catalog-lakehouse-paimon/build.gradle.kts b/catalogs/catalog-lakehouse-paimon/build.gradle.kts index d1846361db7..c0fe7bb663a 100644 --- a/catalogs/catalog-lakehouse-paimon/build.gradle.kts +++ b/catalogs/catalog-lakehouse-paimon/build.gradle.kts @@ -85,7 +85,6 @@ dependencies { testImplementation(libs.paimon.spark) testImplementation(libs.testcontainers) testImplementation(libs.testcontainers.localstack) -// testImplementation(libs.hadoop.aws) testRuntimeOnly(libs.junit.jupiter.engine) } From 48149dd55f46f41a766427a0940c6e48923041f3 Mon Sep 17 00:00:00 2001 From: yuqi Date: Fri, 13 Sep 2024 20:13:47 +0800 Subject: [PATCH 06/15] fix test error. --- .../catalog/lakehouse/paimon/filesystem/FileSystemType.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/catalogs/catalog-lakehouse-paimon/src/main/java/org/apache/gravitino/catalog/lakehouse/paimon/filesystem/FileSystemType.java b/catalogs/catalog-lakehouse-paimon/src/main/java/org/apache/gravitino/catalog/lakehouse/paimon/filesystem/FileSystemType.java index eb531a31dd0..2f53fe729a5 100644 --- a/catalogs/catalog-lakehouse-paimon/src/main/java/org/apache/gravitino/catalog/lakehouse/paimon/filesystem/FileSystemType.java +++ b/catalogs/catalog-lakehouse-paimon/src/main/java/org/apache/gravitino/catalog/lakehouse/paimon/filesystem/FileSystemType.java @@ -19,6 +19,7 @@ package org.apache.gravitino.catalog.lakehouse.paimon.filesystem; public enum FileSystemType { + LOCAL_FILE, HDFS, S3, OSS; @@ -40,6 +41,8 @@ public static FileSystemType fromStoragePath(String storagePath) { return OSS; } else if (storagePath.startsWith("hdfs://")) { return HDFS; + } else if (storagePath.startsWith("/")) { + return LOCAL_FILE; } throw new IllegalArgumentException("Unsupported storage path: " + storagePath); From 6b040e7127f6a235aeda354f055ced576b9652da Mon Sep 17 00:00:00 2001 From: yuqi Date: Fri, 13 Sep 2024 20:31:47 +0800 Subject: [PATCH 07/15] FIX --- .../gravitino/catalog/lakehouse/paimon/utils/CatalogUtils.java | 1 + 1 file changed, 1 insertion(+) 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 87f28a4f6af..a7ddf6803ad 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 @@ -136,6 +136,7 @@ public static void checkWarehouseConfig( break; case HDFS: case OSS: + case LOCAL_FILE: break; default: throw new IllegalArgumentException("Unsupported file system type: " + fileSystemType); From 7fe77361617b1fc874df78f34aa2af28d806a13c Mon Sep 17 00:00:00 2001 From: yuqi Date: Fri, 13 Sep 2024 20:44:56 +0800 Subject: [PATCH 08/15] Support oss storage for paimon catalog. --- .../PaimonCatalogPropertiesMetadata.java | 12 ++ .../oss/PaimonOSSFileSystemConfig.java | 104 ++++++++++++++++++ .../lakehouse/paimon/utils/CatalogUtils.java | 18 ++- docs/lakehouse-paimon-catalog.md | 29 ++--- 4 files changed, 149 insertions(+), 14 deletions(-) create mode 100644 catalogs/catalog-lakehouse-paimon/src/main/java/org/apache/gravitino/catalog/lakehouse/paimon/filesystem/oss/PaimonOSSFileSystemConfig.java 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 5d2ba9e753f..9ba2eac49b0 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,6 +31,7 @@ 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.filesystem.oss.PaimonOSSFileSystemConfig; import org.apache.gravitino.catalog.lakehouse.paimon.filesystem.s3.PaimonS3FileSystemConfig; import org.apache.gravitino.connector.BaseCatalogPropertiesMetadata; import org.apache.gravitino.connector.PropertiesMetadata; @@ -68,6 +69,12 @@ public class PaimonCatalogPropertiesMetadata extends BaseCatalogPropertiesMetada PaimonS3FileSystemConfig.S3_SECRET_KEY, PaimonS3FileSystemConfig.S3_SECRET_KEY, PaimonS3FileSystemConfig.S3_ENDPOINT, PaimonS3FileSystemConfig.S3_ENDPOINT); + private static final Map OSS_CONFIGURATION = + ImmutableMap.of( + PaimonOSSFileSystemConfig.OSS_ACCESS_KEY, PaimonOSSFileSystemConfig.OSS_ACCESS_KEY, + PaimonOSSFileSystemConfig.OSS_SECRET_KEY, PaimonOSSFileSystemConfig.OSS_SECRET_KEY, + PaimonOSSFileSystemConfig.OSS_ENDPOINT, PaimonOSSFileSystemConfig.OSS_ENDPOINT); + static { List> propertyEntries = ImmutableList.of( @@ -96,6 +103,7 @@ public class PaimonCatalogPropertiesMetadata extends BaseCatalogPropertiesMetada result.putAll(KerberosConfig.KERBEROS_PROPERTY_ENTRIES); result.putAll(AuthenticationConfig.AUTHENTICATION_PROPERTY_ENTRIES); result.putAll(PaimonS3FileSystemConfig.S3_FILESYSTEM_PROPERTY_ENTRIES); + result.putAll(PaimonOSSFileSystemConfig.OSS_FILESYSTEM_PROPERTY_ENTRIES); PROPERTIES_METADATA = ImmutableMap.copyOf(result); } @@ -119,6 +127,10 @@ protected Map transformProperties(Map properties if (S3_CONFIGURATION.containsKey(key)) { gravitinoConfig.put(S3_CONFIGURATION.get(key), value); } + + if (OSS_CONFIGURATION.containsKey(key)) { + gravitinoConfig.put(OSS_CONFIGURATION.get(key), value); + } }); return gravitinoConfig; } diff --git a/catalogs/catalog-lakehouse-paimon/src/main/java/org/apache/gravitino/catalog/lakehouse/paimon/filesystem/oss/PaimonOSSFileSystemConfig.java b/catalogs/catalog-lakehouse-paimon/src/main/java/org/apache/gravitino/catalog/lakehouse/paimon/filesystem/oss/PaimonOSSFileSystemConfig.java new file mode 100644 index 00000000000..60bc502c4f6 --- /dev/null +++ b/catalogs/catalog-lakehouse-paimon/src/main/java/org/apache/gravitino/catalog/lakehouse/paimon/filesystem/oss/PaimonOSSFileSystemConfig.java @@ -0,0 +1,104 @@ +/* + * 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.filesystem.oss; + +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 PaimonOSSFileSystemConfig extends Config { + // OSS related properties + public static final String OSS_ENDPOINT = "fs.oss.endpoint"; + public static final String OSS_ACCESS_KEY = "fs.oss.accessKeyId"; + public static final String OSS_SECRET_KEY = "fs.oss.accessKeySecret"; + + public PaimonOSSFileSystemConfig(Map properties) { + super(false); + loadFromMap(properties, k -> true); + } + + public static final ConfigEntry PAIMON_OSS_ENDPOINT_ENTRY = + new ConfigBuilder(OSS_ENDPOINT) + .doc("The endpoint of the Aliyun oss") + .version(ConfigConstants.VERSION_0_7_0) + .stringConf() + .checkValue(StringUtils::isNotBlank, ConfigConstants.NOT_BLANK_ERROR_MSG) + .create(); + + public static final ConfigEntry PAIMON_OSS_ACCESS_KEY_ENTRY = + new ConfigBuilder(OSS_ACCESS_KEY) + .doc("The access key of the Aliyun oss") + .version(ConfigConstants.VERSION_0_7_0) + .stringConf() + .checkValue(StringUtils::isNotBlank, ConfigConstants.NOT_BLANK_ERROR_MSG) + .create(); + + public static final ConfigEntry PAIMON_OSS_SECRET_KEY_ENTRY = + new ConfigBuilder(OSS_SECRET_KEY) + .doc("The secret key of the Aliyun oss") + .version(ConfigConstants.VERSION_0_7_0) + .stringConf() + .checkValue(StringUtils::isNotBlank, ConfigConstants.NOT_BLANK_ERROR_MSG) + .create(); + + public String getOSSEndpoint() { + return get(PAIMON_OSS_ENDPOINT_ENTRY); + } + + public String getOSSAccessKey() { + return get(PAIMON_OSS_ACCESS_KEY_ENTRY); + } + + public String getOSSSecretKey() { + return get(PAIMON_OSS_SECRET_KEY_ENTRY); + } + + public static final Map> OSS_FILESYSTEM_PROPERTY_ENTRIES = + new ImmutableMap.Builder>() + .put( + OSS_ENDPOINT, + PropertyEntry.stringOptionalPropertyEntry( + OSS_ENDPOINT, + "The endpoint of the Aliyun oss", + false /* immutable */, + null /* defaultValue */, + false /* hidden */)) + .put( + OSS_ACCESS_KEY, + PropertyEntry.stringOptionalPropertyEntry( + OSS_ACCESS_KEY, + "The access key of the Aliyun oss", + false /* immutable */, + null /* defaultValue */, + false /* hidden */)) + .put( + OSS_SECRET_KEY, + PropertyEntry.stringOptionalPropertyEntry( + OSS_SECRET_KEY, + "The secret key of the Aliyun oss", + 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 a7ddf6803ad..7d22cfcf14f 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 @@ -34,6 +34,7 @@ import org.apache.gravitino.catalog.lakehouse.paimon.authentication.AuthenticationConfig; import org.apache.gravitino.catalog.lakehouse.paimon.authentication.kerberos.KerberosClient; import org.apache.gravitino.catalog.lakehouse.paimon.filesystem.FileSystemType; +import org.apache.gravitino.catalog.lakehouse.paimon.filesystem.oss.PaimonOSSFileSystemConfig; import org.apache.gravitino.catalog.lakehouse.paimon.filesystem.s3.PaimonS3FileSystemConfig; import org.apache.gravitino.catalog.lakehouse.paimon.ops.PaimonBackendCatalogWrapper; import org.apache.hadoop.conf.Configuration; @@ -134,9 +135,11 @@ public static void checkWarehouseConfig( case S3: checkS3FileSystemConfig(resultConf); break; - case HDFS: case OSS: + checkOSSFileSystemConfig(resultConf); + break; case LOCAL_FILE: + case HDFS: break; default: throw new IllegalArgumentException("Unsupported file system type: " + fileSystemType); @@ -155,4 +158,17 @@ private static void checkS3FileSystemConfig(Map resultConf) { StringUtils.isNotBlank(s3FileSystemConfig.getS3Endpoint()), "S3 endpoint can not be null or empty."); } + + private static void checkOSSFileSystemConfig(Map resultConf) { + PaimonOSSFileSystemConfig ossFileSystemConfig = new PaimonOSSFileSystemConfig(resultConf); + Preconditions.checkArgument( + StringUtils.isNotBlank(ossFileSystemConfig.getOSSAccessKey()), + "OSS access key can not be null or empty."); + Preconditions.checkArgument( + StringUtils.isNotBlank(ossFileSystemConfig.getOSSSecretKey()), + "OSS secret key can not be null or empty."); + Preconditions.checkArgument( + StringUtils.isNotBlank(ossFileSystemConfig.getOSSEndpoint()), + "OSS endpoint can not be null or empty."); + } } diff --git a/docs/lakehouse-paimon-catalog.md b/docs/lakehouse-paimon-catalog.md index 0cf3942684c..230c84810ee 100644 --- a/docs/lakehouse-paimon-catalog.md +++ b/docs/lakehouse-paimon-catalog.md @@ -30,19 +30,22 @@ Builds with Apache Paimon `0.8.0`. ### Catalog properties -| Property name | Description | Default value | Required | Since Version | -|-----------------------------------------------------|-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|------------------------|-----------------------------------------------------------------|---------------| -| `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, `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` | The access key of the AWS s3. | (none) | required if the value of `warehouse` is a S3 path | 0.7.0 | -| `s3.secret-key` | The secret key of the AWS s3. | (none) | required if the value of `warehouse` is a S3 path | 0.7.0 | +| Property name | Description | Default value | Required | Since Version | +|----------------------------------------------------|-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|------------------------|-----------------------------------------------------------------|---------------| +| `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, `hdfs://namespace/hdfs/path` for HDFS , `s3://{bucket-name}/path/` for S3 or `oss://{bucket-name}/path` for Aliyun OSS | (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` | The access key of the AWS s3. | (none) | required if the value of `warehouse` is a S3 path | 0.7.0 | +| `s3.secret-key` | The secret key of the AWS s3. | (none) | required if the value of `warehouse` is a S3 path | 0.7.0 | +| `fs.oss.endpoint` | The endpoint of the Aliyun oss. | (none) | required if the value of `warehouse` is a oss path | 0.7.0 | +| `fs.oss.accessKeyId` | The endpoint of the Aliyun oss. | (none) | required if the value of `warehouse` is a oss path | 0.7.0 | +| `fs.oss.accessKeySecret` | The endpoint of the Aliyun s3. | (none) | required if the value of `warehouse` is a oss 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. From d4be833b67d0416090185f38750d204529554e82 Mon Sep 17 00:00:00 2001 From: yuqi Date: Thu, 10 Oct 2024 11:45:45 +0800 Subject: [PATCH 09/15] Fix --- .../catalog/lakehouse/paimon/PaimonCatalogOperations.java | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/catalogs/catalog-lakehouse-paimon/src/main/java/org/apache/gravitino/catalog/lakehouse/paimon/PaimonCatalogOperations.java b/catalogs/catalog-lakehouse-paimon/src/main/java/org/apache/gravitino/catalog/lakehouse/paimon/PaimonCatalogOperations.java index 3188cd62cbe..fbe6f4be49d 100644 --- a/catalogs/catalog-lakehouse-paimon/src/main/java/org/apache/gravitino/catalog/lakehouse/paimon/PaimonCatalogOperations.java +++ b/catalogs/catalog-lakehouse-paimon/src/main/java/org/apache/gravitino/catalog/lakehouse/paimon/PaimonCatalogOperations.java @@ -39,7 +39,6 @@ import org.apache.gravitino.Namespace; import org.apache.gravitino.SchemaChange; import org.apache.gravitino.catalog.lakehouse.paimon.ops.PaimonCatalogOps; -import org.apache.gravitino.catalog.lakehouse.paimon.utils.CatalogUtils; import org.apache.gravitino.catalog.lakehouse.paimon.utils.TableOpsUtils; import org.apache.gravitino.connector.CatalogInfo; import org.apache.gravitino.connector.CatalogOperations; @@ -115,9 +114,7 @@ public void initialize( Map resultConf = Maps.newHashMap(prefixMap); resultConf.putAll(gravitinoConfig); - PaimonConfig paimonConfig = new PaimonConfig(resultConf); - CatalogUtils.checkWarehouseConfig(paimonConfig, resultConf); - this.paimonCatalogOps = new PaimonCatalogOps(paimonConfig); + this.paimonCatalogOps = new PaimonCatalogOps(new PaimonConfig(resultConf)); } /** From 2da59b61a878e79034749fb24e706c1f195dbba4 Mon Sep 17 00:00:00 2001 From: yuqi Date: Thu, 10 Oct 2024 11:51:20 +0800 Subject: [PATCH 10/15] Fix --- .../PaimonCatalogPropertiesMetadata.java | 4 +- .../paimon/filesystem/FileSystemType.java | 50 ------------------- .../PaimonOSSFileSystemConfig.java | 2 +- .../PaimonS3FileSystemConfig.java | 2 +- .../integration/test/CatalogPaimonS3IT.java | 2 +- 5 files changed, 5 insertions(+), 55 deletions(-) delete mode 100644 catalogs/catalog-lakehouse-paimon/src/main/java/org/apache/gravitino/catalog/lakehouse/paimon/filesystem/FileSystemType.java rename catalogs/catalog-lakehouse-paimon/src/main/java/org/apache/gravitino/catalog/lakehouse/paimon/{filesystem/oss => storage}/PaimonOSSFileSystemConfig.java (98%) rename catalogs/catalog-lakehouse-paimon/src/main/java/org/apache/gravitino/catalog/lakehouse/paimon/{filesystem/s3 => storage}/PaimonS3FileSystemConfig.java (98%) 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 4061c7f8e27..bf7bddfca47 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,8 +31,8 @@ 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.filesystem.oss.PaimonOSSFileSystemConfig; -import org.apache.gravitino.catalog.lakehouse.paimon.filesystem.s3.PaimonS3FileSystemConfig; +import org.apache.gravitino.catalog.lakehouse.paimon.storage.PaimonOSSFileSystemConfig; +import org.apache.gravitino.catalog.lakehouse.paimon.storage.PaimonS3FileSystemConfig; import org.apache.gravitino.connector.BaseCatalogPropertiesMetadata; import org.apache.gravitino.connector.PropertiesMetadata; import org.apache.gravitino.connector.PropertyEntry; diff --git a/catalogs/catalog-lakehouse-paimon/src/main/java/org/apache/gravitino/catalog/lakehouse/paimon/filesystem/FileSystemType.java b/catalogs/catalog-lakehouse-paimon/src/main/java/org/apache/gravitino/catalog/lakehouse/paimon/filesystem/FileSystemType.java deleted file mode 100644 index 2f53fe729a5..00000000000 --- a/catalogs/catalog-lakehouse-paimon/src/main/java/org/apache/gravitino/catalog/lakehouse/paimon/filesystem/FileSystemType.java +++ /dev/null @@ -1,50 +0,0 @@ -/* - * 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.filesystem; - -public enum FileSystemType { - LOCAL_FILE, - HDFS, - S3, - OSS; - - public static FileSystemType fromString(String type) { - for (FileSystemType fileSystemType : FileSystemType.values()) { - if (fileSystemType.name().equalsIgnoreCase(type)) { - return fileSystemType; - } - } - - throw new IllegalArgumentException("Unsupported file system type: " + type); - } - - public static FileSystemType fromStoragePath(String storagePath) { - if (storagePath.startsWith("s3://")) { - return S3; - } else if (storagePath.startsWith("oss://")) { - return OSS; - } else if (storagePath.startsWith("hdfs://")) { - return HDFS; - } else if (storagePath.startsWith("/")) { - return LOCAL_FILE; - } - - throw new IllegalArgumentException("Unsupported storage path: " + storagePath); - } -} diff --git a/catalogs/catalog-lakehouse-paimon/src/main/java/org/apache/gravitino/catalog/lakehouse/paimon/filesystem/oss/PaimonOSSFileSystemConfig.java b/catalogs/catalog-lakehouse-paimon/src/main/java/org/apache/gravitino/catalog/lakehouse/paimon/storage/PaimonOSSFileSystemConfig.java similarity index 98% rename from catalogs/catalog-lakehouse-paimon/src/main/java/org/apache/gravitino/catalog/lakehouse/paimon/filesystem/oss/PaimonOSSFileSystemConfig.java rename to catalogs/catalog-lakehouse-paimon/src/main/java/org/apache/gravitino/catalog/lakehouse/paimon/storage/PaimonOSSFileSystemConfig.java index 60bc502c4f6..ad7fa26f3bc 100644 --- a/catalogs/catalog-lakehouse-paimon/src/main/java/org/apache/gravitino/catalog/lakehouse/paimon/filesystem/oss/PaimonOSSFileSystemConfig.java +++ b/catalogs/catalog-lakehouse-paimon/src/main/java/org/apache/gravitino/catalog/lakehouse/paimon/storage/PaimonOSSFileSystemConfig.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.gravitino.catalog.lakehouse.paimon.filesystem.oss; +package org.apache.gravitino.catalog.lakehouse.paimon.storage; import com.google.common.collect.ImmutableMap; import java.util.Map; diff --git a/catalogs/catalog-lakehouse-paimon/src/main/java/org/apache/gravitino/catalog/lakehouse/paimon/filesystem/s3/PaimonS3FileSystemConfig.java b/catalogs/catalog-lakehouse-paimon/src/main/java/org/apache/gravitino/catalog/lakehouse/paimon/storage/PaimonS3FileSystemConfig.java similarity index 98% rename from catalogs/catalog-lakehouse-paimon/src/main/java/org/apache/gravitino/catalog/lakehouse/paimon/filesystem/s3/PaimonS3FileSystemConfig.java rename to catalogs/catalog-lakehouse-paimon/src/main/java/org/apache/gravitino/catalog/lakehouse/paimon/storage/PaimonS3FileSystemConfig.java index 1f6c61f69f1..4184fcc06f1 100644 --- a/catalogs/catalog-lakehouse-paimon/src/main/java/org/apache/gravitino/catalog/lakehouse/paimon/filesystem/s3/PaimonS3FileSystemConfig.java +++ b/catalogs/catalog-lakehouse-paimon/src/main/java/org/apache/gravitino/catalog/lakehouse/paimon/storage/PaimonS3FileSystemConfig.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.gravitino.catalog.lakehouse.paimon.filesystem.s3; +package org.apache.gravitino.catalog.lakehouse.paimon.storage; import com.google.common.collect.ImmutableMap; import java.util.Map; 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 index 8de2eed9624..2030bbd19df 100644 --- 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 @@ -99,7 +99,7 @@ protected void startNecessaryContainers() { try { Container.ExecResult result = localStackContainer.executeInContainer( - "awslocal", "s3", "ls", "s3://" + S3_BUCKET_NAME); + "awslocal", "s3", "mb", "s3://" + S3_BUCKET_NAME); return result.getExitCode() == 0; } catch (Exception e) { return false; From bb06805f31d27279ece5b2af7e2037075f3135ea Mon Sep 17 00:00:00 2001 From: yuqi Date: Thu, 10 Oct 2024 15:11:22 +0800 Subject: [PATCH 11/15] Fix --- .../catalog-lakehouse-paimon/build.gradle.kts | 1 + .../integration/test/CatalogPaimonOSSIT.java | 101 ++++++++++++++++++ docs/lakehouse-paimon-catalog.md | 4 + gradle/libs.versions.toml | 1 + 4 files changed, 107 insertions(+) create mode 100644 catalogs/catalog-lakehouse-paimon/src/test/java/org/apache/gravitino/catalog/lakehouse/paimon/integration/test/CatalogPaimonOSSIT.java diff --git a/catalogs/catalog-lakehouse-paimon/build.gradle.kts b/catalogs/catalog-lakehouse-paimon/build.gradle.kts index c81959ee7e8..8fee917458b 100644 --- a/catalogs/catalog-lakehouse-paimon/build.gradle.kts +++ b/catalogs/catalog-lakehouse-paimon/build.gradle.kts @@ -124,6 +124,7 @@ dependencies { testImplementation(libs.postgresql.driver) testImplementation(libs.bundles.log4j) testImplementation(libs.junit.jupiter.params) + testImplementation(libs.paimon.oss) testImplementation(libs.paimon.s3) testImplementation(libs.paimon.spark) testImplementation(libs.testcontainers) diff --git a/catalogs/catalog-lakehouse-paimon/src/test/java/org/apache/gravitino/catalog/lakehouse/paimon/integration/test/CatalogPaimonOSSIT.java b/catalogs/catalog-lakehouse-paimon/src/test/java/org/apache/gravitino/catalog/lakehouse/paimon/integration/test/CatalogPaimonOSSIT.java new file mode 100644 index 00000000000..d95a303a5f7 --- /dev/null +++ b/catalogs/catalog-lakehouse-paimon/src/test/java/org/apache/gravitino/catalog/lakehouse/paimon/integration/test/CatalogPaimonOSSIT.java @@ -0,0 +1,101 @@ +/* + * 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 org.apache.gravitino.catalog.lakehouse.paimon.PaimonCatalogPropertiesMetadata; +import org.apache.gravitino.integration.test.util.DownloaderUtils; +import org.apache.gravitino.integration.test.util.ITUtils; +import org.apache.gravitino.storage.OSSProperties; +import org.apache.spark.sql.SparkSession; +import org.junit.jupiter.api.Disabled; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.TestInstance; + +@Tag("gravitino-docker-test") +@TestInstance(TestInstance.Lifecycle.PER_CLASS) +@Disabled( + "You need to specify the real OSS bucket name, access key, secret key and endpoint to run this test") +public class CatalogPaimonOSSIT extends CatalogPaimonBaseIT { + + private static final String OSS_BUCKET_NAME = "YOUR_BUCKET"; + private static final String accessKey = "YOUR_ACCESS_KEY"; + private static final String secretKey = "YOUR_SECRET_KEY"; + private static final String endpoint = "OSS_ENDPOINT"; + private static final String warehouse = "oss://" + OSS_BUCKET_NAME + "/paimon-test"; + + private static final String PAIMON_OSS_JAR_URL = + "https://repo1.maven.org/maven2/org/apache/paimon/paimon-oss/0.8.0/paimon-oss-0.8.0.jar"; + + @Override + protected Map initPaimonCatalogProperties() { + + Map catalogProperties = Maps.newHashMap(); + catalogProperties.put("key1", "val1"); + catalogProperties.put("key2", "val2"); + + TYPE = "filesystem"; + + WAREHOUSE = warehouse; + catalogProperties.put(PaimonCatalogPropertiesMetadata.GRAVITINO_CATALOG_BACKEND, TYPE); + catalogProperties.put(PaimonCatalogPropertiesMetadata.WAREHOUSE, WAREHOUSE); + catalogProperties.put(OSSProperties.GRAVITINO_OSS_ACCESS_KEY_ID, accessKey); + catalogProperties.put(OSSProperties.GRAVITINO_OSS_ACCESS_KEY_SECRET, secretKey); + catalogProperties.put(OSSProperties.GRAVITINO_OSS_ENDPOINT, endpoint); + + // Need to download the S3 dependency in the deploy mode. + downloadOSSDependency(); + + return catalogProperties; + } + + private void downloadOSSDependency() { + 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_OSS_JAR_URL, serverPath, paimonCatalogPath); + } + } catch (Exception e) { + throw new RuntimeException("Failed to download the OSS dependency", e); + } + } + + 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.fs.oss.accessKeyId", accessKey) + .config("spark.sql.catalog.paimon.fs.oss.accessKeySecret", secretKey) + .config("spark.sql.catalog.paimon.fs.oss.endpoint", endpoint) + .enableHiveSupport() + .getOrCreate(); + } +} diff --git a/docs/lakehouse-paimon-catalog.md b/docs/lakehouse-paimon-catalog.md index b25bf35c113..e1bb97d60c2 100644 --- a/docs/lakehouse-paimon-catalog.md +++ b/docs/lakehouse-paimon-catalog.md @@ -47,6 +47,10 @@ Builds with Apache Paimon `0.8.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 | +:::note +If you want to use the `oss` or `s3` warehouse, you need to place related jars in the `catalogs/lakehouse-paimon/lib` directory, more information can be found in the [Paimon S3](https://paimon.apache.org/docs/master/filesystems/s3/). +::: + 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 5e1446a8fae..44a4737d210 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -174,6 +174,7 @@ paimon-core = { group = "org.apache.paimon", name = "paimon-core", version.ref = 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-oss = { group = "org.apache.paimon", name = "paimon-oss", 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" } From 6d5847c86929c39bffb5785157414f441412caa8 Mon Sep 17 00:00:00 2001 From: yuqi Date: Thu, 10 Oct 2024 15:37:13 +0800 Subject: [PATCH 12/15] Fix --- .../lakehouse/paimon/integration/test/CatalogPaimonOSSIT.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/catalogs/catalog-lakehouse-paimon/src/test/java/org/apache/gravitino/catalog/lakehouse/paimon/integration/test/CatalogPaimonOSSIT.java b/catalogs/catalog-lakehouse-paimon/src/test/java/org/apache/gravitino/catalog/lakehouse/paimon/integration/test/CatalogPaimonOSSIT.java index d95a303a5f7..15fb434d74a 100644 --- a/catalogs/catalog-lakehouse-paimon/src/test/java/org/apache/gravitino/catalog/lakehouse/paimon/integration/test/CatalogPaimonOSSIT.java +++ b/catalogs/catalog-lakehouse-paimon/src/test/java/org/apache/gravitino/catalog/lakehouse/paimon/integration/test/CatalogPaimonOSSIT.java @@ -61,7 +61,7 @@ protected Map initPaimonCatalogProperties() { catalogProperties.put(OSSProperties.GRAVITINO_OSS_ACCESS_KEY_SECRET, secretKey); catalogProperties.put(OSSProperties.GRAVITINO_OSS_ENDPOINT, endpoint); - // Need to download the S3 dependency in the deploy mode. + // Need to download the OSS dependency in the deploy mode. downloadOSSDependency(); return catalogProperties; From b87d35dcf4b60cb7458f14ed97ebc06699c884b2 Mon Sep 17 00:00:00 2001 From: yuqi Date: Thu, 10 Oct 2024 16:23:20 +0800 Subject: [PATCH 13/15] Fix --- .../PaimonCatalogPropertiesMetadata.java | 26 +++------------- .../lakehouse/paimon/utils/CatalogUtils.java | 30 ++++++++++++------- .../integration/test/CatalogPaimonBaseIT.java | 2 +- 3 files changed, 24 insertions(+), 34 deletions(-) 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 bf7bddfca47..0c6a87c0269 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 @@ -33,6 +33,7 @@ import org.apache.gravitino.catalog.lakehouse.paimon.authentication.kerberos.KerberosConfig; import org.apache.gravitino.catalog.lakehouse.paimon.storage.PaimonOSSFileSystemConfig; import org.apache.gravitino.catalog.lakehouse.paimon.storage.PaimonS3FileSystemConfig; +import org.apache.gravitino.catalog.lakehouse.paimon.utils.CatalogUtils; import org.apache.gravitino.connector.BaseCatalogPropertiesMetadata; import org.apache.gravitino.connector.PropertiesMetadata; import org.apache.gravitino.connector.PropertyEntry; @@ -57,7 +58,7 @@ public class PaimonCatalogPropertiesMetadata extends BaseCatalogPropertiesMetada public static final Map GRAVITINO_CONFIG_TO_PAIMON = ImmutableMap.of(GRAVITINO_CATALOG_BACKEND, PAIMON_METASTORE, WAREHOUSE, WAREHOUSE, URI, URI); private static final Map> PROPERTIES_METADATA; - private static final Map KERBEROS_CONFIGURATION = + public static final Map KERBEROS_CONFIGURATION = ImmutableMap.of( KerberosConfig.PRINCIPAL_KEY, KerberosConfig.PRINCIPAL_KEY, @@ -119,26 +120,7 @@ protected Map> specificPropertyEntries() { return PROPERTIES_METADATA; } - protected Map transformProperties(Map properties) { - Map gravitinoConfig = Maps.newHashMap(); - properties.forEach( - (key, value) -> { - if (GRAVITINO_CONFIG_TO_PAIMON.containsKey(key)) { - gravitinoConfig.put(GRAVITINO_CONFIG_TO_PAIMON.get(key), value); - } - - if (KERBEROS_CONFIGURATION.containsKey(key)) { - gravitinoConfig.put(KERBEROS_CONFIGURATION.get(key), value); - } - - if (S3_CONFIGURATION.containsKey(key)) { - gravitinoConfig.put(S3_CONFIGURATION.get(key), value); - } - - if (OSS_CONFIGURATION.containsKey(key)) { - gravitinoConfig.put(OSS_CONFIGURATION.get(key), value); - } - }); - return gravitinoConfig; + Map transformProperties(Map properties) { + return CatalogUtils.toInnerProperty(properties, false); } } 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 e9a71f12299..06777b8282f 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,8 @@ */ package org.apache.gravitino.catalog.lakehouse.paimon.utils; +import static org.apache.gravitino.catalog.lakehouse.paimon.PaimonCatalogPropertiesMetadata.GRAVITINO_CONFIG_TO_PAIMON; +import static org.apache.gravitino.catalog.lakehouse.paimon.PaimonCatalogPropertiesMetadata.KERBEROS_CONFIGURATION; import static org.apache.gravitino.catalog.lakehouse.paimon.PaimonCatalogPropertiesMetadata.OSS_CONFIGURATION; import static org.apache.gravitino.catalog.lakehouse.paimon.PaimonCatalogPropertiesMetadata.S3_CONFIGURATION; import static org.apache.gravitino.catalog.lakehouse.paimon.PaimonConfig.CATALOG_BACKEND; @@ -124,17 +126,23 @@ private static void checkPaimonConfig(PaimonConfig paimonConfig) { } } - public static Map toPaimonCatalogProperties( - Map gravitinoProperties) { - Map paimonProperties = new HashMap<>(); - - gravitinoProperties.forEach( - (k, v) -> { - if (S3_CONFIGURATION.containsKey(k)) { - paimonProperties.put(S3_CONFIGURATION.get(k), v); - } else paimonProperties.put(OSS_CONFIGURATION.getOrDefault(k, k), v); + public static Map toInnerProperty( + Map properties, boolean keepUnknown) { + Map gravitinoConfig = new HashMap<>(); + properties.forEach( + (key, value) -> { + if (GRAVITINO_CONFIG_TO_PAIMON.containsKey(key)) { + gravitinoConfig.put(GRAVITINO_CONFIG_TO_PAIMON.get(key), value); + } else if (KERBEROS_CONFIGURATION.containsKey(key)) { + gravitinoConfig.put(KERBEROS_CONFIGURATION.get(key), value); + } else if (S3_CONFIGURATION.containsKey(key)) { + gravitinoConfig.put(S3_CONFIGURATION.get(key), value); + } else if (OSS_CONFIGURATION.containsKey(key)) { + gravitinoConfig.put(OSS_CONFIGURATION.get(key), value); + } else if (keepUnknown) { + gravitinoConfig.put(key, value); + } }); - - return paimonProperties; + return gravitinoConfig; } } 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 bd907d06d92..b5ac224d973 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 @@ -889,7 +889,7 @@ private void createCatalog() { // Why needs this conversion? Because PaimonCatalogOperations#initialize will try to convert // Gravitino general S3 properties to Paimon specific S3 properties. - Map copy = CatalogUtils.toPaimonCatalogProperties(catalogProperties); + Map copy = CatalogUtils.toInnerProperty(catalogProperties, true); PaimonBackendCatalogWrapper paimonBackendCatalogWrapper = CatalogUtils.loadCatalogBackend(new PaimonConfig(copy)); From 444d870c23e3a26829908d7ecd36e6a3d3a47f4d Mon Sep 17 00:00:00 2001 From: yuqi Date: Fri, 11 Oct 2024 15:16:26 +0800 Subject: [PATCH 14/15] fix --- .../lakehouse/paimon/integration/test/CatalogPaimonOSSIT.java | 3 +-- .../lakehouse/paimon/integration/test/CatalogPaimonS3IT.java | 3 +-- 2 files changed, 2 insertions(+), 4 deletions(-) diff --git a/catalogs/catalog-lakehouse-paimon/src/test/java/org/apache/gravitino/catalog/lakehouse/paimon/integration/test/CatalogPaimonOSSIT.java b/catalogs/catalog-lakehouse-paimon/src/test/java/org/apache/gravitino/catalog/lakehouse/paimon/integration/test/CatalogPaimonOSSIT.java index 15fb434d74a..9c4ec2a5676 100644 --- a/catalogs/catalog-lakehouse-paimon/src/test/java/org/apache/gravitino/catalog/lakehouse/paimon/integration/test/CatalogPaimonOSSIT.java +++ b/catalogs/catalog-lakehouse-paimon/src/test/java/org/apache/gravitino/catalog/lakehouse/paimon/integration/test/CatalogPaimonOSSIT.java @@ -71,10 +71,9 @@ private void downloadOSSDependency() { 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_OSS_JAR_URL, serverPath, paimonCatalogPath); + DownloaderUtils.downloadFile(PAIMON_OSS_JAR_URL, paimonCatalogPath); } } catch (Exception e) { throw new RuntimeException("Failed to download the OSS dependency", e); 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 index 2030bbd19df..f3786b391ef 100644 --- 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 @@ -76,10 +76,9 @@ 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); + DownloaderUtils.downloadFile(PAIMON_S3_JAR_URL, paimonCatalogPath); } } catch (Exception e) { throw new RuntimeException("Failed to download the S3 dependency", e); From 50b0a9ea5eebac249a5d0b9ac2a5669b3f10b9f5 Mon Sep 17 00:00:00 2001 From: yuqi Date: Fri, 11 Oct 2024 19:42:30 +0800 Subject: [PATCH 15/15] x --- docs/iceberg-rest-service.md | 12 ++++++------ docs/lakehouse-paimon-catalog.md | 32 ++++++++++++++++---------------- 2 files changed, 22 insertions(+), 22 deletions(-) diff --git a/docs/iceberg-rest-service.md b/docs/iceberg-rest-service.md index 30fddd0b488..4217350dac9 100644 --- a/docs/iceberg-rest-service.md +++ b/docs/iceberg-rest-service.md @@ -88,12 +88,12 @@ The detailed configuration items are as follows: | Configuration item | Description | Default value | Required | Since Version | |---------------------------------------------------------------------------|-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|---------------|------------------------------------------------------------------------------------|------------------| -| `gravitino.iceberg-rest.authentication.type` | The type of authentication for Iceberg rest catalog backend. This configuration only applicable for for Hive backend, and only supports `Kerberos`, `simple` currently. As for JDBC backend, only username/password authentication was supported now. | `simple` | No | 0.6.0-incubating | -| `gravitino.iceberg-rest.authentication.impersonation-enable` | Whether to enable impersonation for the Iceberg catalog | `false` | No | 0.6.0-incubating | -| `gravitino.iceberg-rest.authentication.kerberos.principal` | The principal of the Kerberos authentication | (none) | required if the value of `gravitino.iceberg-rest.authentication.type` is Kerberos. | 0.6.0-incubating | -| `gravitino.iceberg-rest.authentication.kerberos.keytab-uri` | The URI of The keytab for the Kerberos authentication. | (none) | required if the value of `gravitino.iceberg-rest.authentication.type` is Kerberos. | 0.6.0-incubating | -| `gravitino.iceberg-rest.authentication.kerberos.check-interval-sec` | The check interval of Kerberos credential for Iceberg catalog. | 60 | No | 0.6.0-incubating | -| `gravitino.iceberg-rest.authentication.kerberos.keytab-fetch-timeout-sec` | The fetch timeout of retrieving Kerberos keytab from `authentication.kerberos.keytab-uri`. | 60 | No | 0.6.0-incubating | +| `gravitino.iceberg-rest.authentication.type` | The type of authentication for Iceberg rest catalog backend. This configuration only applicable for for Hive backend, and only supports `Kerberos`, `simple` currently. As for JDBC backend, only username/password authentication was supported now. | `simple` | No | 0.7.0-incubating | +| `gravitino.iceberg-rest.authentication.impersonation-enable` | Whether to enable impersonation for the Iceberg catalog | `false` | No | 0.7.0-incubating | +| `gravitino.iceberg-rest.authentication.kerberos.principal` | The principal of the Kerberos authentication | (none) | required if the value of `gravitino.iceberg-rest.authentication.type` is Kerberos. | 0.7.0-incubating | +| `gravitino.iceberg-rest.authentication.kerberos.keytab-uri` | The URI of The keytab for the Kerberos authentication. | (none) | required if the value of `gravitino.iceberg-rest.authentication.type` is Kerberos. | 0.7.0-incubating | +| `gravitino.iceberg-rest.authentication.kerberos.check-interval-sec` | The check interval of Kerberos credential for Iceberg catalog. | 60 | No | 0.7.0-incubating | +| `gravitino.iceberg-rest.authentication.kerberos.keytab-fetch-timeout-sec` | The fetch timeout of retrieving Kerberos keytab from `authentication.kerberos.keytab-uri`. | 60 | No | 0.7.0-incubating | ### Storage diff --git a/docs/lakehouse-paimon-catalog.md b/docs/lakehouse-paimon-catalog.md index ceb460368fd..b9c2772bc7a 100644 --- a/docs/lakehouse-paimon-catalog.md +++ b/docs/lakehouse-paimon-catalog.md @@ -30,22 +30,22 @@ Builds with Apache Paimon `0.8.0`. ### Catalog properties -| Property name | Description | Default value | Required | Since Version | -|----------------------------------------------------|-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|------------------------|-----------------------------------------------------------------|---------------| -| `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, `hdfs://namespace/hdfs/path` for HDFS , `s3://{bucket-name}/path/` for S3 or `oss://{bucket-name}/path` for Aliyun OSS | (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 | -| `oss-endpoint` | The endpoint of the Aliyun oss. | (none) | required if the value of `warehouse` is a oss path | 0.7.0 | -| `oss-access-key-id` | The access key of the Aliyun oss. | (none) | required if the value of `warehouse` is a oss path | 0.7.0 | -| `oss-accesss-key-secret` | The secret key the Aliyun s3. | (none) | required if the value of `warehouse` is a oss path | 0.7.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 | +| Property name | Description | Default value | Required | Since Version | +|----------------------------------------------------|-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|------------------------|-----------------------------------------------------------------|-------------------| +| `catalog-backend` | Catalog backend of Gravitino Paimon catalog. Only supports `filesystem` now. | (none) | Yes | 0.6.0-incubating | +| `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-incubating | +| `warehouse` | Warehouse directory of catalog. `file:///user/hive/warehouse-paimon/` for local fs, `hdfs://namespace/hdfs/path` for HDFS , `s3://{bucket-name}/path/` for S3 or `oss://{bucket-name}/path` for Aliyun OSS | (none) | Yes | 0.6.0-incubating | +| `authentication.type` | The type of authentication for Paimon catalog backend, currently Gravitino only supports `Kerberos` and `simple`. | `simple` | No | 0.6.0-incubating | +| `authentication.kerberos.principal` | The principal of the Kerberos authentication. | (none) | required if the value of `authentication.type` is Kerberos. | 0.6.0-incubating | +| `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-incubating | +| `authentication.kerberos.check-interval-sec` | The check interval of Kerberos credential for Paimon catalog. | 60 | No | 0.6.0-incubating | +| `authentication.kerberos.keytab-fetch-timeout-sec` | The fetch timeout of retrieving Kerberos keytab from `authentication.kerberos.keytab-uri`. | 60 | No | 0.6.0-incubating | +| `oss-endpoint` | The endpoint of the Aliyun oss. | (none) | required if the value of `warehouse` is a oss path | 0.7.0-incubating | +| `oss-access-key-id` | The access key of the Aliyun oss. | (none) | required if the value of `warehouse` is a oss path | 0.7.0-incubating | +| `oss-accesss-key-secret` | The secret key the Aliyun s3. | (none) | required if the value of `warehouse` is a oss path | 0.7.0-incubating | +| `s3-endpoint` | The endpoint of the AWS s3. | (none) | required if the value of `warehouse` is a S3 path | 0.7.0-incubating | +| `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-incubating | +| `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-incubating | :::note If you want to use the `oss` or `s3` warehouse, you need to place related jars in the `catalogs/lakehouse-paimon/lib` directory, more information can be found in the [Paimon S3](https://paimon.apache.org/docs/master/filesystems/s3/).