luoyuxia commented on code in PR #1638:
URL: https://github.com/apache/fluss/pull/1638#discussion_r2321864658
##########
fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/catalog/FlinkCatalog.java:
##########
@@ -380,6 +380,8 @@ public void createTable(ObjectPath objectPath,
CatalogBaseTable table, boolean i
throw new DatabaseNotExistException(getName(),
objectPath.getDatabaseName());
} else if (CatalogExceptionUtils.isTableAlreadyExist(t)) {
throw new TableAlreadyExistException(getName(), objectPath);
+ } else if (CatalogExceptionUtils.isLakeTableAlreadyExist(t)) {
+ throw new TableAlreadyExistException("lakeCatalog",
objectPath, t);
Review Comment:
I'm wondering can we use `CatalogException` to show the detail error message
of
```
The table xx already exists in xx catalog, please "
+ "first drop the table in xx
catalog or use a new table name
```
since it's more clear to me
##########
fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/catalog/LakeEnabledFlinkCatalogTest.java:
##########
@@ -0,0 +1,161 @@
+/*
+ * 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.fluss.flink.catalog;
+
+import org.apache.fluss.config.ConfigOptions;
+import org.apache.fluss.config.Configuration;
+import org.apache.fluss.metadata.DataLakeFormat;
+import org.apache.fluss.server.testutils.FlussClusterExtension;
+import org.apache.fluss.utils.ExceptionUtils;
+
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.Schema;
+import org.apache.flink.table.catalog.Catalog;
+import org.apache.flink.table.catalog.CatalogDatabaseImpl;
+import org.apache.flink.table.catalog.CatalogTable;
+import org.apache.flink.table.catalog.Column;
+import org.apache.flink.table.catalog.ObjectPath;
+import org.apache.flink.table.catalog.ResolvedCatalogTable;
+import org.apache.flink.table.catalog.ResolvedSchema;
+import org.apache.flink.table.catalog.UniqueConstraint;
+import org.apache.flink.table.catalog.exceptions.CatalogException;
+import org.apache.flink.table.catalog.exceptions.TableAlreadyExistException;
+import org.apache.flink.table.catalog.exceptions.TableNotExistException;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.RegisterExtension;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.apache.fluss.config.ConfigOptions.BOOTSTRAP_SERVERS;
+import static org.apache.fluss.config.ConfigOptions.TABLE_DATALAKE_ENABLED;
+import static org.apache.fluss.config.ConfigOptions.TABLE_DATALAKE_FORMAT;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+/** Test enable lake for {@link FlinkCatalog}. */
+public class LakeEnabledFlinkCatalogTest {
+ @RegisterExtension
+ public static final FlussClusterExtension FLUSS_CLUSTER_EXTENSION =
+ FlussClusterExtension.builder()
+ .setClusterConf(initConfig())
+ .setNumOfTabletServers(1)
+ .build();
+
+ private static final String CATALOG_NAME = "test-catalog";
+ private static final String DEFAULT_DB = "default";
+ static Catalog catalog;
+ private final ObjectPath tableInDefaultDb = new ObjectPath(DEFAULT_DB,
"t1");
+
+ @BeforeAll
+ static void beforeAll() {
+ // set fluss conf
+ Configuration flussConf = FLUSS_CLUSTER_EXTENSION.getClientConfig();
+ catalog =
+ new FlinkCatalog(
+ CATALOG_NAME,
+ DEFAULT_DB,
+ String.join(",", flussConf.get(BOOTSTRAP_SERVERS)),
+ Thread.currentThread().getContextClassLoader(),
+ Collections.emptyMap());
+ catalog.open();
+ }
+
+ @AfterAll
+ static void afterAll() {
+ if (catalog != null) {
+ catalog.close();
+ }
+ }
+
+ @BeforeEach
+ void beforeEach() throws Exception {
+ // First check if database exists, and drop it if it does
+ if (catalog.databaseExists(DEFAULT_DB)) {
+ catalog.dropDatabase(DEFAULT_DB, true, true);
+ }
+ try {
+ catalog.createDatabase(
+ DEFAULT_DB, new
CatalogDatabaseImpl(Collections.emptyMap(), null), true);
+ } catch (CatalogException e) {
+ // the auto partitioned manager may create the db zk node
+ // in an another thread, so if exception is NodeExistsException,
just ignore
+ if (!ExceptionUtils.findThrowableWithMessage(e,
"KeeperException$NodeExistsException")
+ .isPresent()) {
+ throw e;
+ }
+ }
+ }
+
+ private static Configuration initConfig() {
+ Configuration configuration = new Configuration();
+ configuration.set(ConfigOptions.DATALAKE_FORMAT,
DataLakeFormat.PAIMON);
+ configuration.set(ConfigOptions.DEFAULT_BUCKET_NUMBER, 3);
+ return configuration;
+ }
+
+ private CatalogTable newCatalogTable(Map<String, String> options) {
+ ResolvedSchema resolvedSchema =
+ new ResolvedSchema(
+ Arrays.asList(
+ Column.physical("first",
DataTypes.STRING().notNull()),
+ Column.physical("second", DataTypes.INT()),
+ Column.physical("third",
DataTypes.STRING().notNull())),
+ Collections.emptyList(),
+ UniqueConstraint.primaryKey(
+ "PK_first_third", Arrays.asList("first",
"third")));
+ CatalogTable origin =
+ CatalogTable.of(
+
Schema.newBuilder().fromResolvedSchema(resolvedSchema).build(),
+ "test comment",
+ Collections.emptyList(),
+ options);
+ return new ResolvedCatalogTable(origin, resolvedSchema);
+ }
+
+ @Test
+ void testCreateLakeTable() throws Exception {
Review Comment:
Can we move the test case to `FlinkCatalogTest`? We can enable datalake for
the FlussCluster used in `FlinkCatalogTest`
##########
fluss-rpc/src/main/java/org/apache/fluss/rpc/protocol/Errors.java:
##########
@@ -214,7 +215,9 @@ public enum Errors {
INVALID_SERVER_RACK_INFO_EXCEPTION(
52, "The server rack info is invalid.",
InvalidServerRackInfoException::new),
LAKE_SNAPSHOT_NOT_EXIST(
- 53, "The lake snapshot is not exist.",
LakeTableSnapshotNotExistException::new);
+ 53, "The lake snapshot is not exist.",
LakeTableSnapshotNotExistException::new),
+ LAKE_TABLE_ALREADY_EXIST(
+ 54, "The lake table is already exist.",
LakeTableAlreadyExistException::new);
Review Comment:
```suggestion
54, "The lake table already exists.",
LakeTableAlreadyExistException::new);
```
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]