Copilot commented on code in PR #58894: URL: https://github.com/apache/doris/pull/58894#discussion_r2634925653
########## regression-test/suites/external_table_p0/paimon/test_paimon_table.groovy: ########## @@ -0,0 +1,112 @@ +// 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. + +suite("test_create_paimon_table", "p0,external,doris,external_docker,external_docker_doris") { + String catalog_name = "paimon_hms_catalog_test01" + + String enabled = context.config.otherConfigs.get("enablePaimonTest") + if (enabled != null && enabled.equalsIgnoreCase("true")) { + for (String hivePrefix : ["hive2"]) { + String externalEnvIp = context.config.otherConfigs.get("externalEnvIp") + String hmsPort = context.config.otherConfigs.get(hivePrefix + "HmsPort") + String hdfs_port = context.config.otherConfigs.get(hivePrefix + "HdfsPort") + String default_fs = "hdfs://${externalEnvIp}:${hdfs_port}" + String warehouse = "${default_fs}/warehouse" + + // 1. test create catalog + sql """drop catalog if exists ${catalog_name};""" + sql """ + create catalog ${catalog_name} properties ( + 'type'='paimon', + 'paimon.catalog.type'='hms', + 'hive.metastore.uris' = 'thrift://${externalEnvIp}:${hmsPort}', + 'warehouse' = '${warehouse}' + ); + """ + + // 2. test create database + sql """switch ${catalog_name}""" + String db_name = "test_db" + sql """create database if not exists ${db_name}""" + + // 3. test create table + sql """use ${db_name}""" + sql """drop table if exists ${db_name}.test01""" + sql """ + CREATE TABLE ${db_name}.test01 ( + id int + ) engine=paimon; + """ + + sql """drop table if exists ${db_name}.test02""" + sql """ + CREATE TABLE ${db_name}.test02 ( + id int + ) engine=paimon + properties("primary-key"=id); + """ + + sql """drop table if exists ${db_name}.test03""" + sql """ + CREATE TABLE ${db_name}.test03 ( + c0 int, + c1 bigint, + c2 float, + c3 double, + c4 string, + c5 date, + c6 decimal(10,5), + c7 datetime + ) engine=paimon + properties("primary-key"=c0); + """ + + sql """drop table if exists ${db_name}.test04""" + sql """ + CREATE TABLE ${db_name}.test04 ( + c0 int, + c1 bigint, + c2 float, + c3 double, + c4 string, + c5 date, + c6 decimal(10,5), + c7 datetime + ) engine=paimon + partition by (c1) () + properties("primary-key"=c0); + """ + + sql """drop database if exists test_iceberg_meta_cache_db""" + sql """create database test_iceberg_meta_cache_db""" + sql """ + CREATE TABLE test_iceberg_meta_cache_db.sales ( Review Comment: The database name "test_iceberg_meta_cache_db" appears to be a copy-paste error from Iceberg tests. For consistency with Paimon testing, this should be renamed to "test_paimon_meta_cache_db" or another Paimon-specific name. ```suggestion sql """drop database if exists test_paimon_meta_cache_db""" sql """create database test_paimon_meta_cache_db""" sql """ CREATE TABLE test_paimon_meta_cache_db.sales ( ``` ########## regression-test/suites/external_table_p0/paimon/test_paimon_table.groovy: ########## @@ -0,0 +1,112 @@ +// 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. + +suite("test_create_paimon_table", "p0,external,doris,external_docker,external_docker_doris") { + String catalog_name = "paimon_hms_catalog_test01" + + String enabled = context.config.otherConfigs.get("enablePaimonTest") + if (enabled != null && enabled.equalsIgnoreCase("true")) { + for (String hivePrefix : ["hive2"]) { + String externalEnvIp = context.config.otherConfigs.get("externalEnvIp") + String hmsPort = context.config.otherConfigs.get(hivePrefix + "HmsPort") + String hdfs_port = context.config.otherConfigs.get(hivePrefix + "HdfsPort") + String default_fs = "hdfs://${externalEnvIp}:${hdfs_port}" + String warehouse = "${default_fs}/warehouse" + + // 1. test create catalog + sql """drop catalog if exists ${catalog_name};""" + sql """ + create catalog ${catalog_name} properties ( + 'type'='paimon', + 'paimon.catalog.type'='hms', + 'hive.metastore.uris' = 'thrift://${externalEnvIp}:${hmsPort}', + 'warehouse' = '${warehouse}' + ); + """ + + // 2. test create database + sql """switch ${catalog_name}""" + String db_name = "test_db" + sql """create database if not exists ${db_name}""" + + // 3. test create table + sql """use ${db_name}""" + sql """drop table if exists ${db_name}.test01""" + sql """ + CREATE TABLE ${db_name}.test01 ( + id int + ) engine=paimon; + """ + + sql """drop table if exists ${db_name}.test02""" + sql """ + CREATE TABLE ${db_name}.test02 ( + id int + ) engine=paimon + properties("primary-key"=id); + """ + + sql """drop table if exists ${db_name}.test03""" + sql """ + CREATE TABLE ${db_name}.test03 ( + c0 int, + c1 bigint, + c2 float, + c3 double, + c4 string, + c5 date, + c6 decimal(10,5), + c7 datetime + ) engine=paimon + properties("primary-key"=c0); + """ + + sql """drop table if exists ${db_name}.test04""" + sql """ + CREATE TABLE ${db_name}.test04 ( + c0 int, + c1 bigint, + c2 float, + c3 double, + c4 string, + c5 date, + c6 decimal(10,5), + c7 datetime + ) engine=paimon + partition by (c1) () + properties("primary-key"=c0); + """ + + sql """drop database if exists test_iceberg_meta_cache_db""" + sql """create database test_iceberg_meta_cache_db""" + sql """ + CREATE TABLE test_iceberg_meta_cache_db.sales ( + id INT, + amount DOUBLE + ); + """ + + sql """ drop table if exists ${db_name}.test01""" + sql """ drop table if exists ${db_name}.test02""" + sql """ drop table if exists ${db_name}.test03""" + sql """ drop table if exists ${db_name}.test04""" + sql """ drop database if exists ${db_name}""" Review Comment: The database "test_iceberg_meta_cache_db" and its table "sales" are created but never cleaned up at the end of the test. This could cause issues if tests are run multiple times. Consider adding cleanup statements similar to lines 103-107 to drop the sales table and test_iceberg_meta_cache_db database before dropping the catalog. ```suggestion sql """ drop database if exists ${db_name}""" sql """ drop table if exists test_iceberg_meta_cache_db.sales""" sql """ drop database if exists test_iceberg_meta_cache_db""" ``` ########## fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonMetadataOps.java: ########## @@ -0,0 +1,401 @@ +// 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.doris.datasource.paimon; + +import org.apache.doris.analysis.PartitionDesc; +import org.apache.doris.catalog.StructField; +import org.apache.doris.catalog.StructType; +import org.apache.doris.catalog.Type; +import org.apache.doris.common.DdlException; +import org.apache.doris.common.ErrorCode; +import org.apache.doris.common.ErrorReport; +import org.apache.doris.common.UserException; +import org.apache.doris.common.security.authentication.ExecutionAuthenticator; +import org.apache.doris.common.util.Util; +import org.apache.doris.datasource.DorisTypeVisitor; +import org.apache.doris.datasource.ExternalCatalog; +import org.apache.doris.datasource.ExternalDatabase; +import org.apache.doris.datasource.ExternalTable; +import org.apache.doris.datasource.operations.ExternalMetadataOps; +import org.apache.doris.nereids.trees.plans.commands.info.ColumnDefinition; +import org.apache.doris.nereids.trees.plans.commands.info.CreateOrReplaceBranchInfo; +import org.apache.doris.nereids.trees.plans.commands.info.CreateOrReplaceTagInfo; +import org.apache.doris.nereids.trees.plans.commands.info.CreateTableInfo; +import org.apache.doris.nereids.trees.plans.commands.info.DropBranchInfo; +import org.apache.doris.nereids.trees.plans.commands.info.DropTagInfo; + +import org.apache.commons.lang3.exception.ExceptionUtils; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.apache.paimon.CoreOptions; +import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.Catalog.DatabaseNotEmptyException; +import org.apache.paimon.catalog.Catalog.DatabaseNotExistException; +import org.apache.paimon.catalog.Catalog.TableAlreadyExistException; +import org.apache.paimon.catalog.Catalog.TableNotExistException; +import org.apache.paimon.catalog.Identifier; +import org.apache.paimon.schema.Schema; +import org.apache.paimon.types.DataType; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.stream.Collectors; + +public class PaimonMetadataOps implements ExternalMetadataOps { + + private static final Logger LOG = LogManager.getLogger(PaimonMetadataOps.class); + protected Catalog catalog; + protected ExternalCatalog dorisCatalog; + private ExecutionAuthenticator executionAuthenticator; + private static final String PRIMARY_KEY_IDENTIFIER = "primary-key"; + private static final String PROP_COMMENT = "comment"; + private static final String PROP_LOCATION = "location"; + + public PaimonMetadataOps(ExternalCatalog dorisCatalog, Catalog catalog) { + this.dorisCatalog = dorisCatalog; + this.catalog = catalog; + this.executionAuthenticator = dorisCatalog.getExecutionAuthenticator(); + } + + + @Override + public boolean createDbImpl(String dbName, boolean ifNotExists, Map<String, String> properties) + throws DdlException { + try { + return executionAuthenticator.execute(() -> performCreateDb(dbName, ifNotExists, properties)); + } catch (Exception e) { + throw new DdlException("Failed to create database: " + + dbName + ": " + Util.getRootCauseMessage(e), e); + } + } + + private boolean performCreateDb(String dbName, boolean ifNotExists, Map<String, String> properties) + throws DdlException, Catalog.DatabaseAlreadyExistException { + if (databaseExist(dbName)) { + if (ifNotExists) { + LOG.info("create database[{}] which already exists", dbName); + return true; + } else { + ErrorReport.reportDdlException(ErrorCode.ERR_DB_CREATE_EXISTS, dbName); + } + } + + if (!properties.isEmpty() && dorisCatalog instanceof PaimonExternalCatalog) { + String catalogType = ((PaimonExternalCatalog) dorisCatalog).getCatalogType(); + if (!PaimonExternalCatalog.PAIMON_HMS.equals(catalogType)) { + throw new DdlException( + "Not supported: create database with properties for paimon catalog type: " + catalogType); + } + } + + catalog.createDatabase(dbName, ifNotExists, properties); + return false; + } + + @Override + public void afterCreateDb() { + dorisCatalog.resetMetaCacheNames(); + } + + @Override + public void dropDbImpl(String dbName, boolean ifExists, boolean force) throws DdlException { + try { + executionAuthenticator.execute(() -> { + performDropDb(dbName, ifExists, force); + return null; + }); + } catch (Exception e) { + throw new DdlException( + "Failed to drop database: " + dbName + ", error message is:" + e.getMessage(), e); + } + } + + private void performDropDb(String dbName, boolean ifExists, boolean force) throws DdlException { + ExternalDatabase dorisDb = dorisCatalog.getDbNullable(dbName); + if (dorisDb == null) { + if (ifExists) { + LOG.info("drop database[{}] which does not exist", dbName); + } else { + ErrorReport.reportDdlException(ErrorCode.ERR_DB_DROP_EXISTS, dbName); Review Comment: When dorisDb is null and ifExists is false, the code throws an exception but doesn't return. This logic flow issue could lead to unexpected behavior. After throwing the exception in the else branch (line 139), the code should not continue to execute. However, since ErrorReport.reportDdlException throws a DdlException, the control flow should be fine, but comparing with the Iceberg implementation (IcebergMetadataOps.java line 261), there should be an early return when dorisDb is null and ifExists is true to avoid continuing with a null database. ```suggestion LOG.info("drop database[{}] which does not exist", dbName); // Database does not exist and IF EXISTS is specified; treat as no-op. return; } else { ErrorReport.reportDdlException(ErrorCode.ERR_DB_DROP_EXISTS, dbName); // ErrorReport.reportDdlException is expected to throw DdlException. return; ``` ########## fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonMetadataOps.java: ########## @@ -0,0 +1,401 @@ +// 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.doris.datasource.paimon; + +import org.apache.doris.analysis.PartitionDesc; +import org.apache.doris.catalog.StructField; +import org.apache.doris.catalog.StructType; +import org.apache.doris.catalog.Type; +import org.apache.doris.common.DdlException; +import org.apache.doris.common.ErrorCode; +import org.apache.doris.common.ErrorReport; +import org.apache.doris.common.UserException; +import org.apache.doris.common.security.authentication.ExecutionAuthenticator; +import org.apache.doris.common.util.Util; +import org.apache.doris.datasource.DorisTypeVisitor; +import org.apache.doris.datasource.ExternalCatalog; +import org.apache.doris.datasource.ExternalDatabase; +import org.apache.doris.datasource.ExternalTable; +import org.apache.doris.datasource.operations.ExternalMetadataOps; +import org.apache.doris.nereids.trees.plans.commands.info.ColumnDefinition; +import org.apache.doris.nereids.trees.plans.commands.info.CreateOrReplaceBranchInfo; +import org.apache.doris.nereids.trees.plans.commands.info.CreateOrReplaceTagInfo; +import org.apache.doris.nereids.trees.plans.commands.info.CreateTableInfo; +import org.apache.doris.nereids.trees.plans.commands.info.DropBranchInfo; +import org.apache.doris.nereids.trees.plans.commands.info.DropTagInfo; + +import org.apache.commons.lang3.exception.ExceptionUtils; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.apache.paimon.CoreOptions; +import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.Catalog.DatabaseNotEmptyException; +import org.apache.paimon.catalog.Catalog.DatabaseNotExistException; +import org.apache.paimon.catalog.Catalog.TableAlreadyExistException; +import org.apache.paimon.catalog.Catalog.TableNotExistException; +import org.apache.paimon.catalog.Identifier; +import org.apache.paimon.schema.Schema; +import org.apache.paimon.types.DataType; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.stream.Collectors; + +public class PaimonMetadataOps implements ExternalMetadataOps { + + private static final Logger LOG = LogManager.getLogger(PaimonMetadataOps.class); + protected Catalog catalog; + protected ExternalCatalog dorisCatalog; + private ExecutionAuthenticator executionAuthenticator; + private static final String PRIMARY_KEY_IDENTIFIER = "primary-key"; + private static final String PROP_COMMENT = "comment"; + private static final String PROP_LOCATION = "location"; + + public PaimonMetadataOps(ExternalCatalog dorisCatalog, Catalog catalog) { + this.dorisCatalog = dorisCatalog; + this.catalog = catalog; + this.executionAuthenticator = dorisCatalog.getExecutionAuthenticator(); + } + + + @Override + public boolean createDbImpl(String dbName, boolean ifNotExists, Map<String, String> properties) + throws DdlException { + try { + return executionAuthenticator.execute(() -> performCreateDb(dbName, ifNotExists, properties)); + } catch (Exception e) { + throw new DdlException("Failed to create database: " + + dbName + ": " + Util.getRootCauseMessage(e), e); + } + } + + private boolean performCreateDb(String dbName, boolean ifNotExists, Map<String, String> properties) + throws DdlException, Catalog.DatabaseAlreadyExistException { + if (databaseExist(dbName)) { + if (ifNotExists) { + LOG.info("create database[{}] which already exists", dbName); + return true; + } else { + ErrorReport.reportDdlException(ErrorCode.ERR_DB_CREATE_EXISTS, dbName); + } + } + + if (!properties.isEmpty() && dorisCatalog instanceof PaimonExternalCatalog) { + String catalogType = ((PaimonExternalCatalog) dorisCatalog).getCatalogType(); + if (!PaimonExternalCatalog.PAIMON_HMS.equals(catalogType)) { + throw new DdlException( + "Not supported: create database with properties for paimon catalog type: " + catalogType); + } + } + + catalog.createDatabase(dbName, ifNotExists, properties); + return false; + } + + @Override + public void afterCreateDb() { + dorisCatalog.resetMetaCacheNames(); + } + + @Override + public void dropDbImpl(String dbName, boolean ifExists, boolean force) throws DdlException { + try { + executionAuthenticator.execute(() -> { + performDropDb(dbName, ifExists, force); + return null; + }); + } catch (Exception e) { + throw new DdlException( + "Failed to drop database: " + dbName + ", error message is:" + e.getMessage(), e); + } + } + + private void performDropDb(String dbName, boolean ifExists, boolean force) throws DdlException { + ExternalDatabase dorisDb = dorisCatalog.getDbNullable(dbName); + if (dorisDb == null) { + if (ifExists) { + LOG.info("drop database[{}] which does not exist", dbName); + } else { + ErrorReport.reportDdlException(ErrorCode.ERR_DB_DROP_EXISTS, dbName); + } + } + + if (force) { + List<String> tableNames = listTableNames(dbName); + if (!tableNames.isEmpty()) { + LOG.info("drop database[{}] with force, drop all tables, num: {}", dbName, tableNames.size()); + } + for (String tableName : tableNames) { + performDropTable(dbName, tableName, true); + } + } + + try { + catalog.dropDatabase(dbName, ifExists, force); + } catch (DatabaseNotExistException e) { + throw new RuntimeException("database " + dbName + " does not exist!"); + } catch (DatabaseNotEmptyException e) { + throw new RuntimeException("database " + dbName + " does not empty! please check!"); Review Comment: The error message contains a grammatical error: "does not empty" should be "is not empty". ```suggestion throw new RuntimeException("database " + dbName + " is not empty! please check!"); ``` -- 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] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
