Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[#4938]feat(lakehouse-paimon): Support S3 filesystem for Paimon catalog. #4939

Merged
merged 24 commits into from
Oct 10, 2024
Merged
Show file tree
Hide file tree
Changes from 21 commits
Commits
Show all changes
24 commits
Select commit Hold shift + click to select a range
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
3 changes: 2 additions & 1 deletion catalogs/catalog-common/build.gradle.kts
Original file line number Diff line number Diff line change
Expand Up @@ -23,8 +23,9 @@ plugins {

// try to avoid adding extra dependencies because it is used by catalogs and connectors.
dependencies {
implementation(libs.slf4j.api)
implementation(libs.commons.lang3)
implementation(libs.guava)
implementation(libs.slf4j.api)

testImplementation(libs.junit.jupiter.api)
testImplementation(libs.junit.jupiter.params)
Expand Down
6 changes: 6 additions & 0 deletions catalogs/catalog-lakehouse-paimon/build.gradle.kts
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,9 @@ dependencies {
implementation(project(":api")) {
exclude("*")
}
implementation(project(":catalogs:catalog-common")) {
exclude("*")
}
implementation(project(":common")) {
exclude("*")
}
Expand Down Expand Up @@ -121,7 +124,10 @@ dependencies {
testImplementation(libs.postgresql.driver)
testImplementation(libs.bundles.log4j)
testImplementation(libs.junit.jupiter.params)
testImplementation(libs.paimon.s3)
testImplementation(libs.paimon.spark)
testImplementation(libs.testcontainers)
testImplementation(libs.testcontainers.localstack)

testRuntimeOnly(libs.junit.jupiter.engine)
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -114,7 +115,9 @@ public void initialize(
Map<String, String> 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);
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,9 +31,11 @@
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.S3StorageConfig;
import org.apache.gravitino.connector.BaseCatalogPropertiesMetadata;
import org.apache.gravitino.connector.PropertiesMetadata;
import org.apache.gravitino.connector.PropertyEntry;
import org.apache.gravitino.storage.S3Properties;

/**
* Implementation of {@link PropertiesMetadata} that represents Paimon catalog properties metadata.
Expand All @@ -45,6 +47,11 @@ public class PaimonCatalogPropertiesMetadata extends BaseCatalogPropertiesMetada
public static final String WAREHOUSE = "warehouse";
public static final String URI = "uri";

// S3 properties needed by Paimon
public static final String S3_ENDPOINT = "s3.endpoint";
public static final String S3_ACCESS_KEY = "s3.access-key";
public static final String S3_SECRET_KEY = "s3.secret-key";

public static final Map<String, String> GRAVITINO_CONFIG_TO_PAIMON =
ImmutableMap.of(GRAVITINO_CATALOG_BACKEND, PAIMON_METASTORE, WAREHOUSE, WAREHOUSE, URI, URI);
private static final Map<String, PropertyEntry<?>> PROPERTIES_METADATA;
Expand All @@ -61,6 +68,12 @@ public class PaimonCatalogPropertiesMetadata extends BaseCatalogPropertiesMetada
AuthenticationConfig.AUTH_TYPE_KEY,
AuthenticationConfig.AUTH_TYPE_KEY);

public static final Map<String, String> S3_CONFIGURATION =
ImmutableMap.of(
S3Properties.GRAVITINO_S3_ACCESS_KEY_ID, S3_ACCESS_KEY,
S3Properties.GRAVITINO_S3_SECRET_ACCESS_KEY, S3_SECRET_KEY,
S3Properties.GRAVITINO_S3_ENDPOINT, S3_ENDPOINT);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

should we consider s3 region?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

According to the doc provided by Paimon, region is excessive.


static {
List<PropertyEntry<?>> propertyEntries =
ImmutableList.of(
Expand Down Expand Up @@ -88,6 +101,8 @@ public class PaimonCatalogPropertiesMetadata extends BaseCatalogPropertiesMetada
result.putAll(Maps.uniqueIndex(propertyEntries, PropertyEntry::getName));
result.putAll(KerberosConfig.KERBEROS_PROPERTY_ENTRIES);
result.putAll(AuthenticationConfig.AUTHENTICATION_PROPERTY_ENTRIES);
result.putAll(S3StorageConfig.S3_FILESYSTEM_PROPERTY_ENTRIES);

PROPERTIES_METADATA = ImmutableMap.copyOf(result);
}

Expand All @@ -107,6 +122,10 @@ protected Map<String, String> transformProperties(Map<String, String> 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;
}
Expand Down
Original file line number Diff line number Diff line change
@@ -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 {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

please remove this class if not used

LOCAL_FILE,
HDFS,
S3;

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) {
FANNG1 marked this conversation as resolved.
Show resolved Hide resolved
if (storagePath.startsWith("s3://")) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

s3a?

Copy link
Contributor Author

@yuqi1129 yuqi1129 Oct 9, 2024

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

let me check it for Paimon, the value is s3:// in the example shown in https://paimon.apache.org/docs/0.8/filesystems/s3/

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Is the value s3a for Iceberg S3 storage?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

shoud we consider upper cases like S3://?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

HDFS only support lower case 'hdfs', if we use HDFS://xxx

Caused by: MetaException(message:Got exception: java.io.IOException No FileSystem for scheme: HDFS)
	at org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore$create_database_result$create_database_resultStandardScheme.read(ThriftHiveMetastore.java:26660)
	at org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore$create_database_result$create_database_resultStandardScheme.read(ThriftHiveMetastore.java:26628)
	at org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore$create_database_result.read(ThriftHiveMetastore.java:26562)
	at org.apache.thrift.TServiceClient.receiveBase(TServiceClient.java:88)

return S3;
} else if (storagePath.startsWith("hdfs://")) {
return HDFS;
} else if (storagePath.startsWith("/") || storagePath.startsWith("file://")) {
return LOCAL_FILE;
}

throw new IllegalArgumentException("Unsupported storage path: " + storagePath);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,105 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.gravitino.catalog.lakehouse.paimon.filesystem;

import static org.apache.gravitino.storage.S3Properties.GRAVITINO_S3_ACCESS_KEY_ID;
import static org.apache.gravitino.storage.S3Properties.GRAVITINO_S3_ENDPOINT;
import static org.apache.gravitino.storage.S3Properties.GRAVITINO_S3_SECRET_ACCESS_KEY;

import com.google.common.collect.ImmutableMap;
import java.util.Map;
import org.apache.commons.lang3.StringUtils;
import org.apache.gravitino.Config;
import org.apache.gravitino.config.ConfigBuilder;
import org.apache.gravitino.config.ConfigConstants;
import org.apache.gravitino.config.ConfigEntry;
import org.apache.gravitino.connector.PropertyEntry;

public class S3StorageConfig extends Config {

public S3StorageConfig(Map<String, String> properties) {
super(false);
loadFromMap(properties, k -> true);
}

// Unified S3
public static final ConfigEntry<String> PAIMON_S3_ENDPOINT_ENTRY =
new ConfigBuilder(GRAVITINO_S3_ENDPOINT)
.doc("The endpoint of the AWS s3")
.version(ConfigConstants.VERSION_0_7_0)
.stringConf()
.checkValue(StringUtils::isNotBlank, ConfigConstants.NOT_BLANK_ERROR_MSG)
.create();

public static final ConfigEntry<String> PAIMON_S3_ACCESS_KEY_ENTRY =
new ConfigBuilder(GRAVITINO_S3_ACCESS_KEY_ID)
.doc("The access key of the AWS s3")
.version(ConfigConstants.VERSION_0_7_0)
.stringConf()
.checkValue(StringUtils::isNotBlank, ConfigConstants.NOT_BLANK_ERROR_MSG)
.create();

public static final ConfigEntry<String> PAIMON_S3_SECRET_KEY_ENTRY =
new ConfigBuilder(GRAVITINO_S3_SECRET_ACCESS_KEY)
.doc("The secret key of the AWS s3")
.version(ConfigConstants.VERSION_0_7_0)
.stringConf()
.checkValue(StringUtils::isNotBlank, ConfigConstants.NOT_BLANK_ERROR_MSG)
.create();

public String getS3Endpoint() {
return get(PAIMON_S3_ENDPOINT_ENTRY);
}

public String getS3AccessKey() {
return get(PAIMON_S3_ACCESS_KEY_ENTRY);
}

public String getS3SecretKey() {
return get(PAIMON_S3_SECRET_KEY_ENTRY);
}

public static final Map<String, PropertyEntry<?>> S3_FILESYSTEM_PROPERTY_ENTRIES =
new ImmutableMap.Builder<String, PropertyEntry<?>>()
.put(
GRAVITINO_S3_ENDPOINT,
PropertyEntry.stringOptionalPropertyEntry(
GRAVITINO_S3_ENDPOINT,
"The endpoint of the AWS s3",
false /* immutable */,
null /* defaultValue */,
false /* hidden */))
.put(
GRAVITINO_S3_ACCESS_KEY_ID,
PropertyEntry.stringOptionalPropertyEntry(
GRAVITINO_S3_ACCESS_KEY_ID,
"The access key of the AWS s3",
false /* immutable */,
null /* defaultValue */,
false /* hidden */))
.put(
GRAVITINO_S3_SECRET_ACCESS_KEY,
PropertyEntry.stringOptionalPropertyEntry(
GRAVITINO_S3_SECRET_ACCESS_KEY,
"The secret key of the AWS s3",
false /* immutable */,
null /* defaultValue */,
false /* hidden */))
.build();
}
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,7 @@
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.ops.PaimonBackendCatalogWrapper;
import org.apache.hadoop.conf.Configuration;
import org.apache.paimon.catalog.Catalog;
Expand Down Expand Up @@ -120,4 +121,37 @@ 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<String, String> 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 LOCAL_FILE:
break;
default:
throw new IllegalArgumentException("Unsupported file system type: " + fileSystemType);
}
}

