This is an automated email from the ASF dual-hosted git repository.
fanng pushed a commit to branch branch-0.8
in repository https://gitbox.apache.org/repos/asf/gravitino.git
The following commit(s) were added to refs/heads/branch-0.8 by this push:
new 016e415964 [#5192] [#5193] feat(flink): Support Catalog&Schema
Operation DDL for paimon-catalog (#6215)
016e415964 is described below
commit 016e415964940689363d598d20a4870b608b8617
Author: github-actions[bot]
<41898282+github-actions[bot]@users.noreply.github.com>
AuthorDate: Tue Jan 14 08:54:27 2025 +0800
[#5192] [#5193] feat(flink): Support Catalog&Schema Operation DDL for
paimon-catalog (#6215)
### What changes were proposed in this pull request?
Support Catalog Operation DDL for paimon-catalog
### Why are the changes needed?
Fix #5192 #5193
### Does this PR introduce _any_ user-facing change?
None
### How was this patch tested?
org.apache.gravitino.flink.connector.paimon.TestPaimonPropertiesConverter
org.apache.gravitino.flink.connector.integration.test.paimon.FlinkPaimonCatalogIT
Co-authored-by: yangyang zhong <[email protected]>
---
.../lakehouse/paimon/PaimonPropertiesUtils.java | 46 ++++++---
flink-connector/flink/build.gradle.kts | 5 +-
.../connector/paimon/GravitinoPaimonCatalog.java | 48 +++++++++
.../paimon/GravitinoPaimonCatalogFactory.java | 80 +++++++++++++++
.../GravitinoPaimonCatalogFactoryOptions.java | 26 +++++
.../paimon/PaimonPropertiesConverter.java | 80 +++++++++++++++
.../connector/store/GravitinoCatalogStore.java | 3 +-
.../org.apache.flink.table.factories.Factory | 3 +-
.../connector/integration/test/FlinkCommonIT.java | 54 +++++++++-
.../test/paimon/FlinkPaimonCatalogIT.java | 111 +++++++++++++++++++++
.../paimon/TestPaimonPropertiesConverter.java | 101 +++++++++++++++++++
11 files changed, 536 insertions(+), 21 deletions(-)
diff --git
a/catalogs/catalog-common/src/main/java/org/apache/gravitino/catalog/lakehouse/paimon/PaimonPropertiesUtils.java
b/catalogs/catalog-common/src/main/java/org/apache/gravitino/catalog/lakehouse/paimon/PaimonPropertiesUtils.java
index 0dcf24f3a6..7b1832fe56 100644
---
a/catalogs/catalog-common/src/main/java/org/apache/gravitino/catalog/lakehouse/paimon/PaimonPropertiesUtils.java
+++
b/catalogs/catalog-common/src/main/java/org/apache/gravitino/catalog/lakehouse/paimon/PaimonPropertiesUtils.java
@@ -32,25 +32,41 @@ public class PaimonPropertiesUtils {
// will only need to set the configuration 'catalog-backend' in Gravitino
and Gravitino will
// change it to `catalogType` automatically and pass it to Paimon.
public static final Map<String, String> GRAVITINO_CONFIG_TO_PAIMON;
+ public static final Map<String, String> PAIMON_CATALOG_CONFIG_TO_GRAVITINO;
static {
- Map<String, String> map = new HashMap();
- map.put(PaimonConstants.CATALOG_BACKEND, PaimonConstants.CATALOG_BACKEND);
- map.put(PaimonConstants.GRAVITINO_JDBC_DRIVER,
PaimonConstants.GRAVITINO_JDBC_DRIVER);
- map.put(PaimonConstants.GRAVITINO_JDBC_USER,
PaimonConstants.PAIMON_JDBC_USER);
- map.put(PaimonConstants.GRAVITINO_JDBC_PASSWORD,
PaimonConstants.PAIMON_JDBC_PASSWORD);
- map.put(PaimonConstants.URI, PaimonConstants.URI);
- map.put(PaimonConstants.WAREHOUSE, PaimonConstants.WAREHOUSE);
- map.put(PaimonConstants.CATALOG_BACKEND_NAME,
PaimonConstants.CATALOG_BACKEND_NAME);
+ Map<String, String> gravitinoConfigToPaimon = new HashMap<>();
+ Map<String, String> paimonCatalogConfigToGravitino = new HashMap<>();
+ gravitinoConfigToPaimon.put(PaimonConstants.CATALOG_BACKEND,
PaimonConstants.CATALOG_BACKEND);
+ gravitinoConfigToPaimon.put(
+ PaimonConstants.GRAVITINO_JDBC_DRIVER,
PaimonConstants.GRAVITINO_JDBC_DRIVER);
+ gravitinoConfigToPaimon.put(
+ PaimonConstants.GRAVITINO_JDBC_USER, PaimonConstants.PAIMON_JDBC_USER);
+ gravitinoConfigToPaimon.put(
+ PaimonConstants.GRAVITINO_JDBC_PASSWORD,
PaimonConstants.PAIMON_JDBC_PASSWORD);
+ gravitinoConfigToPaimon.put(PaimonConstants.URI, PaimonConstants.URI);
+ gravitinoConfigToPaimon.put(PaimonConstants.WAREHOUSE,
PaimonConstants.WAREHOUSE);
+ gravitinoConfigToPaimon.put(
+ PaimonConstants.CATALOG_BACKEND_NAME,
PaimonConstants.CATALOG_BACKEND_NAME);
// S3
- map.put(S3Properties.GRAVITINO_S3_ENDPOINT, PaimonConstants.S3_ENDPOINT);
- map.put(S3Properties.GRAVITINO_S3_ACCESS_KEY_ID,
PaimonConstants.S3_ACCESS_KEY);
- map.put(S3Properties.GRAVITINO_S3_SECRET_ACCESS_KEY,
PaimonConstants.S3_SECRET_KEY);
+ gravitinoConfigToPaimon.put(S3Properties.GRAVITINO_S3_ENDPOINT,
PaimonConstants.S3_ENDPOINT);
+ gravitinoConfigToPaimon.put(
+ S3Properties.GRAVITINO_S3_ACCESS_KEY_ID,
PaimonConstants.S3_ACCESS_KEY);
+ gravitinoConfigToPaimon.put(
+ S3Properties.GRAVITINO_S3_SECRET_ACCESS_KEY,
PaimonConstants.S3_SECRET_KEY);
// OSS
- map.put(OSSProperties.GRAVITINO_OSS_ENDPOINT,
PaimonConstants.OSS_ENDPOINT);
- map.put(OSSProperties.GRAVITINO_OSS_ACCESS_KEY_ID,
PaimonConstants.OSS_ACCESS_KEY);
- map.put(OSSProperties.GRAVITINO_OSS_ACCESS_KEY_SECRET,
PaimonConstants.OSS_SECRET_KEY);
- GRAVITINO_CONFIG_TO_PAIMON = Collections.unmodifiableMap(map);
+ gravitinoConfigToPaimon.put(OSSProperties.GRAVITINO_OSS_ENDPOINT,
PaimonConstants.OSS_ENDPOINT);
+ gravitinoConfigToPaimon.put(
+ OSSProperties.GRAVITINO_OSS_ACCESS_KEY_ID,
PaimonConstants.OSS_ACCESS_KEY);
+ gravitinoConfigToPaimon.put(
+ OSSProperties.GRAVITINO_OSS_ACCESS_KEY_SECRET,
PaimonConstants.OSS_SECRET_KEY);
+ GRAVITINO_CONFIG_TO_PAIMON =
Collections.unmodifiableMap(gravitinoConfigToPaimon);
+ gravitinoConfigToPaimon.forEach(
+ (key, value) -> {
+ paimonCatalogConfigToGravitino.put(value, key);
+ });
+ PAIMON_CATALOG_CONFIG_TO_GRAVITINO =
+ Collections.unmodifiableMap(paimonCatalogConfigToGravitino);
}
/**
diff --git a/flink-connector/flink/build.gradle.kts
b/flink-connector/flink/build.gradle.kts
index 9e2a48c036..f137a3eae1 100644
--- a/flink-connector/flink/build.gradle.kts
+++ b/flink-connector/flink/build.gradle.kts
@@ -26,6 +26,7 @@ repositories {
mavenCentral()
}
+var paimonVersion: String = libs.versions.paimon.get()
val flinkVersion: String = libs.versions.flink.get()
val flinkMajorVersion: String = flinkVersion.substringBeforeLast(".")
@@ -38,14 +39,15 @@ val scalaVersion: String = "2.12"
val artifactName =
"${rootProject.name}-flink-${flinkMajorVersion}_$scalaVersion"
dependencies {
+ implementation(project(":core"))
implementation(project(":catalogs:catalog-common"))
implementation(libs.guava)
compileOnly(project(":clients:client-java-runtime", configuration =
"shadow"))
-
compileOnly("org.apache.flink:flink-connector-hive_$scalaVersion:$flinkVersion")
compileOnly("org.apache.flink:flink-table-common:$flinkVersion")
compileOnly("org.apache.flink:flink-table-api-java:$flinkVersion")
+ compileOnly("org.apache.paimon:paimon-flink-1.18:$paimonVersion")
compileOnly(libs.hive2.exec) {
artifact {
@@ -90,6 +92,7 @@ dependencies {
testImplementation("org.apache.flink:flink-connector-hive_$scalaVersion:$flinkVersion")
testImplementation("org.apache.flink:flink-table-common:$flinkVersion")
testImplementation("org.apache.flink:flink-table-api-java:$flinkVersion")
+
testImplementation("org.apache.paimon:paimon-flink-$flinkMajorVersion:$paimonVersion")
testImplementation(libs.hive2.exec) {
artifact {
diff --git
a/flink-connector/flink/src/main/java/org/apache/gravitino/flink/connector/paimon/GravitinoPaimonCatalog.java
b/flink-connector/flink/src/main/java/org/apache/gravitino/flink/connector/paimon/GravitinoPaimonCatalog.java
new file mode 100644
index 0000000000..017ac6e708
--- /dev/null
+++
b/flink-connector/flink/src/main/java/org/apache/gravitino/flink/connector/paimon/GravitinoPaimonCatalog.java
@@ -0,0 +1,48 @@
+/*
+ * 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.flink.connector.paimon;
+
+import org.apache.flink.table.catalog.AbstractCatalog;
+import org.apache.gravitino.flink.connector.PartitionConverter;
+import org.apache.gravitino.flink.connector.PropertiesConverter;
+import org.apache.gravitino.flink.connector.catalog.BaseCatalog;
+
+/**
+ * The GravitinoPaimonCatalog class is an implementation of the BaseCatalog
class that is used to
+ * proxy the PaimonCatalog class.
+ */
+public class GravitinoPaimonCatalog extends BaseCatalog {
+
+ private final AbstractCatalog paimonCatalog;
+
+ protected GravitinoPaimonCatalog(
+ String catalogName,
+ AbstractCatalog paimonCatalog,
+ PropertiesConverter propertiesConverter,
+ PartitionConverter partitionConverter) {
+ super(catalogName, paimonCatalog.getDefaultDatabase(),
propertiesConverter, partitionConverter);
+ this.paimonCatalog = paimonCatalog;
+ }
+
+ @Override
+ protected AbstractCatalog realCatalog() {
+ return paimonCatalog;
+ }
+}
diff --git
a/flink-connector/flink/src/main/java/org/apache/gravitino/flink/connector/paimon/GravitinoPaimonCatalogFactory.java
b/flink-connector/flink/src/main/java/org/apache/gravitino/flink/connector/paimon/GravitinoPaimonCatalogFactory.java
new file mode 100644
index 0000000000..52489fc667
--- /dev/null
+++
b/flink-connector/flink/src/main/java/org/apache/gravitino/flink/connector/paimon/GravitinoPaimonCatalogFactory.java
@@ -0,0 +1,80 @@
+/*
+ * 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.flink.connector.paimon;
+
+import java.util.Collections;
+import java.util.Set;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.table.catalog.Catalog;
+import org.apache.gravitino.flink.connector.DefaultPartitionConverter;
+import org.apache.gravitino.flink.connector.PartitionConverter;
+import org.apache.gravitino.flink.connector.PropertiesConverter;
+import org.apache.gravitino.flink.connector.catalog.BaseCatalogFactory;
+import org.apache.paimon.flink.FlinkCatalog;
+import org.apache.paimon.flink.FlinkCatalogFactory;
+
+/**
+ * Factory for creating instances of {@link GravitinoPaimonCatalog}. It will
be created by SPI
+ * discovery in Flink.
+ */
+public class GravitinoPaimonCatalogFactory implements BaseCatalogFactory {
+
+ @Override
+ public Catalog createCatalog(Context context) {
+ FlinkCatalog catalog = new FlinkCatalogFactory().createCatalog(context);
+ return new GravitinoPaimonCatalog(
+ context.getName(), catalog, propertiesConverter(),
partitionConverter());
+ }
+
+ @Override
+ public String factoryIdentifier() {
+ return GravitinoPaimonCatalogFactoryOptions.IDENTIFIER;
+ }
+
+ @Override
+ public Set<ConfigOption<?>> requiredOptions() {
+ return Collections.emptySet();
+ }
+
+ @Override
+ public Set<ConfigOption<?>> optionalOptions() {
+ return Collections.emptySet();
+ }
+
+ @Override
+ public String gravitinoCatalogProvider() {
+ return "lakehouse-paimon";
+ }
+
+ @Override
+ public org.apache.gravitino.Catalog.Type gravitinoCatalogType() {
+ return org.apache.gravitino.Catalog.Type.RELATIONAL;
+ }
+
+ @Override
+ public PropertiesConverter propertiesConverter() {
+ return PaimonPropertiesConverter.INSTANCE;
+ }
+
+ @Override
+ public PartitionConverter partitionConverter() {
+ return DefaultPartitionConverter.INSTANCE;
+ }
+}
diff --git
a/flink-connector/flink/src/main/java/org/apache/gravitino/flink/connector/paimon/GravitinoPaimonCatalogFactoryOptions.java
b/flink-connector/flink/src/main/java/org/apache/gravitino/flink/connector/paimon/GravitinoPaimonCatalogFactoryOptions.java
new file mode 100644
index 0000000000..dd78f96d24
--- /dev/null
+++
b/flink-connector/flink/src/main/java/org/apache/gravitino/flink/connector/paimon/GravitinoPaimonCatalogFactoryOptions.java
@@ -0,0 +1,26 @@
+/*
+ * 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.flink.connector.paimon;
+
+public class GravitinoPaimonCatalogFactoryOptions {
+
+ /** Identifier for the {@link GravitinoPaimonCatalog}. */
+ public static final String IDENTIFIER = "gravitino-paimon";
+}
diff --git
a/flink-connector/flink/src/main/java/org/apache/gravitino/flink/connector/paimon/PaimonPropertiesConverter.java
b/flink-connector/flink/src/main/java/org/apache/gravitino/flink/connector/paimon/PaimonPropertiesConverter.java
new file mode 100644
index 0000000000..58613bee37
--- /dev/null
+++
b/flink-connector/flink/src/main/java/org/apache/gravitino/flink/connector/paimon/PaimonPropertiesConverter.java
@@ -0,0 +1,80 @@
+/*
+ * 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.flink.connector.paimon;
+
+import com.google.common.collect.Maps;
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.table.catalog.CommonCatalogOptions;
+import org.apache.gravitino.catalog.lakehouse.paimon.PaimonConstants;
+import org.apache.gravitino.catalog.lakehouse.paimon.PaimonPropertiesUtils;
+import org.apache.gravitino.flink.connector.PropertiesConverter;
+import org.apache.paimon.catalog.FileSystemCatalogFactory;
+
+public class PaimonPropertiesConverter implements PropertiesConverter {
+
+ public static final PaimonPropertiesConverter INSTANCE = new
PaimonPropertiesConverter();
+
+ private PaimonPropertiesConverter() {}
+
+ @Override
+ public Map<String, String> toGravitinoCatalogProperties(Configuration
flinkConf) {
+ Map<String, String> gravitinoProperties = Maps.newHashMap();
+ Map<String, String> flinkConfMap = flinkConf.toMap();
+ for (Map.Entry<String, String> entry : flinkConfMap.entrySet()) {
+ String gravitinoKey =
+
PaimonPropertiesUtils.PAIMON_CATALOG_CONFIG_TO_GRAVITINO.get(entry.getKey());
+ if (gravitinoKey != null) {
+ gravitinoProperties.put(gravitinoKey, entry.getValue());
+ } else if (!entry.getKey().startsWith(FLINK_PROPERTY_PREFIX)) {
+ gravitinoProperties.put(FLINK_PROPERTY_PREFIX + entry.getKey(),
entry.getValue());
+ } else {
+ gravitinoProperties.put(entry.getKey(), entry.getValue());
+ }
+ }
+ gravitinoProperties.put(
+ PaimonConstants.CATALOG_BACKEND,
+ flinkConfMap.getOrDefault(PaimonConstants.METASTORE,
FileSystemCatalogFactory.IDENTIFIER));
+ return gravitinoProperties;
+ }
+
+ @Override
+ public Map<String, String> toFlinkCatalogProperties(Map<String, String>
gravitinoProperties) {
+ Map<String, String> all = new HashMap<>();
+ gravitinoProperties.forEach(
+ (key, value) -> {
+ String flinkConfigKey = key;
+ if (key.startsWith(PropertiesConverter.FLINK_PROPERTY_PREFIX)) {
+ flinkConfigKey =
key.substring(PropertiesConverter.FLINK_PROPERTY_PREFIX.length());
+ }
+ all.put(flinkConfigKey, value);
+ });
+ Map<String, String> paimonCatalogProperties =
+ PaimonPropertiesUtils.toPaimonCatalogProperties(all);
+ paimonCatalogProperties.put(
+ PaimonConstants.METASTORE,
+ paimonCatalogProperties.getOrDefault(
+ PaimonConstants.CATALOG_BACKEND,
FileSystemCatalogFactory.IDENTIFIER));
+ paimonCatalogProperties.put(
+ CommonCatalogOptions.CATALOG_TYPE.key(),
GravitinoPaimonCatalogFactoryOptions.IDENTIFIER);
+ return paimonCatalogProperties;
+ }
+}
diff --git
a/flink-connector/flink/src/main/java/org/apache/gravitino/flink/connector/store/GravitinoCatalogStore.java
b/flink-connector/flink/src/main/java/org/apache/gravitino/flink/connector/store/GravitinoCatalogStore.java
index 92e778ce29..4c29b7fde3 100644
---
a/flink-connector/flink/src/main/java/org/apache/gravitino/flink/connector/store/GravitinoCatalogStore.java
+++
b/flink-connector/flink/src/main/java/org/apache/gravitino/flink/connector/store/GravitinoCatalogStore.java
@@ -54,7 +54,8 @@ public class GravitinoCatalogStore extends
AbstractCatalogStore {
public void storeCatalog(String catalogName, CatalogDescriptor descriptor)
throws CatalogException {
Configuration configuration = descriptor.getConfiguration();
- BaseCatalogFactory catalogFactory =
getCatalogFactory(configuration.toMap());
+ Map<String, String> gravitino = configuration.toMap();
+ BaseCatalogFactory catalogFactory = getCatalogFactory(gravitino);
Map<String, String> gravitinoProperties =
catalogFactory.propertiesConverter().toGravitinoCatalogProperties(configuration);
gravitinoCatalogManager.createCatalog(
diff --git
a/flink-connector/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory
b/flink-connector/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory
index c9d9c92b5e..a535afb6dc 100644
---
a/flink-connector/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory
+++
b/flink-connector/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory
@@ -18,4 +18,5 @@
#
org.apache.gravitino.flink.connector.store.GravitinoCatalogStoreFactory
-org.apache.gravitino.flink.connector.hive.GravitinoHiveCatalogFactory
\ No newline at end of file
+org.apache.gravitino.flink.connector.hive.GravitinoHiveCatalogFactory
+org.apache.gravitino.flink.connector.paimon.GravitinoPaimonCatalogFactory
\ No newline at end of file
diff --git
a/flink-connector/flink/src/test/java/org/apache/gravitino/flink/connector/integration/test/FlinkCommonIT.java
b/flink-connector/flink/src/test/java/org/apache/gravitino/flink/connector/integration/test/FlinkCommonIT.java
index 2d022b4a8a..5a363e4e51 100644
---
a/flink-connector/flink/src/test/java/org/apache/gravitino/flink/connector/integration/test/FlinkCommonIT.java
+++
b/flink-connector/flink/src/test/java/org/apache/gravitino/flink/connector/integration/test/FlinkCommonIT.java
@@ -27,6 +27,7 @@ import static org.junit.jupiter.api.Assertions.fail;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import java.util.ArrayList;
+import java.util.Arrays;
import java.util.List;
import java.util.Map;
import java.util.Optional;
@@ -53,11 +54,24 @@ import org.apache.gravitino.rel.Table;
import org.apache.gravitino.rel.types.Types;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.condition.EnabledIf;
public abstract class FlinkCommonIT extends FlinkEnvIT {
protected abstract Catalog currentCatalog();
+ protected boolean supportTableOperation() {
+ return true;
+ }
+
+ protected boolean supportColumnOperation() {
+ return true;
+ }
+
+ protected boolean supportSchemaOperationWithCommentAndOptions() {
+ return true;
+ }
+
@Test
public void testCreateSchema() {
doWithCatalog(
@@ -76,7 +90,29 @@ public abstract class FlinkCommonIT extends FlinkEnvIT {
}
@Test
- public void testGetSchema() {
+ public void testGetSchemaWithoutCommentAndOption() {
+ doWithCatalog(
+ currentCatalog(),
+ catalog -> {
+ String schema = "test_get_schema";
+ try {
+ TestUtils.assertTableResult(
+ sql("CREATE DATABASE IF NOT EXISTS %s", schema),
ResultKind.SUCCESS);
+ TestUtils.assertTableResult(tableEnv.executeSql("USE " + schema),
ResultKind.SUCCESS);
+
+ catalog.asSchemas().schemaExists(schema);
+ Schema loadedSchema = catalog.asSchemas().loadSchema(schema);
+ Assertions.assertEquals(schema, loadedSchema.name());
+ } finally {
+ catalog.asSchemas().dropSchema(schema, true);
+ Assertions.assertFalse(catalog.asSchemas().schemaExists(schema));
+ }
+ });
+ }
+
+ @Test
+ @EnabledIf("supportSchemaOperationWithCommentAndOptions")
+ public void testGetSchemaWithCommentAndOptions() {
doWithCatalog(
currentCatalog(),
catalog -> {
@@ -114,7 +150,6 @@ public abstract class FlinkCommonIT extends FlinkEnvIT {
doWithCatalog(
currentCatalog(),
catalog -> {
- Assertions.assertEquals(1, catalog.asSchemas().listSchemas().length);
String schema = "test_list_schema";
String schema2 = "test_list_schema2";
String schema3 = "test_list_schema3";
@@ -135,6 +170,7 @@ public abstract class FlinkCommonIT extends FlinkEnvIT {
Row.of(schema3));
String[] schemas = catalog.asSchemas().listSchemas();
+ Arrays.sort(schemas);
Assertions.assertEquals(4, schemas.length);
Assertions.assertEquals("default", schemas[0]);
Assertions.assertEquals(schema, schemas[1]);
@@ -150,7 +186,8 @@ public abstract class FlinkCommonIT extends FlinkEnvIT {
}
@Test
- public void testAlterSchema() {
+ @EnabledIf("supportSchemaOperationWithCommentAndOptions")
+ public void testAlterSchemaWithCommentAndOptions() {
doWithCatalog(
currentCatalog(),
catalog -> {
@@ -188,6 +225,7 @@ public abstract class FlinkCommonIT extends FlinkEnvIT {
}
@Test
+ @EnabledIf("supportTableOperation")
public void testCreateSimpleTable() {
String databaseName = "test_create_no_partition_table_db";
String tableName = "test_create_no_partition_table";
@@ -236,6 +274,7 @@ public abstract class FlinkCommonIT extends FlinkEnvIT {
}
@Test
+ @EnabledIf("supportTableOperation")
public void testListTables() {
String newSchema = "test_list_table_catalog";
Column[] columns = new Column[] {Column.of("user_id",
Types.IntegerType.get(), "USER_ID")};
@@ -268,6 +307,7 @@ public abstract class FlinkCommonIT extends FlinkEnvIT {
}
@Test
+ @EnabledIf("supportTableOperation")
public void testDropTable() {
String databaseName = "test_drop_table_db";
doWithSchema(
@@ -289,6 +329,7 @@ public abstract class FlinkCommonIT extends FlinkEnvIT {
}
@Test
+ @EnabledIf("supportTableOperation")
public void testGetSimpleTable() {
String databaseName = "test_get_simple_table";
Column[] columns =
@@ -342,6 +383,7 @@ public abstract class FlinkCommonIT extends FlinkEnvIT {
}
@Test
+ @EnabledIf("supportColumnOperation")
public void testRenameColumn() {
String databaseName = "test_rename_column_db";
String tableName = "test_rename_column";
@@ -377,6 +419,7 @@ public abstract class FlinkCommonIT extends FlinkEnvIT {
}
@Test
+ @EnabledIf("supportColumnOperation")
public void testAlterTableComment() {
String databaseName = "test_alter_table_comment_database";
String tableName = "test_alter_table_comment";
@@ -436,6 +479,7 @@ public abstract class FlinkCommonIT extends FlinkEnvIT {
}
@Test
+ @EnabledIf("supportColumnOperation")
public void testAlterTableAddColumn() {
String databaseName = "test_alter_table_add_column_db";
String tableName = "test_alter_table_add_column";
@@ -471,6 +515,7 @@ public abstract class FlinkCommonIT extends FlinkEnvIT {
}
@Test
+ @EnabledIf("supportColumnOperation")
public void testAlterTableDropColumn() {
String databaseName = "test_alter_table_drop_column_db";
String tableName = "test_alter_table_drop_column";
@@ -501,6 +546,7 @@ public abstract class FlinkCommonIT extends FlinkEnvIT {
}
@Test
+ @EnabledIf("supportColumnOperation")
public void testAlterColumnTypeAndChangeOrder() {
String databaseName = "test_alter_table_alter_column_db";
String tableName = "test_alter_table_rename_column";
@@ -542,6 +588,7 @@ public abstract class FlinkCommonIT extends FlinkEnvIT {
}
@Test
+ @EnabledIf("supportTableOperation")
public void testRenameTable() {
String databaseName = "test_rename_table_db";
String tableName = "test_rename_table";
@@ -569,6 +616,7 @@ public abstract class FlinkCommonIT extends FlinkEnvIT {
}
@Test
+ @EnabledIf("supportTableOperation")
public void testAlterTableProperties() {
String databaseName = "test_alter_table_properties_db";
String tableName = "test_alter_table_properties";
diff --git
a/flink-connector/flink/src/test/java/org/apache/gravitino/flink/connector/integration/test/paimon/FlinkPaimonCatalogIT.java
b/flink-connector/flink/src/test/java/org/apache/gravitino/flink/connector/integration/test/paimon/FlinkPaimonCatalogIT.java
new file mode 100644
index 0000000000..10fab3567a
--- /dev/null
+++
b/flink-connector/flink/src/test/java/org/apache/gravitino/flink/connector/integration/test/paimon/FlinkPaimonCatalogIT.java
@@ -0,0 +1,111 @@
+/*
+ * 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.flink.connector.integration.test.paimon;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableMap;
+import java.nio.file.Path;
+import java.util.Map;
+import org.apache.gravitino.Catalog;
+import org.apache.gravitino.catalog.lakehouse.paimon.PaimonConstants;
+import org.apache.gravitino.flink.connector.integration.test.FlinkCommonIT;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Tag;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+
+@Tag("gravitino-docker-test")
+public class FlinkPaimonCatalogIT extends FlinkCommonIT {
+
+ @TempDir private static Path warehouseDir;
+
+ private static final String DEFAULT_PAIMON_CATALOG =
+ "test_flink_paimon_filesystem_schema_catalog";
+
+ private static org.apache.gravitino.Catalog catalog;
+
+ @Override
+ protected boolean supportColumnOperation() {
+ return false;
+ }
+
+ @Override
+ protected boolean supportTableOperation() {
+ return false;
+ }
+
+ @Override
+ protected boolean supportSchemaOperationWithCommentAndOptions() {
+ return false;
+ }
+
+ protected Catalog currentCatalog() {
+ return catalog;
+ }
+
+ @BeforeAll
+ static void setup() {
+ initPaimonCatalog();
+ }
+
+ @AfterAll
+ static void stop() {
+ Preconditions.checkNotNull(metalake);
+ metalake.dropCatalog(DEFAULT_PAIMON_CATALOG, true);
+ }
+
+ private static void initPaimonCatalog() {
+ Preconditions.checkNotNull(metalake);
+ catalog =
+ metalake.createCatalog(
+ DEFAULT_PAIMON_CATALOG,
+ org.apache.gravitino.Catalog.Type.RELATIONAL,
+ "lakehouse-paimon",
+ null,
+ ImmutableMap.of(
+ PaimonConstants.CATALOG_BACKEND,
+ "filesystem",
+ "warehouse",
+ warehouseDir.toString()));
+ }
+
+ @Test
+ public void testCreateGravitinoPaimonCatalogUsingSQL() {
+ tableEnv.useCatalog(DEFAULT_CATALOG);
+ int numCatalogs = tableEnv.listCatalogs().length;
+ String catalogName = "gravitino_hive_sql";
+ String warehouse = warehouseDir.toString();
+ tableEnv.executeSql(
+ String.format(
+ "create catalog %s with ("
+ + "'type'='gravitino-paimon', "
+ + "'warehouse'='%s',"
+ + "'catalog.backend'='filesystem'"
+ + ")",
+ catalogName, warehouse));
+ String[] catalogs = tableEnv.listCatalogs();
+ Assertions.assertEquals(numCatalogs + 1, catalogs.length, "Should create a
new catalog");
+ Assertions.assertTrue(metalake.catalogExists(catalogName));
+ org.apache.gravitino.Catalog gravitinoCatalog =
metalake.loadCatalog(catalogName);
+ Map<String, String> properties = gravitinoCatalog.properties();
+ Assertions.assertEquals(warehouse, properties.get("warehouse"));
+ }
+}
diff --git
a/flink-connector/flink/src/test/java/org/apache/gravitino/flink/connector/paimon/TestPaimonPropertiesConverter.java
b/flink-connector/flink/src/test/java/org/apache/gravitino/flink/connector/paimon/TestPaimonPropertiesConverter.java
new file mode 100644
index 0000000000..4496d94c0a
--- /dev/null
+++
b/flink-connector/flink/src/test/java/org/apache/gravitino/flink/connector/paimon/TestPaimonPropertiesConverter.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.flink.connector.paimon;
+
+import com.google.common.collect.ImmutableMap;
+import java.util.Map;
+import org.apache.flink.configuration.Configuration;
+import org.apache.gravitino.catalog.lakehouse.paimon.PaimonConstants;
+import org.apache.gravitino.flink.connector.PropertiesConverter;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+
+/** Test for {@link PaimonPropertiesConverter} */
+public class TestPaimonPropertiesConverter {
+
+ private static final PaimonPropertiesConverter CONVERTER =
PaimonPropertiesConverter.INSTANCE;
+
+ private static final String localWarehouse = "file:///tmp/paimon_warehouse";
+
+ @Test
+ public void testToPaimonFileSystemCatalog() {
+ Map<String, String> catalogProperties = ImmutableMap.of("warehouse",
localWarehouse);
+ Map<String, String> flinkCatalogProperties =
+ CONVERTER.toFlinkCatalogProperties(catalogProperties);
+ Assertions.assertEquals(
+ GravitinoPaimonCatalogFactoryOptions.IDENTIFIER,
flinkCatalogProperties.get("type"));
+ Assertions.assertEquals(localWarehouse,
flinkCatalogProperties.get("warehouse"));
+ }
+
+ @Test
+ public void testToPaimonJdbcCatalog() {
+ String testUser = "testUser";
+ String testPassword = "testPassword";
+ String testUri = "testUri";
+ Map<String, String> catalogProperties =
+ ImmutableMap.of(
+ PaimonConstants.WAREHOUSE,
+ localWarehouse,
+ PaimonConstants.CATALOG_BACKEND,
+ "jdbc",
+ PaimonConstants.GRAVITINO_JDBC_USER,
+ testUser,
+ PaimonConstants.GRAVITINO_JDBC_PASSWORD,
+ testPassword,
+ PropertiesConverter.FLINK_PROPERTY_PREFIX + PaimonConstants.URI,
+ testUri);
+ Map<String, String> flinkCatalogProperties =
+ CONVERTER.toFlinkCatalogProperties(catalogProperties);
+ Assertions.assertEquals(
+ GravitinoPaimonCatalogFactoryOptions.IDENTIFIER,
flinkCatalogProperties.get("type"));
+ Assertions.assertEquals(localWarehouse,
flinkCatalogProperties.get(PaimonConstants.WAREHOUSE));
+ Assertions.assertEquals(testUser,
flinkCatalogProperties.get(PaimonConstants.PAIMON_JDBC_USER));
+ Assertions.assertEquals(
+ testPassword,
flinkCatalogProperties.get(PaimonConstants.PAIMON_JDBC_PASSWORD));
+ Assertions.assertEquals("jdbc",
flinkCatalogProperties.get(PaimonConstants.METASTORE));
+ Assertions.assertEquals(testUri,
flinkCatalogProperties.get(PaimonConstants.URI));
+ }
+
+ @Test
+ public void testToGravitinoCatalogProperties() {
+ String testUser = "testUser";
+ String testPassword = "testPassword";
+ String testUri = "testUri";
+ String testBackend = "jdbc";
+ Configuration configuration =
+ Configuration.fromMap(
+ ImmutableMap.of(
+ PaimonConstants.WAREHOUSE,
+ localWarehouse,
+ PaimonConstants.METASTORE,
+ testBackend,
+ PaimonConstants.PAIMON_JDBC_USER,
+ testUser,
+ PaimonConstants.PAIMON_JDBC_PASSWORD,
+ testPassword,
+ PaimonConstants.URI,
+ testUri));
+ Map<String, String> properties =
CONVERTER.toGravitinoCatalogProperties(configuration);
+ Assertions.assertEquals(localWarehouse,
properties.get(PaimonConstants.WAREHOUSE));
+ Assertions.assertEquals(testUser,
properties.get(PaimonConstants.GRAVITINO_JDBC_USER));
+ Assertions.assertEquals(testPassword,
properties.get(PaimonConstants.GRAVITINO_JDBC_PASSWORD));
+ Assertions.assertEquals(testUri, properties.get(PaimonConstants.URI));
+ Assertions.assertEquals(testBackend,
properties.get(PaimonConstants.CATALOG_BACKEND));
+ }
+}