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

[#4940] feat(lakehouse-paimon): Support OSS filesystem for Paimon catalog. #4941

Merged
merged 19 commits into from
Oct 11, 2024
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
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
1 change: 1 addition & 0 deletions catalogs/catalog-lakehouse-paimon/build.gradle.kts
Original file line number Diff line number Diff line change
Expand Up @@ -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)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,10 +31,13 @@
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;

/**
Expand All @@ -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,
Expand All @@ -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(
Expand Down Expand Up @@ -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);
}

Expand All @@ -111,22 +120,7 @@ protected Map<String, PropertyEntry<?>> specificPropertyEntries() {
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);
}
}
Original file line number Diff line number Diff line change
@@ -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();
}
Original file line number Diff line number Diff line change
@@ -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();
}
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -123,11 +126,23 @@ private static void checkPaimonConfig(PaimonConfig paimonConfig) {
}
}

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;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -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<String, String> copy = CatalogUtils.toPaimonCatalogProperties(catalogProperties);
Map<String, String> copy = CatalogUtils.toInnerProperty(catalogProperties, true);

PaimonBackendCatalogWrapper paimonBackendCatalogWrapper =
CatalogUtils.loadCatalogBackend(new PaimonConfig(copy));
Expand Down
Loading
Loading