@SuppressWarnings("unused")
private static void checkS3FileSystemConfig(Map<String, String> resultConf) {
FANNG1 marked this conversation as resolved.
Show resolved Hide resolved
// S3StorageConfig s3FileSystemConfig = new S3StorageConfig(resultConf);
yuqi1129 marked this conversation as resolved.
Show resolved Hide resolved
// 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.");
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -67,6 +67,7 @@
import org.apache.gravitino.rel.expressions.transforms.Transforms;
import org.apache.gravitino.rel.indexes.Index;
import org.apache.gravitino.rel.types.Types;
import org.apache.gravitino.storage.S3Properties;
import org.apache.paimon.catalog.Catalog.DatabaseNotExistException;
import org.apache.paimon.catalog.Identifier;
import org.apache.paimon.schema.TableSchema;
Expand Down Expand Up @@ -109,19 +110,23 @@ public abstract class CatalogPaimonBaseIT extends AbstractIT {
private GravitinoMetalake metalake;
private Catalog catalog;
private org.apache.paimon.catalog.Catalog paimonCatalog;
private SparkSession spark;
protected SparkSession spark;
private Map<String, String> catalogProperties;

@BeforeAll
public void startup() {
containerSuite.startHiveContainer();
startNecessaryContainers();
catalogProperties = initPaimonCatalogProperties();
createMetalake();
createCatalog();
createSchema();
initSparkEnv();
}

protected void startNecessaryContainers() {
containerSuite.startHiveContainer();
}

@AfterAll
public void stop() {
clearTableAndSchema();
Expand Down Expand Up @@ -882,8 +887,30 @@ private void createCatalog() {
Preconditions.checkArgument(
StringUtils.isNotBlank(type), "Paimon Catalog backend type can not be null or empty.");
catalogProperties.put(PaimonCatalogPropertiesMetadata.PAIMON_METASTORE, type);

// Why needs this conversion? Because PaimonCatalogOperations#initialize will try to convert
// Gravitino general S3 properties to Paimon specific S3 properties.
Map<String, String> copy = Maps.newHashMap(catalogProperties);
yuqi1129 marked this conversation as resolved.
Show resolved Hide resolved
if (catalogProperties.containsKey(S3Properties.GRAVITINO_S3_ENDPOINT)) {
copy.put(
PaimonCatalogPropertiesMetadata.S3_ENDPOINT,
catalogProperties.get(S3Properties.GRAVITINO_S3_ENDPOINT));
}

if (catalogProperties.containsKey(S3Properties.GRAVITINO_S3_ACCESS_KEY_ID)) {
copy.put(
PaimonCatalogPropertiesMetadata.S3_ACCESS_KEY,
catalogProperties.get(S3Properties.GRAVITINO_S3_ACCESS_KEY_ID));
}

if (catalogProperties.containsKey(S3Properties.GRAVITINO_S3_SECRET_ACCESS_KEY)) {
copy.put(
PaimonCatalogPropertiesMetadata.S3_SECRET_KEY,
catalogProperties.get(S3Properties.GRAVITINO_S3_SECRET_ACCESS_KEY));
}

PaimonBackendCatalogWrapper paimonBackendCatalogWrapper =
CatalogUtils.loadCatalogBackend(new PaimonConfig(catalogProperties));
CatalogUtils.loadCatalogBackend(new PaimonConfig(copy));
paimonCatalog = paimonBackendCatalogWrapper.getCatalog();
}

Expand Down Expand Up @@ -926,7 +953,7 @@ private Map<String, String> createProperties() {
return properties;
}

private void initSparkEnv() {
protected void initSparkEnv() {
spark =
SparkSession.builder()
.master("local[1]")
Expand Down
Loading
Loading