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 0d2435e25 [#4940] feat(lakehouse-paimon): Support OSS filesystem for Paimon catalog. (#4941) 0d2435e25 is described below commit 0d2435e2594384b0ba658548adff46af5026e762 Author: Qi Yu <y...@datastrato.com> AuthorDate: Fri Oct 11 20:46:22 2024 +0800 [#4940] feat(lakehouse-paimon): Support OSS filesystem for Paimon catalog. (#4941) ### What changes were proposed in this pull request? Support OSS filesystem for Paimon catalog. ### Why are the changes needed? For better user experience. Fix: #4940 ### Does this PR introduce _any_ user-facing change? N/A. ### How was this patch tested? Only test local as there is no simulator for OSS. --- catalogs/catalog-lakehouse-paimon/build.gradle.kts | 1 + .../paimon/PaimonCatalogPropertiesMetadata.java | 36 +++---- .../paimon/storage/PaimonOSSFileSystemConfig.java | 104 +++++++++++++++++++++ .../paimon/storage/PaimonS3FileSystemConfig.java | 104 +++++++++++++++++++++ .../lakehouse/paimon/utils/CatalogUtils.java | 27 ++++-- .../integration/test/CatalogPaimonBaseIT.java | 2 +- ...alogPaimonS3IT.java => CatalogPaimonOSSIT.java} | 75 +++++---------- .../paimon/integration/test/CatalogPaimonS3IT.java | 3 +- docs/iceberg-rest-service.md | 12 +-- docs/lakehouse-paimon-catalog.md | 33 ++++--- gradle/libs.versions.toml | 1 + 11 files changed, 298 insertions(+), 100 deletions(-) diff --git a/catalogs/catalog-lakehouse-paimon/build.gradle.kts b/catalogs/catalog-lakehouse-paimon/build.gradle.kts index c81959ee7..8fee91745 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/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 589d79672..0c6a87c02 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,10 +31,13 @@ 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.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; +import org.apache.gravitino.storage.OSSProperties; import org.apache.gravitino.storage.S3Properties; /** @@ -55,7 +58,7 @@ public class PaimonCatalogPropertiesMetadata extends BaseCatalogPropertiesMetada 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; - private static final Map<String, String> KERBEROS_CONFIGURATION = + public static final Map<String, String> KERBEROS_CONFIGURATION = ImmutableMap.of( KerberosConfig.PRINCIPAL_KEY, KerberosConfig.PRINCIPAL_KEY, @@ -74,6 +77,12 @@ public class PaimonCatalogPropertiesMetadata extends BaseCatalogPropertiesMetada S3Properties.GRAVITINO_S3_SECRET_ACCESS_KEY, S3_SECRET_KEY, S3Properties.GRAVITINO_S3_ENDPOINT, S3_ENDPOINT); + public static final Map<String, String> OSS_CONFIGURATION = + ImmutableMap.of( + OSSProperties.GRAVITINO_OSS_ACCESS_KEY_ID, PaimonOSSFileSystemConfig.OSS_ACCESS_KEY, + OSSProperties.GRAVITINO_OSS_ACCESS_KEY_SECRET, PaimonOSSFileSystemConfig.OSS_SECRET_KEY, + OSSProperties.GRAVITINO_OSS_ENDPOINT, PaimonOSSFileSystemConfig.OSS_ENDPOINT); + static { List<PropertyEntry<?>> propertyEntries = ImmutableList.of( @@ -101,8 +110,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); - + result.putAll(PaimonS3FileSystemConfig.S3_FILESYSTEM_PROPERTY_ENTRIES); + result.putAll(PaimonOSSFileSystemConfig.OSS_FILESYSTEM_PROPERTY_ENTRIES); PROPERTIES_METADATA = ImmutableMap.copyOf(result); } @@ -111,22 +120,7 @@ public class PaimonCatalogPropertiesMetadata extends BaseCatalogPropertiesMetada return PROPERTIES_METADATA; } - protected Map<String, String> transformProperties(Map<String, String> properties) { - Map<String, String> 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); - } - }); - return gravitinoConfig; + Map<String, String> transformProperties(Map<String, String> properties) { + return CatalogUtils.toInnerProperty(properties, false); } } diff --git a/catalogs/catalog-lakehouse-paimon/src/main/java/org/apache/gravitino/catalog/lakehouse/paimon/storage/PaimonOSSFileSystemConfig.java b/catalogs/catalog-lakehouse-paimon/src/main/java/org/apache/gravitino/catalog/lakehouse/paimon/storage/PaimonOSSFileSystemConfig.java new file mode 100644 index 000000000..ad7fa26f3 --- /dev/null +++ b/catalogs/catalog-lakehouse-paimon/src/main/java/org/apache/gravitino/catalog/lakehouse/paimon/storage/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.storage; + +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<String, String> properties) { + super(false); + loadFromMap(properties, k -> true); + } + + public static final ConfigEntry<String> 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<String> 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<String> 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<String, PropertyEntry<?>> OSS_FILESYSTEM_PROPERTY_ENTRIES = + new ImmutableMap.Builder<String, PropertyEntry<?>>() + .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/storage/PaimonS3FileSystemConfig.java b/catalogs/catalog-lakehouse-paimon/src/main/java/org/apache/gravitino/catalog/lakehouse/paimon/storage/PaimonS3FileSystemConfig.java new file mode 100644 index 000000000..4184fcc06 --- /dev/null +++ b/catalogs/catalog-lakehouse-paimon/src/main/java/org/apache/gravitino/catalog/lakehouse/paimon/storage/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.storage; + +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<String, String> properties) { + super(false); + loadFromMap(properties, k -> true); + } + + public static final ConfigEntry<String> 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<String> 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<String> 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<String, PropertyEntry<?>> S3_FILESYSTEM_PROPERTY_ENTRIES = + new ImmutableMap.Builder<String, PropertyEntry<?>>() + .put( + S3_ENDPOINT, + PropertyEntry.stringOptionalPropertyEntry( + S3_ENDPOINT, + "The endpoint of the AWS s3", + false /* immutable */, + null /* defaultValue */, + false /* hidden */)) + .put( + S3_ACCESS_KEY, + PropertyEntry.stringOptionalPropertyEntry( + S3_ACCESS_KEY, + "The access key of the AWS s3", + false /* immutable */, + null /* defaultValue */, + false /* hidden */)) + .put( + S3_SECRET_KEY, + PropertyEntry.stringOptionalPropertyEntry( + S3_SECRET_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 ffce417a0..06777b828 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,9 @@ */ 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; import static org.apache.gravitino.catalog.lakehouse.paimon.PaimonConfig.CATALOG_URI; @@ -123,11 +126,23 @@ public class CatalogUtils { } } - 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; + public static Map<String, String> toInnerProperty( + Map<String, String> properties, boolean keepUnknown) { + Map<String, String> 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 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 bd907d06d..b5ac224d9 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 @@ public abstract class CatalogPaimonBaseIT extends AbstractIT { // 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); + Map<String, String> copy = CatalogUtils.toInnerProperty(catalogProperties, true); PaimonBackendCatalogWrapper paimonBackendCatalogWrapper = CatalogUtils.loadCatalogBackend(new PaimonConfig(copy)); 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/CatalogPaimonOSSIT.java similarity index 53% copy from catalogs/catalog-lakehouse-paimon/src/test/java/org/apache/gravitino/catalog/lakehouse/paimon/integration/test/CatalogPaimonS3IT.java copy to catalogs/catalog-lakehouse-paimon/src/test/java/org/apache/gravitino/catalog/lakehouse/paimon/integration/test/CatalogPaimonOSSIT.java index 2030bbd19..9c4ec2a56 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/CatalogPaimonOSSIT.java @@ -21,30 +21,29 @@ 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.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; -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 { +@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 S3_BUCKET_NAME = "my-test-bucket"; - private static GravitinoLocalStackContainer localStackContainer; - private String accessKey; - private String secretKey; - private String endpoint; + 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_S3_JAR_URL = - "https://repo1.maven.org/maven2/org/apache/paimon/paimon-s3/0.8.0/paimon-s3-0.8.0.jar"; + 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<String, String> initPaimonCatalogProperties() { @@ -54,59 +53,33 @@ public class CatalogPaimonS3IT extends CatalogPaimonBaseIT { catalogProperties.put("key2", "val2"); TYPE = "filesystem"; - WAREHOUSE = "s3://" + S3_BUCKET_NAME + "/"; - - accessKey = "accessKey"; - secretKey = "secretKey"; - endpoint = String.format("http://%s:%d", localStackContainer.getContainerIpAddress(), 4566); + WAREHOUSE = warehouse; 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); + 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. - downloadS3Dependency(); + // Need to download the OSS dependency in the deploy mode. + downloadOSSDependency(); return catalogProperties; } - private void downloadS3Dependency() { + 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_S3_JAR_URL, serverPath, paimonCatalogPath); + DownloaderUtils.downloadFile(PAIMON_OSS_JAR_URL, paimonCatalogPath); } } catch (Exception e) { - throw new RuntimeException("Failed to download the S3 dependency", e); + throw new RuntimeException("Failed to download the OSS 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() @@ -118,9 +91,9 @@ public class CatalogPaimonS3IT extends CatalogPaimonBaseIT { .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) + .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/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 2030bbd19..f3786b391 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 @@ public class CatalogPaimonS3IT extends CatalogPaimonBaseIT { 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); diff --git a/docs/iceberg-rest-service.md b/docs/iceberg-rest-service.md index 30fddd0b4..4217350da 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 e5fb05a9b..b9c2772bc 100644 --- a/docs/lakehouse-paimon-catalog.md +++ b/docs/lakehouse-paimon-catalog.md @@ -30,19 +30,26 @@ 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-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 or `s3://{bucket-name}/path/` for S3 | (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 | -| `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 | +| 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/). +::: 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 5e1446a8f..44a4737d2 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" }