diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java b/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java index 8d9ec930e3ea6b..c4fbf2875215f6 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java @@ -390,19 +390,19 @@ private void processAlterTableForExternalTable( setExternalTableAutoAnalyzePolicy(table, alterClauses); } else if (alterClause instanceof CreateOrReplaceBranchClause) { table.getCatalog().createOrReplaceBranch( - table.getDbName(), table.getName(), + table, ((CreateOrReplaceBranchClause) alterClause).getBranchInfo()); } else if (alterClause instanceof CreateOrReplaceTagClause) { table.getCatalog().createOrReplaceTag( - table.getDbName(), table.getName(), + table, ((CreateOrReplaceTagClause) alterClause).getTagInfo()); } else if (alterClause instanceof DropBranchClause) { table.getCatalog().dropBranch( - table.getDbName(), table.getName(), + table, ((DropBranchClause) alterClause).getDropBranchInfo()); } else if (alterClause instanceof DropTagClause) { table.getCatalog().dropTag( - table.getDbName(), table.getName(), + table, ((DropTagClause) alterClause).getDropTagInfo()); } else { throw new UserException("Invalid alter operations for external table: " + alterClauses); diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Database.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Database.java index 3a17b755278582..d89b4cf24eef81 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Database.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Database.java @@ -456,6 +456,7 @@ public boolean registerTable(TableIf table) { return result; } + @Override public void unregisterTable(String tableName) { if (Env.isStoredTableNamesLowerCase()) { tableName = tableName.toLowerCase(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java index c5fe5a9e0b3149..7d28e9e4fbc169 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java @@ -202,6 +202,7 @@ import org.apache.doris.nereids.trees.plans.commands.UninstallPluginCommand; import org.apache.doris.nereids.trees.plans.commands.info.AlterMTMVPropertyInfo; import org.apache.doris.nereids.trees.plans.commands.info.AlterMTMVRefreshInfo; +import org.apache.doris.nereids.trees.plans.commands.info.PartitionNamesInfo; import org.apache.doris.nereids.trees.plans.commands.info.TableNameInfo; import org.apache.doris.persist.AlterMTMV; import org.apache.doris.persist.AutoIncrementIdUpdateLog; @@ -3334,7 +3335,7 @@ public void createDb(CreateDbStmt stmt) throws DdlException { } else { catalogIf = catalogMgr.getCatalog(stmt.getCtlName()); } - catalogIf.createDb(stmt); + catalogIf.createDb(stmt.getFullDbName(), stmt.isSetIfNotExists(), stmt.getProperties()); } // The interface which DdlExecutor needs. @@ -3345,7 +3346,7 @@ public void createDb(CreateDatabaseCommand command) throws DdlException { } else { catalogIf = catalogMgr.getCatalog(command.getCtlName()); } - catalogIf.createDb(command); + catalogIf.createDb(command.getDbName(), command.isIfNotExists(), command.getProperties()); } // For replay edit log, need't lock metadata @@ -6068,7 +6069,11 @@ public String dumpImage() { public void truncateTable(TruncateTableCommand command) throws DdlException { CatalogIf catalogIf = catalogMgr.getCatalogOrException(command.getTableNameInfo().getCtl(), catalog -> new DdlException(("Unknown catalog " + catalog))); - catalogIf.truncateTable(command); + TableNameInfo nameInfo = command.getTableNameInfo(); + PartitionNamesInfo partitionNamesInfo = command.getPartitionNamesInfo().orElse(null); + catalogIf.truncateTable(nameInfo.getDb(), nameInfo.getTbl(), + partitionNamesInfo == null ? null : partitionNamesInfo.translateToLegacyPartitionNames(), + command.isForceDrop(), command.toSqlWithoutTable()); } public void replayTruncateTable(TruncateTableInfo info) throws MetaNotFoundException { diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/InternalSchemaInitializer.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/InternalSchemaInitializer.java index 1444a4febaa009..7a1e18bd839b2a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/InternalSchemaInitializer.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/InternalSchemaInitializer.java @@ -25,7 +25,6 @@ import org.apache.doris.analysis.CreateTableStmt; import org.apache.doris.analysis.DbName; import org.apache.doris.analysis.DistributionDesc; -import org.apache.doris.analysis.DropTableStmt; import org.apache.doris.analysis.HashDistributionDesc; import org.apache.doris.analysis.KeysDesc; import org.apache.doris.analysis.ModifyColumnClause; @@ -370,8 +369,8 @@ private boolean created() { if (!optionalColumn.isPresent() || !optionalColumn.get().isAllowNull()) { try { Env.getCurrentEnv().getInternalCatalog() - .dropTable(new DropTableStmt(true, new TableName(null, - StatisticConstants.DB_NAME, StatisticConstants.TABLE_STATISTIC_TBL_NAME), true)); + .dropTable(StatisticConstants.DB_NAME, StatisticConstants.TABLE_STATISTIC_TBL_NAME, + false, false, true, true); } catch (Exception e) { LOG.warn("Failed to drop outdated table", e); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/RefreshManager.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/RefreshManager.java index e421cc1475c03d..bc1f8576ca5446 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/RefreshManager.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/RefreshManager.java @@ -203,12 +203,11 @@ public void refreshExternalTableFromEvent(String catalogName, String dbName, Str refreshTableInternal(catalog, db, table, updateTime); } - private void refreshTableInternal(CatalogIf catalog, DatabaseIf db, TableIf table, long updateTime) { + public void refreshTableInternal(CatalogIf catalog, DatabaseIf db, TableIf table, long updateTime) { if (table instanceof ExternalTable) { ((ExternalTable) table).unsetObjectCreated(); } - Env.getCurrentEnv().getExtMetaCacheMgr() - .invalidateTableCache(catalog.getId(), db.getFullName(), table.getName()); + Env.getCurrentEnv().getExtMetaCacheMgr().invalidateTableCache((ExternalTable) table); if (table instanceof HMSExternalTable && updateTime > 0) { ((HMSExternalTable) table).setEventUpdateTime(updateTime); } @@ -245,8 +244,7 @@ public void refreshPartitions(String catalogName, String dbName, String tableNam return; } - Env.getCurrentEnv().getExtMetaCacheMgr().invalidatePartitionsCache( - catalog.getId(), db.getFullName(), table.getName(), partitionNames); + Env.getCurrentEnv().getExtMetaCacheMgr().invalidatePartitionsCache((ExternalTable) table, partitionNames); ((HMSExternalTable) table).setEventUpdateTime(updateTime); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/info/SimpleTableInfo.java b/fe/fe-core/src/main/java/org/apache/doris/common/info/SimpleTableInfo.java deleted file mode 100644 index 6fdb27e1d0b9df..00000000000000 --- a/fe/fe-core/src/main/java/org/apache/doris/common/info/SimpleTableInfo.java +++ /dev/null @@ -1,66 +0,0 @@ -// 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. -// This file is copied from -// https://github.com/apache/impala/blob/branch-2.9.0/fe/src/main/java/org/apache/impala/AnalysisException.java -// and modified by Doris - -package org.apache.doris.common.info; - -import java.util.Objects; - -public class SimpleTableInfo { - - private final String dbName; - private final String tbName; - - public SimpleTableInfo(String dbName, String tbName) { - this.dbName = dbName; - this.tbName = tbName; - } - - public String getDbName() { - return dbName; - } - - public String getTbName() { - return tbName; - } - - @Override - public int hashCode() { - return Objects.hash(dbName, tbName); - } - - @Override - public boolean equals(Object other) { - if (this == other) { - return true; - } - - if (other == null || getClass() != other.getClass()) { - return false; - } - - SimpleTableInfo that = (SimpleTableInfo) other; - return Objects.equals(dbName, that.dbName) && Objects.equals(tbName, that.tbName); - } - - @Override - public String toString() { - return String.format("%s.%s", dbName, tbName); - } -} diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/util/Util.java b/fe/fe-core/src/main/java/org/apache/doris/common/util/Util.java index ec27d923f1edac..c5a4c90b64c7b5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/common/util/Util.java +++ b/fe/fe-core/src/main/java/org/apache/doris/common/util/Util.java @@ -698,6 +698,16 @@ public static String getRootCauseStack(Throwable t) { return sw.toString(); } + public static Throwable getRootCause(Throwable t) { + Throwable p = t; + Throwable r = t; + while (p != null) { + r = p; + p = p.getCause(); + } + return r; + } + public static long sha256long(String str) { try { MessageDigest digest = MessageDigest.getInstance("SHA-256"); diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/CatalogIf.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/CatalogIf.java index d120b9008ef345..895c0331eaed55 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/CatalogIf.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/CatalogIf.java @@ -17,10 +17,8 @@ package org.apache.doris.datasource; -import org.apache.doris.analysis.CreateDbStmt; import org.apache.doris.analysis.CreateTableStmt; -import org.apache.doris.analysis.DropDbStmt; -import org.apache.doris.analysis.DropTableStmt; +import org.apache.doris.analysis.PartitionNames; import org.apache.doris.analysis.TableName; import org.apache.doris.catalog.DatabaseIf; import org.apache.doris.catalog.Env; @@ -30,8 +28,6 @@ import org.apache.doris.common.ErrorCode; import org.apache.doris.common.MetaNotFoundException; import org.apache.doris.common.UserException; -import org.apache.doris.nereids.trees.plans.commands.CreateDatabaseCommand; -import org.apache.doris.nereids.trees.plans.commands.TruncateTableCommand; 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.DropBranchInfo; @@ -189,13 +185,7 @@ default CatalogLog constructEditLog() { boolean enableAutoAnalyze(); - void createDb(CreateDbStmt stmt) throws DdlException; - - void createDb(CreateDatabaseCommand command) throws DdlException; - - default void dropDb(DropDbStmt stmt) throws DdlException { - dropDb(stmt.getDbName(), stmt.isSetIfExists(), stmt.isForceDrop()); - } + void createDb(String dbName, boolean ifNotExists, Map properties) throws DdlException; void dropDb(String dbName, boolean ifExists, boolean force) throws DdlException; @@ -205,12 +195,12 @@ default void dropDb(DropDbStmt stmt) throws DdlException { */ boolean createTable(CreateTableStmt stmt) throws UserException; - void dropTable(DropTableStmt stmt) throws DdlException; - void dropTable(String dbName, String tableName, boolean isView, boolean isMtmv, boolean ifExists, boolean force) throws DdlException; - void truncateTable(TruncateTableCommand truncateTableCommand) throws DdlException; + void truncateTable(String dbName, String tableName, PartitionNames partitionNames, boolean forceDrop, + String rawTruncateSql) + throws DdlException; // Convert from remote database name to local database name, overridden by subclass if necessary default String fromRemoteDatabaseName(String remoteDatabaseName) { @@ -223,13 +213,13 @@ default String fromRemoteTableName(String remoteDatabaseName, String remoteTable } // Create or replace branch operations, overridden by subclass if necessary - default void createOrReplaceBranch(String db, String tbl, CreateOrReplaceBranchInfo branchInfo) + default void createOrReplaceBranch(TableIf dorisTable, CreateOrReplaceBranchInfo branchInfo) throws UserException { throw new UserException("Not support create or replace branch operation"); } // Create or replace tag operation, overridden by subclass if necessary - default void createOrReplaceTag(String db, String tbl, CreateOrReplaceTagInfo tagInfo) throws UserException { + default void createOrReplaceTag(TableIf dorisTable, CreateOrReplaceTagInfo tagInfo) throws UserException { throw new UserException("Not support create or replace tag operation"); } @@ -237,11 +227,11 @@ default void replayOperateOnBranchOrTag(String dbName, String tblName) { } - default void dropBranch(String db, String tbl, DropBranchInfo branchInfo) throws UserException { + default void dropBranch(TableIf dorisTable, DropBranchInfo branchInfo) throws UserException { throw new UserException("Not support drop branch operation"); } - default void dropTag(String db, String tbl, DropTagInfo tagInfo) throws UserException { + default void dropTag(TableIf dorisTable, DropTagInfo tagInfo) throws UserException { throw new UserException("Not support drop tag operation"); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalCatalog.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalCatalog.java index e4f735e69629be..c7ad269844b316 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalCatalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalCatalog.java @@ -17,9 +17,8 @@ package org.apache.doris.datasource; -import org.apache.doris.analysis.CreateDbStmt; import org.apache.doris.analysis.CreateTableStmt; -import org.apache.doris.analysis.DropTableStmt; +import org.apache.doris.analysis.PartitionNames; import org.apache.doris.analysis.TableName; import org.apache.doris.catalog.DatabaseIf; import org.apache.doris.catalog.Env; @@ -55,8 +54,6 @@ import org.apache.doris.datasource.test.TestExternalDatabase; import org.apache.doris.datasource.trinoconnector.TrinoConnectorExternalDatabase; import org.apache.doris.fs.remote.dfs.DFSFileSystem; -import org.apache.doris.nereids.trees.plans.commands.CreateDatabaseCommand; -import org.apache.doris.nereids.trees.plans.commands.TruncateTableCommand; 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.DropBranchInfo; @@ -311,18 +308,7 @@ public final synchronized void makeSureInitialized() { initLocalObjects(); if (!initialized) { if (useMetaCache.get()) { - if (metaCache == null) { - metaCache = Env.getCurrentEnv().getExtMetaCacheMgr().buildMetaCache( - name, - OptionalLong.of(Config.external_cache_expire_time_seconds_after_access), - OptionalLong.of(Config.external_cache_refresh_time_minutes * 60L), - Config.max_meta_object_cache_num, - ignored -> getFilteredDatabaseNames(), - localDbName -> Optional.ofNullable( - buildDbForInit(null, localDbName, Util.genIdByName(name, localDbName), logType, - true)), - (key, value, cause) -> value.ifPresent(v -> v.setUnInitialized(invalidCacheInInit))); - } + buildMetaCache(); setLastUpdateTime(System.currentTimeMillis()); } else { if (!Env.getCurrentEnv().isMaster()) { @@ -357,6 +343,21 @@ public boolean isInitialized() { return this.initialized; } + private void buildMetaCache() { + if (metaCache == null) { + metaCache = Env.getCurrentEnv().getExtMetaCacheMgr().buildMetaCache( + name, + OptionalLong.of(Config.external_cache_expire_time_seconds_after_access), + OptionalLong.of(Config.external_cache_refresh_time_minutes * 60L), + Config.max_meta_object_cache_num, + ignored -> getFilteredDatabaseNames(), + localDbName -> Optional.ofNullable( + buildDbForInit(null, localDbName, Util.genIdByName(name, localDbName), logType, + true)), + (key, value, cause) -> value.ifPresent(v -> v.setUnInitialized(invalidCacheInInit))); + } + } + // check if all required properties are set when creating catalog public void checkProperties() throws DdlException { // check refresh parameter of catalog @@ -591,9 +592,9 @@ private void refreshOnlyCatalogCache(boolean invalidCache) { public final Optional getSchema(SchemaCacheKey key) { makeSureInitialized(); - Optional> db = getDb(key.getDbName()); + Optional> db = getDb(key.getNameMapping().getLocalDbName()); if (db.isPresent()) { - Optional table = db.get().getTable(key.getTblName()); + Optional table = db.get().getTable(key.getNameMapping().getLocalTblName()); if (table.isPresent()) { return table.get().initSchemaAndUpdateTime(key); } @@ -848,7 +849,18 @@ public void replayInitCatalog(InitCatalogLog log) { initialized = true; } + /** + * This method will try getting db from cache only, + * If there is no cache, it will return empty. + * Different from "getDbNullable()", this method will not visit the remote catalog to get db when it does not exist + * in cache. + * This is used for replaying the metadata, to avoid exception when trying to get db from remote catalog. + * + * @param dbId + * @return + */ public Optional> getDbForReplay(long dbId) { + Preconditions.checkState(useMetaCache.isPresent(), name); if (useMetaCache.get()) { if (!isInitialized()) { return Optional.empty(); @@ -859,6 +871,26 @@ public Optional> getDbForReplay(long d } } + /** + * Same as "getDbForReplay(long dbId)", use "tryGetMetaObj" to get db from cache only. + * + * @param dbName + * @return + */ + public Optional> getDbForReplay(String dbName) { + Preconditions.checkState(useMetaCache.isPresent(), name); + if (useMetaCache.get()) { + if (!isInitialized()) { + return Optional.empty(); + } + return metaCache.tryGetMetaObj(dbName); + } else if (dbNameToId.containsKey(dbName)) { + return Optional.ofNullable(idToDb.get(dbNameToId.get(dbName))); + } else { + return Optional.empty(); + } + } + /** * Build a database instance. * If checkExists is true, it will check if the database exists in the remote system. @@ -991,8 +1023,12 @@ public void gsonPostProcess() throws IOException { } public void addDatabaseForTest(ExternalDatabase db) { + // 1. add for "use_meta_cache = false" idToDb.put(db.getId(), db); dbNameToId.put(ClusterNamespace.getNameFromFullName(db.getFullName()), db.getId()); + // 2. add for "use_meta_cache = true" + buildMetaCache(); + metaCache.addObjForTest(db.getId(), db.getFullName(), db); } /** @@ -1001,43 +1037,34 @@ public void addDatabaseForTest(ExternalDatabase db) { */ public void setInitializedForTest(boolean initialized) { this.initialized = initialized; - } - - @Override - public void createDb(CreateDbStmt stmt) throws DdlException { - makeSureInitialized(); - if (metadataOps == null) { - throw new DdlException("Create database is not supported for catalog: " + getName()); - } - try { - metadataOps.createDb(stmt); - CreateDbInfo info = new CreateDbInfo(getName(), stmt.getFullDbName(), null); - Env.getCurrentEnv().getEditLog().logCreateDb(info); - } catch (Exception e) { - LOG.warn("Failed to create database {} in catalog {}.", stmt.getFullDbName(), getName(), e); - throw e; + if (this.initialized) { + buildMetaCache(); + this.useMetaCache = Optional.of(true); } } @Override - public void createDb(CreateDatabaseCommand command) throws DdlException { + public void createDb(String dbName, boolean ifNotExists, Map properties) throws DdlException { makeSureInitialized(); if (metadataOps == null) { throw new DdlException("Create database is not supported for catalog: " + getName()); } try { - metadataOps.createDb(command); - CreateDbInfo info = new CreateDbInfo(getName(), command.getDbName(), null); - Env.getCurrentEnv().getEditLog().logCreateDb(info); + boolean res = metadataOps.createDb(dbName, ifNotExists, properties); + if (!res) { + // we should get the db stored in Doris, and use local name in edit log. + CreateDbInfo info = new CreateDbInfo(getName(), dbName, null); + Env.getCurrentEnv().getEditLog().logCreateDb(info); + } } catch (Exception e) { - LOG.warn("Failed to create database {} in catalog {}.", command.getDbName(), getName(), e); + LOG.warn("Failed to create database {} in catalog {}.", dbName, getName(), e); throw e; } } public void replayCreateDb(String dbName) { if (metadataOps != null) { - metadataOps.afterCreateDb(dbName); + metadataOps.afterCreateDb(); } } @@ -1048,7 +1075,7 @@ public void dropDb(String dbName, boolean ifExists, boolean force) throws DdlExc throw new DdlException("Drop database is not supported for catalog: " + getName()); } try { - metadataOps.dropDb(getName(), dbName, ifExists, force); + metadataOps.dropDb(dbName, ifExists, force); DropDbInfo info = new DropDbInfo(getName(), dbName); Env.getCurrentEnv().getEditLog().logDropDb(info); } catch (Exception e) { @@ -1073,6 +1100,7 @@ public boolean createTable(CreateTableStmt stmt) throws UserException { boolean res = metadataOps.createTable(stmt); if (!res) { // res == false means the table does not exist before, and we create it. + // we should get the table stored in Doris, and use local name in edit log. CreateTableInfo info = new CreateTableInfo(getName(), stmt.getDbName(), stmt.getTableName()); Env.getCurrentEnv().getEditLog().logCreateTable(info); } @@ -1089,15 +1117,6 @@ public void replayCreateTable(String dbName, String tblName) { } } - @Override - public void dropTable(DropTableStmt stmt) throws DdlException { - if (stmt == null) { - throw new DdlException("DropTableStmt is null"); - } - dropTable(stmt.getDbName(), stmt.getTableName(), stmt.isView(), stmt.isMaterializedView(), stmt.isSetIfExists(), - stmt.isForceDrop()); - } - @Override public void dropTable(String dbName, String tableName, boolean isView, boolean isMtmv, boolean ifExists, boolean force) throws DdlException { @@ -1105,8 +1124,20 @@ public void dropTable(String dbName, String tableName, boolean isView, boolean i if (metadataOps == null) { throw new DdlException("Drop table is not supported for catalog: " + getName()); } + // 1. get table in doris catalog first. + ExternalDatabase db = getDbNullable(dbName); + if (db == null) { + throw new DdlException("Failed to get database: '" + dbName + "' in catalog: " + getName()); + } + ExternalTable dorisTable = db.getTableNullable(tableName); + if (dorisTable == null) { + if (ifExists) { + return; + } + throw new DdlException("Failed to get table: '" + tableName + "' in database: " + dbName); + } try { - metadataOps.dropTable(dbName, tableName, ifExists); + metadataOps.dropTable(dorisTable, ifExists); DropInfo info = new DropInfo(getName(), dbName, tableName); Env.getCurrentEnv().getEditLog().logDropTable(info); } catch (Exception e) { @@ -1205,27 +1236,24 @@ public boolean enableAutoAnalyze() { } @Override - public void truncateTable(TruncateTableCommand command) throws DdlException { + public void truncateTable(String dbName, String tableName, PartitionNames partitionNames, boolean forceDrop, + String rawTruncateSql) throws DdlException { makeSureInitialized(); if (metadataOps == null) { throw new DdlException("Truncate table is not supported for catalog: " + getName()); } try { - String db = command.getTableNameInfo().getDb(); - String tbl = command.getTableNameInfo().getTbl(); - // delete all table data if null List partitions = null; - if (command.getPartitionNamesInfo().isPresent()) { - partitions = command.getPartitionNamesInfo().get().getPartitionNames(); + if (partitionNames != null) { + partitions = partitionNames.getPartitionNames(); } - - metadataOps.truncateTable(db, tbl, partitions); - TruncateTableInfo info = new TruncateTableInfo(getName(), db, tbl, partitions); + ExternalTable dorisTable = getDbOrDdlException(dbName).getTableOrDdlException(tableName); + metadataOps.truncateTable(dorisTable, partitions); + TruncateTableInfo info = new TruncateTableInfo(getName(), dbName, tableName, partitions); Env.getCurrentEnv().getEditLog().logTruncateTable(info); } catch (Exception e) { - LOG.warn("Failed to truncate table {}.{} in catalog {}", command.getTableNameInfo().getDb(), - command.getTableNameInfo().getTbl(), getName(), e); + LOG.warn("Failed to truncate table {}.{} in catalog {}", dbName, tableName, getName(), e); throw e; } } @@ -1313,37 +1341,43 @@ public boolean viewExists(String dbName, String viewName) { } @Override - public void createOrReplaceBranch(String db, String tbl, CreateOrReplaceBranchInfo branchInfo) + public void createOrReplaceBranch(TableIf dorisTable, CreateOrReplaceBranchInfo branchInfo) throws UserException { makeSureInitialized(); + Preconditions.checkState(dorisTable instanceof ExternalTable, dorisTable.getName()); + ExternalTable externalTable = (ExternalTable) dorisTable; if (metadataOps == null) { throw new DdlException("branching operation is not supported for catalog: " + getName()); } try { - metadataOps.createOrReplaceBranch(db, tbl, branchInfo); - TableBranchOrTagInfo info = new TableBranchOrTagInfo(getName(), db, tbl); + metadataOps.createOrReplaceBranch(externalTable, branchInfo); + TableBranchOrTagInfo info = new TableBranchOrTagInfo(getName(), externalTable.getDbName(), + externalTable.getName()); Env.getCurrentEnv().getEditLog().logBranchOrTag(info); } catch (Exception e) { LOG.warn("Failed to create or replace branch for table {}.{} in catalog {}", - db, tbl, getName(), e); + externalTable.getDbName(), externalTable.getName(), getName(), e); throw e; } } @Override - public void createOrReplaceTag(String db, String tbl, CreateOrReplaceTagInfo tagInfo) + public void createOrReplaceTag(TableIf dorisTable, CreateOrReplaceTagInfo tagInfo) throws UserException { makeSureInitialized(); + Preconditions.checkState(dorisTable instanceof ExternalTable, dorisTable.getName()); + ExternalTable externalTable = (ExternalTable) dorisTable; if (metadataOps == null) { throw new DdlException("Tagging operation is not supported for catalog: " + getName()); } try { - metadataOps.createOrReplaceTag(db, tbl, tagInfo); - TableBranchOrTagInfo info = new TableBranchOrTagInfo(getName(), db, tbl); + metadataOps.createOrReplaceTag(externalTable, tagInfo); + TableBranchOrTagInfo info = new TableBranchOrTagInfo(getName(), externalTable.getDbName(), + externalTable.getName()); Env.getCurrentEnv().getEditLog().logBranchOrTag(info); } catch (Exception e) { LOG.warn("Failed to create or replace tag for table {}.{} in catalog {}", - db, tbl, getName(), e); + externalTable.getDbName(), externalTable.getName(), getName(), e); throw e; } } @@ -1356,35 +1390,41 @@ public void replayOperateOnBranchOrTag(String dbName, String tblName) { } @Override - public void dropBranch(String db, String tbl, DropBranchInfo branchInfo) throws UserException { + public void dropBranch(TableIf dorisTable, DropBranchInfo branchInfo) throws UserException { makeSureInitialized(); + Preconditions.checkState(dorisTable instanceof ExternalTable, dorisTable.getName()); + ExternalTable externalTable = (ExternalTable) dorisTable; if (metadataOps == null) { throw new DdlException("DropBranch operation is not supported for catalog: " + getName()); } try { - metadataOps.dropBranch(db, tbl, branchInfo); - TableBranchOrTagInfo info = new TableBranchOrTagInfo(getName(), db, tbl); + metadataOps.dropBranch(externalTable, branchInfo); + TableBranchOrTagInfo info = new TableBranchOrTagInfo(getName(), externalTable.getDbName(), + externalTable.getName()); Env.getCurrentEnv().getEditLog().logBranchOrTag(info); } catch (Exception e) { LOG.warn("Failed to drop branch for table {}.{} in catalog {}", - db, tbl, getName(), e); + externalTable.getDbName(), externalTable.getName(), getName(), e); throw e; } } @Override - public void dropTag(String db, String tbl, DropTagInfo tagInfo) throws UserException { + public void dropTag(TableIf dorisTable, DropTagInfo tagInfo) throws UserException { makeSureInitialized(); + Preconditions.checkState(dorisTable instanceof ExternalTable, dorisTable.getName()); + ExternalTable externalTable = (ExternalTable) dorisTable; if (metadataOps == null) { throw new DdlException("DropTag operation is not supported for catalog: " + getName()); } try { - metadataOps.dropTag(db, tbl, tagInfo); - TableBranchOrTagInfo info = new TableBranchOrTagInfo(getName(), db, tbl); + metadataOps.dropTag(externalTable, tagInfo); + TableBranchOrTagInfo info = new TableBranchOrTagInfo(getName(), externalTable.getDbName(), + externalTable.getName()); Env.getCurrentEnv().getEditLog().logBranchOrTag(info); } catch (Exception e) { LOG.warn("Failed to drop tag for table {}.{} in catalog {}", - db, tbl, getName(), e); + externalTable.getDbName(), externalTable.getName(), getName(), e); throw e; } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalDatabase.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalDatabase.java index a726f604dcdb53..9b151a2f354802 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalDatabase.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalDatabase.java @@ -41,6 +41,7 @@ import org.apache.doris.qe.ConnectContext; import org.apache.doris.qe.MasterCatalogExecutor; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Objects; import com.google.common.base.Preconditions; import com.google.common.base.Strings; @@ -160,20 +161,7 @@ public final synchronized void makeSureInitialized() { extCatalog.makeSureInitialized(); if (!initialized) { if (extCatalog.getUseMetaCache().get()) { - if (metaCache == null) { - metaCache = Env.getCurrentEnv().getExtMetaCacheMgr().buildMetaCache( - name, - OptionalLong.of(Config.external_cache_expire_time_seconds_after_access), - OptionalLong.of(Config.external_cache_refresh_time_minutes * 60L), - Config.max_meta_object_cache_num, - ignored -> listTableNames(), - localTableName -> Optional.ofNullable( - buildTableForInit(null, localTableName, - Util.genIdByName(extCatalog.getName(), name, localTableName), - extCatalog, - this, true)), - (key, value, cause) -> value.ifPresent(ExternalTable::unsetObjectCreated)); - } + buildMetaCache(); setLastUpdateTime(System.currentTimeMillis()); } else { if (!Env.getCurrentEnv().isMaster()) { @@ -233,8 +221,7 @@ public void replayInitDb(InitDatabaseLog log, ExternalCatalog catalog) { } } for (int i = 0; i < log.getCreateCount(); i++) { - T table = - buildTableForInit(log.getRemoteTableNames().get(i), log.getCreateTableNames().get(i), + T table = buildTableForInit(log.getRemoteTableNames().get(i), log.getCreateTableNames().get(i), log.getCreateTableIds().get(i), catalog, this, false); tmpTableNameToId.put(table.getName(), table.getId()); tmpIdToTbl.put(table.getId(), table); @@ -309,6 +296,23 @@ private void init() { Env.getCurrentEnv().getEditLog().logInitExternalDb(initDatabaseLog); } + private void buildMetaCache() { + if (metaCache == null) { + metaCache = Env.getCurrentEnv().getExtMetaCacheMgr().buildMetaCache( + name, + OptionalLong.of(Config.external_cache_expire_time_seconds_after_access), + OptionalLong.of(Config.external_cache_refresh_time_minutes * 60L), + Config.max_meta_object_cache_num, + ignored -> listTableNames(), + localTableName -> Optional.ofNullable( + buildTableForInit(null, localTableName, + Util.genIdByName(extCatalog.getName(), name, localTableName), + extCatalog, + this, true)), + (key, value, cause) -> value.ifPresent(ExternalTable::unsetObjectCreated)); + } + } + private List> listTableNames() { List> tableNames; if (name.equals(InfoSchemaDb.DATABASE_NAME)) { @@ -429,7 +433,18 @@ public T buildTableForInit(String remoteTableName, String localTableName, long t protected abstract T buildTableInternal(String remoteTableName, String localTableName, long tblId, ExternalCatalog catalog, ExternalDatabase db); + /** + * This method will try getting table from cache only, + * If there is no cache, it will return empty. + * Different from "getTableNullable()", this method will not visit the remote catalog to get table + * when it does not exist in cache. + * This is used for replaying the metadata, to avoid exception when trying to get table from remote catalog. + * + * @param tableId + * @return + */ public Optional getTableForReplay(long tableId) { + Preconditions.checkState(extCatalog.getUseMetaCache().isPresent(), extCatalog.getName() + "." + name); if (extCatalog.getUseMetaCache().get()) { if (!isInitialized()) { return Optional.empty(); @@ -440,6 +455,26 @@ public Optional getTableForReplay(long tableId) { } } + /** + * Same as "getTableForReplay(long tableId)", use "tryGetMetaObj" to get table from cache only. + * + * @param tblName + * @return + */ + public Optional getTableForReplay(String tblName) { + Preconditions.checkState(extCatalog.getUseMetaCache().isPresent(), extCatalog.getName() + "." + name); + if (extCatalog.getUseMetaCache().get()) { + if (!isInitialized()) { + return Optional.empty(); + } + return metaCache.tryGetMetaObj(tblName); + } else if (tableNameToId.containsKey(tblName)) { + return Optional.ofNullable(idToTbl.get(tableNameToId.get(tblName))); + } else { + return Optional.empty(); + } + } + @Override public void readLock() { this.rwLock.readLock().lock(); @@ -512,21 +547,20 @@ public DatabaseProperty getDbProperties() { @Override public boolean isTableExist(String tableName) { + String remoteTblName = tableName; if (this.isTableNamesCaseInsensitive()) { - String realTableName = lowerCaseToTableName.get(tableName.toLowerCase()); - if (realTableName == null) { + remoteTblName = lowerCaseToTableName.get(tableName.toLowerCase()); + if (remoteTblName == null) { // Here we need to execute listTableNames() once to fill in lowerCaseToTableName // to prevent lowerCaseToTableName from being empty in some cases listTableNames(); - tableName = lowerCaseToTableName.get(tableName.toLowerCase()); - if (tableName == null) { + remoteTblName = lowerCaseToTableName.get(tableName.toLowerCase()); + if (remoteTblName == null) { return false; } - } else { - tableName = realTableName; } } - return extCatalog.tableExist(ConnectContext.get().getSessionContext(), name, tableName); + return extCatalog.tableExist(ConnectContext.get().getSessionContext(), remoteName, remoteTblName); } // ATTN: this method only returned cached tables. @@ -688,30 +722,33 @@ public void gsonPostProcess() throws IOException { @Override public void unregisterTable(String tableName) { makeSureInitialized(); - if (this.isStoredTableNamesLowerCase()) { - tableName = tableName.toLowerCase(); - } if (LOG.isDebugEnabled()) { - LOG.debug("create table [{}]", tableName); + LOG.debug("try unregister table [{}]", tableName); } - + setLastUpdateTime(System.currentTimeMillis()); + // check if the table exists in cache, it not, does return + ExternalTable dorisTable = getTableForReplay(tableName).orElse(null); + if (dorisTable == null) { + return; + } + // clear the cache related to this table. if (extCatalog.getUseMetaCache().get()) { if (isInitialized()) { - metaCache.invalidate(tableName, Util.genIdByName(extCatalog.getName(), name, tableName)); - lowerCaseToTableName.remove(tableName.toLowerCase()); + metaCache.invalidate(dorisTable.getName(), + Util.genIdByName(extCatalog.getName(), name, dorisTable.getName())); + lowerCaseToTableName.remove(dorisTable.getName().toLowerCase()); } } else { - Long tableId = tableNameToId.remove(tableName); + Long tableId = tableNameToId.remove(dorisTable.getName()); if (tableId == null) { - LOG.warn("table [{}] does not exist when drop", tableName); + LOG.warn("table [{}] does not exist when drop", dorisTable.getName()); return; } idToTbl.remove(tableId); - lowerCaseToTableName.remove(tableName.toLowerCase()); + lowerCaseToTableName.remove(dorisTable.getName().toLowerCase()); } - setLastUpdateTime(System.currentTimeMillis()); - Env.getCurrentEnv().getExtMetaCacheMgr().invalidateTableCache( - extCatalog.getId(), getFullName(), tableName); + + Env.getCurrentEnv().getExtMetaCacheMgr().invalidateTableCache(dorisTable); } @Override @@ -777,4 +814,14 @@ public boolean equals(Object o) { public int hashCode() { return Objects.hashCode(name, extCatalog); } + + @VisibleForTesting + public void addTableForTest(T tbl) { + // 1. add for "use_meta_cache = false" + idToTbl.put(tbl.getId(), tbl); + tableNameToId.put(tbl.getName(), tbl.getId()); + // 2. add for "use_meta_cache = true" + buildMetaCache(); + metaCache.addObjForTest(tbl.getId(), tbl.getName(), tbl); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalMetaCacheMgr.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalMetaCacheMgr.java index aa53a8f3142cf0..e777285a07f587 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalMetaCacheMgr.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalMetaCacheMgr.java @@ -234,24 +234,24 @@ public void removeCache(long catalogId) { paimonMetadataCacheMgr.removeCache(catalogId); } - public void invalidateTableCache(long catalogId, String dbName, String tblName) { - dbName = ClusterNamespace.getNameFromFullName(dbName); + public void invalidateTableCache(ExternalTable dorisTable) { synchronized (schemaCacheMap) { - ExternalSchemaCache schemaCache = schemaCacheMap.get(catalogId); + ExternalSchemaCache schemaCache = schemaCacheMap.get(dorisTable.getCatalog().getId()); if (schemaCache != null) { - schemaCache.invalidateTableCache(dbName, tblName); + schemaCache.invalidateTableCache(dorisTable); } } - HiveMetaStoreCache metaCache = cacheMap.get(catalogId); + HiveMetaStoreCache metaCache = cacheMap.get(dorisTable.getCatalog().getId()); if (metaCache != null) { - metaCache.invalidateTableCache(dbName, tblName); + metaCache.invalidateTableCache(dorisTable.getOrBuildNameMapping()); } - hudiMetadataCacheMgr.invalidateTableCache(catalogId, dbName, tblName); - icebergMetadataCacheMgr.invalidateTableCache(catalogId, dbName, tblName); - maxComputeMetadataCacheMgr.invalidateTableCache(catalogId, dbName, tblName); - paimonMetadataCacheMgr.invalidateTableCache(catalogId, dbName, tblName); + hudiMetadataCacheMgr.invalidateTableCache(dorisTable); + icebergMetadataCacheMgr.invalidateTableCache(dorisTable); + maxComputeMetadataCacheMgr.invalidateTableCache(dorisTable); + paimonMetadataCacheMgr.invalidateTableCache(dorisTable); if (LOG.isDebugEnabled()) { - LOG.debug("invalid table cache for {}.{} in catalog {}", dbName, tblName, catalogId); + LOG.debug("invalid table cache for {}.{} in catalog {}", dorisTable.getRemoteDbName(), + dorisTable.getRemoteName(), dorisTable.getCatalog().getName()); } } @@ -310,7 +310,7 @@ public void addPartitionsCache(long catalogId, HMSExternalTable table, List partitionNames) { - HiveMetaStoreCache metaCache = cacheMap.get(catalogId); + public void invalidatePartitionsCache(ExternalTable dorisTable, List partitionNames) { + HiveMetaStoreCache metaCache = cacheMap.get(dorisTable.getCatalog().getId()); if (metaCache != null) { - dbName = ClusterNamespace.getNameFromFullName(dbName); for (String partitionName : partitionNames) { - metaCache.invalidatePartitionCache(dbName, tableName, partitionName); + metaCache.invalidatePartitionCache(dorisTable, partitionName); } - } if (LOG.isDebugEnabled()) { - LOG.debug("invalidate partition cache for {}.{} in catalog {}", dbName, tableName, catalogId); + LOG.debug("invalidate partition cache for {}.{} in catalog {}", + dorisTable.getDbName(), dorisTable.getName(), dorisTable.getCatalog().getName()); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalSchemaCache.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalSchemaCache.java index 07946ad6619de2..73b96198ccf860 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalSchemaCache.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalSchemaCache.java @@ -17,7 +17,6 @@ package org.apache.doris.datasource; -import org.apache.doris.catalog.Column; import org.apache.doris.common.CacheFactory; import org.apache.doris.common.Config; import org.apache.doris.metric.GaugeMetric; @@ -26,13 +25,11 @@ import org.apache.doris.metric.MetricRepo; import com.github.benmanes.caffeine.cache.LoadingCache; -import com.google.common.collect.ImmutableList; import lombok.Data; import org.apache.commons.lang3.math.NumberUtils; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; -import java.util.Objects; import java.util.Optional; import java.util.OptionalLong; import java.util.Set; @@ -85,31 +82,22 @@ private Optional loadSchema(SchemaCacheKey key) { return schema; } - public Optional getSchemaValue(String dbName, String tblName) { - SchemaCacheKey key = new SchemaCacheKey(dbName, tblName); - return getSchemaValue(key); - } - public Optional getSchemaValue(SchemaCacheKey key) { return schemaCache.get(key); } - public void addSchemaForTest(String dbName, String tblName, ImmutableList schema) { - SchemaCacheKey key = new SchemaCacheKey(dbName, tblName); - schemaCache.put(key, Optional.of(new SchemaCacheValue(schema))); - } - - public void invalidateTableCache(String dbName, String tblName) { + public void invalidateTableCache(ExternalTable dorisTable) { schemaCache.asMap().keySet().stream() - .filter(key -> key.dbName.equals(dbName) && key.tblName.equals(tblName)) - .forEach(schemaCache::invalidate); + .filter(key -> key.getNameMapping().getLocalDbName().equals(dorisTable.getDbName()) + && key.getNameMapping().getLocalTblName().equals(dorisTable.getName())) + .forEach(schemaCache::invalidate); } public void invalidateDbCache(String dbName) { long start = System.currentTimeMillis(); Set keys = schemaCache.asMap().keySet(); for (SchemaCacheKey key : keys) { - if (key.dbName.equals(dbName)) { + if (key.getNameMapping().getLocalDbName().equals(dbName)) { schemaCache.invalidate(key); } } @@ -128,12 +116,10 @@ public void invalidateAll() { @Data public static class SchemaCacheKey { - private String dbName; - private String tblName; + private NameMapping nameMapping; - public SchemaCacheKey(String dbName, String tblName) { - this.dbName = dbName; - this.tblName = tblName; + public SchemaCacheKey(NameMapping nameMapping) { + this.nameMapping = nameMapping; } @Override @@ -144,17 +130,18 @@ public boolean equals(Object obj) { if (!(obj instanceof SchemaCacheKey)) { return false; } - return dbName.equals(((SchemaCacheKey) obj).dbName) && tblName.equals(((SchemaCacheKey) obj).tblName); + return nameMapping.equals(((SchemaCacheKey) obj).nameMapping); } @Override public int hashCode() { - return Objects.hash(dbName, tblName); + return nameMapping.hashCode(); } @Override public String toString() { - return "SchemaCacheKey{" + "dbName='" + dbName + '\'' + ", tblName='" + tblName + '\'' + '}'; + return "SchemaCacheKey{" + "dbName='" + + nameMapping.getLocalDbName() + '\'' + ", tblName='" + nameMapping.getLocalTblName() + '\'' + '}'; } } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalTable.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalTable.java index bdeb31b3559495..91c087a109225e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalTable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalTable.java @@ -90,6 +90,9 @@ public class ExternalTable implements TableIf, Writable, GsonPostProcessable { protected boolean objectCreated; protected ExternalCatalog catalog; protected ExternalDatabase db; + // Used to save the mapping between local and remote names. + // prebuild it for performance. + protected NameMapping nameMapping; /** * No args constructor for persist. @@ -118,6 +121,7 @@ public ExternalTable(long id, String name, String remoteName, ExternalCatalog ca this.dbName = db.getFullName(); this.type = type; this.objectCreated = false; + this.nameMapping = new NameMapping(catalog.getId(), dbName, name, db.getRemoteName(), getRemoteName()); } public void setCatalog(ExternalCatalog catalog) { @@ -169,7 +173,7 @@ public TableType getType() { @Override public List getFullSchema() { ExternalSchemaCache cache = Env.getCurrentEnv().getExtMetaCacheMgr().getSchemaCache(catalog); - Optional schemaCacheValue = cache.getSchemaValue(dbName, name); + Optional schemaCacheValue = cache.getSchemaValue(new SchemaCacheKey(getOrBuildNameMapping())); return schemaCacheValue.map(SchemaCacheValue::getSchema).orElse(null); } @@ -392,7 +396,7 @@ public List getChunkSizes() { public Optional getSchemaCacheValue() { ExternalSchemaCache cache = Env.getCurrentEnv().getExtMetaCacheMgr().getSchemaCache(catalog); - return cache.getSchemaValue(dbName, name); + return cache.getSchemaValue(new SchemaCacheKey(getOrBuildNameMapping())); } @Override @@ -498,8 +502,29 @@ public String getRemoteDbName() { return db.getRemoteName(); } - public TableAttributes getTableAttributes() { return tableAttributes; } + + /** + * Build the name mapping for this table. + * If "use_meta_cache" is true, the "nameMapping" should already be created in constructor. + * But if "use_meta_cache" is false, we can not create "nameMapping" in constructor because the catalog and db + * object may be null at that time. + * So we need to check and build the name mapping here, for both "use_meta_cache" true or false. + * + * @return + */ + public NameMapping getOrBuildNameMapping() { + if (nameMapping != null) { + return nameMapping; + } + synchronized (this) { + if (nameMapping != null) { + return nameMapping; + } + nameMapping = new NameMapping(catalog.getId(), dbName, name, db.getRemoteName(), getRemoteName()); + return nameMapping; + } + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java index 561de217b0846e..98b385e4f2e7e5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java @@ -24,12 +24,10 @@ import org.apache.doris.analysis.AlterClause; import org.apache.doris.analysis.AlterDatabasePropertyStmt; import org.apache.doris.analysis.AlterMultiPartitionClause; -import org.apache.doris.analysis.CreateDbStmt; import org.apache.doris.analysis.CreateTableStmt; import org.apache.doris.analysis.DataSortInfo; import org.apache.doris.analysis.DistributionDesc; import org.apache.doris.analysis.DropPartitionClause; -import org.apache.doris.analysis.DropTableStmt; import org.apache.doris.analysis.Expr; import org.apache.doris.analysis.FunctionCallExpr; import org.apache.doris.analysis.KeysDesc; @@ -131,9 +129,7 @@ import org.apache.doris.event.DropPartitionEvent; import org.apache.doris.mtmv.MTMVUtil; import org.apache.doris.mysql.privilege.PrivPredicate; -import org.apache.doris.nereids.trees.plans.commands.CreateDatabaseCommand; import org.apache.doris.nereids.trees.plans.commands.DropCatalogRecycleBinCommand.IdType; -import org.apache.doris.nereids.trees.plans.commands.TruncateTableCommand; import org.apache.doris.persist.AlterDatabasePropertyInfo; import org.apache.doris.persist.AutoIncrementIdUpdateLog; import org.apache.doris.persist.ColocatePersistInfo; @@ -406,15 +402,14 @@ public void recreateTabletInvertIndex() { /** * Entry of creating a database. * - * @param stmt + * @param dbName + * @param ifNotExists + * @param properties * @throws DdlException */ - public void createDb(CreateDbStmt stmt) throws DdlException { - String fullDbName = stmt.getFullDbName(); - Map properties = stmt.getProperties(); - + public void createDb(String dbName, boolean ifNotExists, Map properties) throws DdlException { long id = Env.getCurrentEnv().getNextId(); - Database db = new Database(id, fullDbName); + Database db = new Database(id, dbName); // check and analyze database properties before create database db.checkStorageVault(properties); db.setDbProperties(new DatabaseProperty(properties)); @@ -423,18 +418,18 @@ public void createDb(CreateDbStmt stmt) throws DdlException { throw new DdlException("Failed to acquire catalog lock. Try again"); } try { - if (fullNameToDb.containsKey(fullDbName)) { - if (stmt.isSetIfNotExists()) { - LOG.info("create database[{}] which already exists", fullDbName); + if (fullNameToDb.containsKey(dbName)) { + if (ifNotExists) { + LOG.info("create database[{}] which already exists", dbName); return; } else { - ErrorReport.reportDdlException(ErrorCode.ERR_DB_CREATE_EXISTS, fullDbName); + ErrorReport.reportDdlException(ErrorCode.ERR_DB_CREATE_EXISTS, dbName); } } else { if (!db.tryWriteLock(100, TimeUnit.SECONDS)) { - LOG.warn("try lock failed, create database failed {}", fullDbName); + LOG.warn("try lock failed, create database failed {}", dbName); ErrorReport.reportDdlException(ErrorCode.ERR_EXECUTE_TIMEOUT, - "create database " + fullDbName + " time out"); + "create database " + dbName + " time out"); } try { unprotectCreateDb(db); @@ -447,54 +442,7 @@ public void createDb(CreateDbStmt stmt) throws DdlException { } finally { unlock(); } - LOG.info("createDb dbName = " + fullDbName + ", id = " + id); - } - - /** - * Entry of creating a database. - * - * @param command - * @throws DdlException - */ - public void createDb(CreateDatabaseCommand command) throws DdlException { - String fullDbName = command.getDbName(); - Map properties = command.getProperties(); - - long id = Env.getCurrentEnv().getNextId(); - Database db = new Database(id, fullDbName); - // check and analyze database properties before create database - db.checkStorageVault(properties); - db.setDbProperties(new DatabaseProperty(properties)); - - if (!tryLock(false)) { - throw new DdlException("Failed to acquire catalog lock. Try again"); - } - try { - if (fullNameToDb.containsKey(fullDbName)) { - if (command.isIfNotExists()) { - LOG.info("create database[{}] which already exists", fullDbName); - return; - } else { - ErrorReport.reportDdlException(ErrorCode.ERR_DB_CREATE_EXISTS, fullDbName); - } - } else { - if (!db.tryWriteLock(100, TimeUnit.SECONDS)) { - LOG.warn("try lock failed, create database failed {}", fullDbName); - ErrorReport.reportDdlException(ErrorCode.ERR_EXECUTE_TIMEOUT, - "create database " + fullDbName + " time out"); - } - try { - unprotectCreateDb(db); - CreateDbInfo dbInfo = new CreateDbInfo(InternalCatalog.INTERNAL_CATALOG_NAME, db.getName(), db); - Env.getCurrentEnv().getEditLog().logCreateDb(dbInfo); - } finally { - db.writeUnlock(); - } - } - } finally { - unlock(); - } - LOG.info("createDb dbName = " + fullDbName + ", id = " + id); + LOG.info("create dbName = " + dbName + ", id = " + id); } /** @@ -914,15 +862,7 @@ public void replayRenameDatabase(String dbName, String newDbName) { LOG.info("replay rename database {} to {}", dbName, newDbName); } - // Drop table - public void dropTable(DropTableStmt stmt) throws DdlException { - if (stmt == null) { - throw new DdlException("DropTableStmt is null"); - } - dropTable(stmt.getDbName(), stmt.getTableName(), stmt.isView(), stmt.isMaterializedView(), - stmt.isSetIfExists(), stmt.isForceDrop()); - } - + @Override public void dropTable(String dbName, String tableName, boolean isView, boolean isMtmv, boolean ifExists, boolean force) throws DdlException { Map costTimes = new TreeMap(); @@ -3392,20 +3332,18 @@ private void checkAutoIncColumns(List columns, KeysType type) throws Ddl * otherwise, it will only truncate those specified partitions. * */ - public void truncateTable(TruncateTableCommand truncateTableCommand) throws DdlException { - boolean isForceDrop = truncateTableCommand.isForceDrop(); - String database = truncateTableCommand.getTableNameInfo().getDb(); - String tbl = truncateTableCommand.getTableNameInfo().getTbl(); - + public void truncateTable(String dbName, String tableName, PartitionNames partitionNames, boolean forceDrop, + String rawTruncateSql) + throws DdlException { // check, and save some info which need to be checked again later Map origPartitions = Maps.newTreeMap(String.CASE_INSENSITIVE_ORDER); Map partitionsDistributionInfo = Maps.newHashMap(); OlapTable copiedTbl; - boolean truncateEntireTable = !truncateTableCommand.getPartitionNamesInfo().isPresent(); + boolean truncateEntireTable = partitionNames == null; - Database db = (Database) getDbOrDdlException(database); - OlapTable olapTable = db.getOlapTableOrDdlException(tbl); + Database db = getDbOrDdlException(dbName); + OlapTable olapTable = db.getOlapTableOrDdlException(tableName); if (olapTable instanceof MTMV && !MTMVUtil.allowModifyMTMVData(ConnectContext.get())) { throw new DdlException("Not allowed to perform current operation on async materialized view"); @@ -3418,14 +3356,14 @@ public void truncateTable(TruncateTableCommand truncateTableCommand) throws DdlE try { olapTable.checkNormalStateForAlter(); if (!truncateEntireTable) { - for (String partName : truncateTableCommand.getPartitionNamesInfo().get().getPartitionNames()) { + for (String partName : partitionNames.getPartitionNames()) { Partition partition = olapTable.getPartition(partName); if (partition == null) { throw new DdlException("Partition " + partName + " does not exist"); } // If need absolutely correct, should check running txn here. - // But if the txn is in prepare state, cann't known which partitions had load data. - if ((isForceDrop) && (!partition.hasData())) { + // But if the txn is in prepare state, can't known which partitions had load data. + if ((forceDrop) && (!partition.hasData())) { // if not force drop, then need to add partition to // recycle bin, so behavior for recover would be clear continue; @@ -3437,8 +3375,8 @@ public void truncateTable(TruncateTableCommand truncateTableCommand) throws DdlE } else { for (Partition partition : olapTable.getPartitions()) { // If need absolutely correct, should check running txn here. - // But if the txn is in prepare state, cann't known which partitions had load data. - if ((isForceDrop) && (!partition.hasData())) { + // But if the txn is in prepare state, can't known which partitions had load data. + if ((forceDrop) && (!partition.hasData())) { // if not force drop, then need to add partition to // recycle bin, so behavior for recover would be clear continue; @@ -3451,8 +3389,8 @@ public void truncateTable(TruncateTableCommand truncateTableCommand) throws DdlE // if table currently has no partitions, this sql like empty command and do nothing, should return directly. // but if truncate whole table, the temporary partitions also need drop if (origPartitions.isEmpty() && (!truncateEntireTable || olapTable.getAllTempPartitions().isEmpty())) { - LOG.info("finished to truncate table {}, no partition contains data, do nothing", - truncateTableCommand.getTableNameInfo().toSql()); + LOG.info("finished to truncate table {}.{}, no partition contains data, do nothing", + dbName, tableName); return; } copiedTbl = olapTable.selectiveCopy(origPartitions.keySet(), IndexExtState.VISIBLE, false); @@ -3590,13 +3528,13 @@ public void truncateTable(TruncateTableCommand truncateTableCommand) throws DdlE //replace Map recyclePartitionParamMap = new HashMap<>(); oldPartitions = truncateTableInternal(olapTable, newPartitions, - truncateEntireTable, recyclePartitionParamMap, isForceDrop); + truncateEntireTable, recyclePartitionParamMap, forceDrop); // write edit log TruncateTableInfo info = new TruncateTableInfo(db.getId(), db.getFullName(), olapTable.getId(), olapTable.getName(), newPartitions, truncateEntireTable, - truncateTableCommand.toSqlWithoutTable(), oldPartitions, isForceDrop); + rawTruncateSql, oldPartitions, forceDrop); Env.getCurrentEnv().getEditLog().logTruncateTable(info); } catch (DdlException e) { failedCleanCallback.run(); @@ -3607,14 +3545,9 @@ public void truncateTable(TruncateTableCommand truncateTableCommand) throws DdlE } } - PartitionNames partitionNames = truncateEntireTable ? null - : new PartitionNames(false, truncateTableCommand.getPartitionNamesInfo().get().getPartitionNames()); Env.getCurrentEnv().getAnalysisManager().dropStats(olapTable, partitionNames); Env.getCurrentEnv().getAnalysisManager().updateUpdatedRows(updateRecords, db.getId(), olapTable.getId(), 0); - LOG.info("finished to truncate table {}, partitions: {}", - truncateTableCommand.getTableNameInfo().toSql(), - !truncateTableCommand.getPartitionNamesInfo().isPresent() - ? "null" : truncateTableCommand.getPartitionNamesInfo().get().getPartitionNames()); + LOG.info("finished to truncate table {}.{}, partitions: {}", dbName, tableName, partitionNames); } private List truncateTableInternal(OlapTable olapTable, List newPartitions, diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/NameMapping.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/NameMapping.java new file mode 100644 index 00000000000000..192b86d0995987 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/NameMapping.java @@ -0,0 +1,88 @@ +// 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; + +import com.google.common.annotations.VisibleForTesting; +import lombok.Getter; + +import java.util.Objects; + +/** + * NameMapping class represents a mapping between local and remote database and table names. + */ +@Getter +public class NameMapping { + private final long ctlId; + private final String localDbName; + private final String localTblName; + private final String remoteDbName; + private final String remoteTblName; + + public NameMapping(long ctlId, String localDbName, String localTblName, String remoteDbName, String remoteTblName) { + this.ctlId = ctlId; + this.localDbName = localDbName; + this.localTblName = localTblName; + this.remoteDbName = remoteDbName; + this.remoteTblName = remoteTblName; + } + + @VisibleForTesting + public static NameMapping createForTest(String dbName, String tblName) { + return new NameMapping(0, dbName, tblName, dbName, tblName); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (!(o instanceof NameMapping)) { + return false; + } + NameMapping that = (NameMapping) o; + return ctlId == that.ctlId + && localDbName.equals(that.localDbName) + && localTblName.equals(that.localTblName) + && remoteDbName.equals(that.remoteDbName) + && remoteTblName.equals(that.remoteTblName); + } + + public String getFullLocalName() { + return String.format("%s.%s", localDbName, localTblName); + } + + public String getFullRemoteName() { + return String.format("%s.%s", remoteDbName, remoteTblName); + } + + @Override + public int hashCode() { + return Objects.hash(ctlId, localDbName, localTblName, remoteDbName, remoteTblName); + } + + @Override + public String toString() { + return "NameMapping{" + + "ctlId=" + ctlId + + ", localDbName='" + localDbName + '\'' + + ", localTblName='" + localTblName + '\'' + + ", remoteDbName='" + remoteDbName + '\'' + + ", remoteTblName='" + remoteTblName + '\'' + + '}'; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/TablePartitionValues.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/TablePartitionValues.java index e928d3c739eed3..008fa940178910 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/TablePartitionValues.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/TablePartitionValues.java @@ -95,7 +95,6 @@ public void addPartitions(List partitionNames, List> partit addPartitionItems(addPartitionNames, addPartitionItems, types); } - private void addPartitionItems(List partitionNames, List partitionItems, List types) { Preconditions.checkState(partitionNames.size() == partitionItems.size()); Preconditions.checkState(nextPartitionId == 0); @@ -172,16 +171,13 @@ private ListPartitionItem toListPartitionItem(List partitionValues, List @Data public static class TablePartitionKey { + // use local db/table name here + // because these names are only used to compare with local names private final String dbName; private final String tblName; // not in key private List types; - public TablePartitionKey(String dbName, String tblName) { - this.dbName = dbName; - this.tblName = tblName; - } - public TablePartitionKey(String dbName, String tblName, List types) { this.dbName = dbName; this.tblName = tblName; diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/es/EsExternalDatabase.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/es/EsExternalDatabase.java index 39db1452833eac..b6974e6bf41176 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/es/EsExternalDatabase.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/es/EsExternalDatabase.java @@ -45,9 +45,4 @@ public EsExternalTable buildTableInternal(String remoteTableName, String localTa return new EsExternalTable(tblId, localTableName, remoteTableName, (EsExternalCatalog) extCatalog, (EsExternalDatabase) db); } - - public void addTableForTest(EsExternalTable tbl) { - idToTbl.put(tbl.getId(), tbl); - tableNameToId.put(tbl.getName(), tbl.getId()); - } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSExternalTable.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSExternalTable.java index e35cf36ba0a589..3f48ad1246029d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSExternalTable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSExternalTable.java @@ -216,7 +216,7 @@ public boolean isSupportedHmsTable() { protected synchronized void makeSureInitialized() { super.makeSureInitialized(); if (!objectCreated) { - remoteTable = ((HMSExternalCatalog) catalog).getClient().getTable(dbName, name); + remoteTable = loadHiveTable(); if (remoteTable == null) { throw new IllegalArgumentException("Hms table not exists, table: " + getNameWithFullQualifiers()); } else { @@ -346,9 +346,9 @@ public List getFullSchema() { return ((HudiDlaTable) dlaTable).getHudiSchemaCacheValue(MvccUtil.getSnapshotFromContext(this)) .getSchema(); } else if (getDlaType() == DLAType.ICEBERG) { - return IcebergUtils.getIcebergSchema(this, getCatalog(), getDbName(), getName()); + return IcebergUtils.getIcebergSchema(this); } - Optional schemaCacheValue = cache.getSchemaValue(dbName, name); + Optional schemaCacheValue = cache.getSchemaValue(new SchemaCacheKey(getOrBuildNameMapping())); return schemaCacheValue.map(SchemaCacheValue::getSchema).orElse(null); } @@ -366,7 +366,7 @@ public List getHudiPartitionColumnTypes(long timestamp) { makeSureInitialized(); ExternalSchemaCache cache = Env.getCurrentEnv().getExtMetaCacheMgr().getSchemaCache(catalog); Optional schemaCacheValue = cache.getSchemaValue( - new HudiSchemaCacheKey(dbName, name, timestamp)); + new HudiSchemaCacheKey(getOrBuildNameMapping(), timestamp)); return schemaCacheValue.map(value -> ((HMSSchemaCacheValue) value).getPartitionColTypes()) .orElse(Collections.emptyList()); } @@ -420,7 +420,7 @@ public Map getNameToPartitionItems() { .getMetaStoreCache((HMSExternalCatalog) this.getCatalog()); List partitionColumnTypes = this.getPartitionColumnTypes(MvccUtil.getSnapshotFromContext(this)); HiveMetaStoreCache.HivePartitionValues hivePartitionValues = cache.getPartitionValues( - this.getDbName(), this.getName(), partitionColumnTypes); + this, partitionColumnTypes); Map idToPartitionItem = hivePartitionValues.getIdToPartitionItem(); // transfer id to name BiMap idToName = hivePartitionValues.getPartitionNameToIdMap().inverse(); @@ -476,7 +476,7 @@ private long getRowCountFromExternalSource() { rowCount = StatisticsUtil.getHiveRowCount(this); break; case ICEBERG: - rowCount = IcebergUtils.getIcebergRowCount(getCatalog(), getDbName(), getName()); + rowCount = IcebergUtils.getIcebergRowCount(this); break; default: if (LOG.isDebugEnabled()) { @@ -611,32 +611,31 @@ public Map getHadoopProperties() { public List getHiveTableColumnStats(List columns) { HMSCachedClient client = ((HMSExternalCatalog) catalog).getClient(); - return client.getTableColumnStatistics(dbName, name, columns); + return client.getTableColumnStatistics(getRemoteDbName(), remoteName, columns); } public Map> getHivePartitionColumnStats( List partNames, List columns) { HMSCachedClient client = ((HMSExternalCatalog) catalog).getClient(); - return client.getPartitionColumnStatistics(dbName, name, partNames, columns); + return client.getPartitionColumnStatistics(getRemoteDbName(), remoteName, partNames, columns); } public Partition getPartition(List partitionValues) { HMSCachedClient client = ((HMSExternalCatalog) catalog).getClient(); - return client.getPartition(dbName, name, partitionValues); + return client.getPartition(getRemoteDbName(), remoteName, partitionValues); } @Override public Set getPartitionNames() { makeSureInitialized(); HMSCachedClient client = ((HMSExternalCatalog) catalog).getClient(); - List names = client.listPartitionNames(dbName, name); + List names = client.listPartitionNames(getRemoteDbName(), getRemoteName()); return new HashSet<>(names); } @Override public Optional initSchemaAndUpdateTime(SchemaCacheKey key) { - org.apache.hadoop.hive.metastore.api.Table table = ((HMSExternalCatalog) catalog).getClient() - .getTable(dbName, name); + Table table = loadHiveTable(); // try to use transient_lastDdlTime from hms client schemaUpdateTime = MapUtils.isNotEmpty(table.getParameters()) && table.getParameters().containsKey(TBL_PROP_TRANSIENT_LAST_DDL_TIME) @@ -659,20 +658,20 @@ public long getLastDdlTime() { public Optional initSchema(SchemaCacheKey key) { makeSureInitialized(); if (dlaType.equals(DLAType.ICEBERG)) { - return getIcebergSchema(key); + return initIcebergSchema(key); } else if (dlaType.equals(DLAType.HUDI)) { - return getHudiSchema(key); + return initHudiSchema(key); } else { - return getHiveSchema(); + return initHiveSchema(); } } - private Optional getIcebergSchema(SchemaCacheKey key) { + private Optional initIcebergSchema(SchemaCacheKey key) { return IcebergUtils.loadSchemaCacheValue( - catalog, dbName, name, ((IcebergSchemaCacheKey) key).getSchemaId(), isView()); + this, ((IcebergSchemaCacheKey) key).getSchemaId(), isView()); } - private Optional getHudiSchema(SchemaCacheKey key) { + private Optional initHudiSchema(SchemaCacheKey key) { boolean[] enableSchemaEvolution = {false}; HudiSchemaCacheKey hudiSchemaCacheKey = (HudiSchemaCacheKey) key; InternalSchema hudiInternalSchema = HiveMetaStoreClientHelper.getHudiTableSchema(this, enableSchemaEvolution, @@ -696,7 +695,7 @@ private Optional getHudiSchema(SchemaCacheKey key) { return Optional.of(hudiSchemaCacheValue); } - private Optional getHiveSchema() { + private Optional initHiveSchema() { boolean getFromTable = catalog.getCatalogProperty() .getOrDefault(HMSExternalCatalog.GET_SCHEMA_FROM_TABLE, "false") .equalsIgnoreCase("true"); @@ -706,8 +705,8 @@ private Optional getHiveSchema() { schema = getSchemaFromRemoteTable(); } else { HMSCachedClient client = ((HMSExternalCatalog) catalog).getClient(); - schema = client.getSchema(dbName, name); - colDefaultValues = client.getDefaultColumnValues(dbName, name); + schema = client.getSchema(getRemoteDbName(), remoteName); + colDefaultValues = client.getDefaultColumnValues(getRemoteDbName(), remoteName); } List columns = Lists.newArrayListWithCapacity(schema.size()); for (FieldSchema field : schema) { @@ -724,7 +723,7 @@ private Optional getHiveSchema() { private List getSchemaFromRemoteTable() { // Here we should get a new remote table instead of using this.remoteTable // Because we need to get the latest schema from HMS. - Table newTable = ((HMSExternalCatalog) catalog).getClient().getTable(dbName, name); + Table newTable = loadHiveTable(); List schema = Lists.newArrayList(); schema.addAll(newTable.getSd().getCols()); schema.addAll(newTable.getPartitionKeys()); @@ -799,9 +798,7 @@ public Optional getColumnStatistic(String colName) { case ICEBERG: if (GlobalVariable.enableFetchIcebergStats) { return StatisticsUtil.getIcebergColumnStats(colName, - Env.getCurrentEnv().getExtMetaCacheMgr().getIcebergMetadataCache().getIcebergTable( - catalog, dbName, name - )); + Env.getCurrentEnv().getExtMetaCacheMgr().getIcebergMetadataCache().getIcebergTable(this)); } else { break; } @@ -989,9 +986,9 @@ public MTMVSnapshotIf getTableSnapshot(Optional snapshot) throws A public long getNewestUpdateVersionOrTime() { HiveMetaStoreCache cache = Env.getCurrentEnv().getExtMetaCacheMgr() .getMetaStoreCache((HMSExternalCatalog) getCatalog()); - HiveMetaStoreCache.HivePartitionValues hivePartitionValues = cache.getPartitionValues(getDbName(), getName(), + HiveMetaStoreCache.HivePartitionValues hivePartitionValues = cache.getPartitionValues(this, getPartitionColumnTypes(MvccUtil.getSnapshotFromContext(this))); - List partitionList = cache.getAllPartitionsWithCache(getDbName(), getName(), + List partitionList = cache.getAllPartitionsWithCache(this, Lists.newArrayList(hivePartitionValues.getPartitionValuesMap().values())); if (CollectionUtils.isEmpty(partitionList)) { return 0; @@ -1075,7 +1072,7 @@ private HiveMetaStoreCache.HivePartitionValues getAllPartitionValues() { // It is ok to get partition values from cache, // no need to worry that this call will invalid or refresh the cache. // because it has enough space to keep partition info of all tables in cache. - partitionValues = cache.getPartitionValues(dbName, name, partitionColumnTypes); + partitionValues = cache.getPartitionValues(this, partitionColumnTypes); if (partitionValues == null || partitionValues.getPartitionNameToIdMap() == null) { LOG.warn("Partition values for hive table {} is null", name); } else { @@ -1116,10 +1113,10 @@ private List getFilesForPartitions( } // get partitions without cache, so that it will not invalid the cache when executing // non query request such as `show table status` - hivePartitions = cache.getAllPartitionsWithoutCache(dbName, name, partitionValuesList); + hivePartitions = cache.getAllPartitionsWithoutCache(this, partitionValuesList); LOG.info("Partition list size for hive partition table {} is {}", name, hivePartitions.size()); } else { - hivePartitions.add(new HivePartition(dbName, name, true, + hivePartitions.add(new HivePartition(getOrBuildNameMapping(), true, getRemoteTable().getSd().getInputFormat(), getRemoteTable().getSd().getLocation(), null, Maps.newHashMap())); } @@ -1149,8 +1146,7 @@ public MvccSnapshot loadSnapshot(Optional tableSnapshot, Optional return HudiUtils.getHudiMvccSnapshot(tableSnapshot, this); } else if (getDlaType() == DLAType.ICEBERG) { return new IcebergMvccSnapshot( - IcebergUtils.getIcebergSnapshotCacheValue( - tableSnapshot, getCatalog(), getDbName(), getName(), scanParams)); + IcebergUtils.getIcebergSnapshotCacheValue(tableSnapshot, this, scanParams)); } else { return new EmptyMvccSnapshot(); } @@ -1169,8 +1165,7 @@ public HoodieTableMetaClient getHudiClient() { .getExtMetaCacheMgr() .getMetaClientProcessor(getCatalog()) .getHoodieTableMetaClient( - getDbName(), - getName(), + getOrBuildNameMapping(), getRemoteTable().getSd().getLocation(), getCatalog().getConfiguration()); } @@ -1195,4 +1190,8 @@ public List getSupportedSysTables() { } } + private Table loadHiveTable() { + HMSCachedClient client = ((HMSExternalCatalog) catalog).getClient(); + return client.getTable(getRemoteDbName(), remoteName); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSTransaction.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSTransaction.java index dce7cc7cdd7383..7dc0154c14c477 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSTransaction.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSTransaction.java @@ -24,8 +24,8 @@ import org.apache.doris.backup.Status; import org.apache.doris.common.Pair; import org.apache.doris.common.UserException; -import org.apache.doris.common.info.SimpleTableInfo; import org.apache.doris.common.profile.SummaryProfile; +import org.apache.doris.datasource.NameMapping; import org.apache.doris.datasource.statistics.CommonStatistics; import org.apache.doris.fs.FileSystem; import org.apache.doris.fs.FileSystemProvider; @@ -93,10 +93,10 @@ public class HMSTransaction implements Transaction { private boolean isOverwrite = false; TFileType fileType; - private final Map> tableActions = new HashMap<>(); - private final Map, Action>> + private final Map> tableActions = new HashMap<>(); + private final Map, Action>> partitionActions = new HashMap<>(); - private final Map> tableColumns = new HashMap<>(); + private final Map> tableColumns = new HashMap<>(); private final Executor fileSystemExecutor; private HmsCommitter hmsCommitter; @@ -194,8 +194,8 @@ public void beginInsertTable(HiveInsertCommandContext ctx) { } } - public void finishInsertTable(SimpleTableInfo tableInfo) { - Table table = getTable(tableInfo); + public void finishInsertTable(NameMapping nameMapping) { + Table table = getTable(nameMapping); if (hivePartitionUpdates.isEmpty() && isOverwrite && table.getPartitionKeysSize() == 0) { // use an empty hivePartitionUpdate to clean source table isMockedPartitionUpdate = true; @@ -248,15 +248,15 @@ public void finishInsertTable(SimpleTableInfo tableInfo) { case APPEND: finishChangingExistingTable( ActionType.INSERT_EXISTING, - tableInfo, + nameMapping, writePath, pu.getFileNames(), hivePartitionStatistics, pu); break; case OVERWRITE: - dropTable(tableInfo); - createTable(tableInfo, table, writePath, pu.getFileNames(), hivePartitionStatistics, pu); + dropTable(nameMapping); + createTable(nameMapping, table, writePath, pu.getFileNames(), hivePartitionStatistics, pu); break; default: throw new RuntimeException("Not support mode:[" + updateMode + "] in unPartitioned table"); @@ -271,7 +271,7 @@ public void finishInsertTable(SimpleTableInfo tableInfo) { case OVERWRITE: StorageDescriptor sd = table.getSd(); HivePartition hivePartition = new HivePartition( - tableInfo, + nameMapping, false, sd.getInputFormat(), pu.getLocation().getTargetPath(), @@ -282,10 +282,10 @@ public void finishInsertTable(SimpleTableInfo tableInfo) { sd.getCols() ); if (updateMode == TUpdateMode.OVERWRITE) { - dropPartition(tableInfo, hivePartition.getPartitionValues(), true); + dropPartition(nameMapping, hivePartition.getPartitionValues(), true); } addPartition( - tableInfo, hivePartition, writePath, + nameMapping, hivePartition, writePath, pu.getName(), pu.getFileNames(), hivePartitionStatistics, pu); break; default: @@ -295,7 +295,7 @@ public void finishInsertTable(SimpleTableInfo tableInfo) { } if (!insertExistsPartitions.isEmpty()) { - convertToInsertExistingPartitionAction(tableInfo, insertExistsPartitions); + convertToInsertExistingPartitionAction(nameMapping, insertExistsPartitions); } } @@ -303,36 +303,36 @@ public void doCommit() { hmsCommitter = new HmsCommitter(); try { - for (Map.Entry> entry : tableActions.entrySet()) { - SimpleTableInfo tableInfo = entry.getKey(); + for (Map.Entry> entry : tableActions.entrySet()) { + NameMapping nameMapping = entry.getKey(); Action action = entry.getValue(); switch (action.getType()) { case INSERT_EXISTING: - hmsCommitter.prepareInsertExistingTable(tableInfo, action.getData()); + hmsCommitter.prepareInsertExistingTable(nameMapping, action.getData()); break; case ALTER: - hmsCommitter.prepareAlterTable(tableInfo, action.getData()); + hmsCommitter.prepareAlterTable(nameMapping, action.getData()); break; default: throw new UnsupportedOperationException("Unsupported table action type: " + action.getType()); } } - for (Map.Entry, Action>> tableEntry + for (Map.Entry, Action>> tableEntry : partitionActions.entrySet()) { - SimpleTableInfo tableInfo = tableEntry.getKey(); + NameMapping nameMapping = tableEntry.getKey(); for (Map.Entry, Action> partitionEntry : tableEntry.getValue().entrySet()) { Action action = partitionEntry.getValue(); switch (action.getType()) { case INSERT_EXISTING: - hmsCommitter.prepareInsertExistPartition(tableInfo, action.getData()); + hmsCommitter.prepareInsertExistPartition(nameMapping, action.getData()); break; case ADD: - hmsCommitter.prepareAddPartition(tableInfo, action.getData()); + hmsCommitter.prepareAddPartition(nameMapping, action.getData()); break; case ALTER: - hmsCommitter.prepareAlterPartition(tableInfo, action.getData()); + hmsCommitter.prepareAlterPartition(nameMapping, action.getData()); break; default: throw new UnsupportedOperationException( @@ -373,10 +373,10 @@ public long getUpdateCnt() { } private void convertToInsertExistingPartitionAction( - SimpleTableInfo tableInfo, + NameMapping nameMapping, List> partitions) { Map, Action> partitionActionsForTable = - partitionActions.computeIfAbsent(tableInfo, k -> new HashMap<>()); + partitionActions.computeIfAbsent(nameMapping, k -> new HashMap<>()); for (List> partitionBatch : Iterables.partition(partitions, 100)) { @@ -394,7 +394,7 @@ private void convertToInsertExistingPartitionAction( case DROP_PRESERVE_DATA: throw new RuntimeException( "Not found partition from partition actions" - + "for " + tableInfo + ", partitions: " + partitionNames); + + "for " + nameMapping.getFullLocalName() + ", partitions: " + partitionNames); case ADD: case ALTER: case INSERT_EXISTING: @@ -409,7 +409,8 @@ private void convertToInsertExistingPartitionAction( Map partitionsByNamesMap = HiveUtil.convertToNamePartitionMap( partitionNames, - hiveOps.getClient().getPartitions(tableInfo.getDbName(), tableInfo.getTbName(), partitionNames)); + hiveOps.getClient().getPartitions(nameMapping.getRemoteDbName(), nameMapping.getRemoteTblName(), + partitionNames)); for (int i = 0; i < partitionsByNamesMap.size(); i++) { String partitionName = partitionNames.get(i); @@ -418,7 +419,7 @@ private void convertToInsertExistingPartitionAction( if (partition == null) { // Prevent this partition from being deleted by other engines throw new RuntimeException( - "Not found partition from hms for " + tableInfo + "Not found partition from hms for " + nameMapping.getFullLocalName() + ", partitions: " + partitionNames); } THivePartitionUpdate pu = partitionBatch.get(i).first; @@ -428,7 +429,7 @@ private void convertToInsertExistingPartitionAction( List partitionValues = HiveUtil.toPartitionValues(pu.getName()); HivePartition hivePartition = new HivePartition( - tableInfo, + nameMapping, false, sd.getInputFormat(), partition.getSd().getLocation(), @@ -469,16 +470,16 @@ private static void addSuppressedExceptions( } public static class UpdateStatisticsTask { - private final SimpleTableInfo tableInfo; + private final NameMapping nameMapping; private final Optional partitionName; private final HivePartitionStatistics updatePartitionStat; private final boolean merge; private boolean done; - public UpdateStatisticsTask(SimpleTableInfo tableInfo, Optional partitionName, + public UpdateStatisticsTask(NameMapping nameMapping, Optional partitionName, HivePartitionStatistics statistics, boolean merge) { - this.tableInfo = Objects.requireNonNull(tableInfo, "tableInfo is null"); + this.nameMapping = Objects.requireNonNull(nameMapping, "nameMapping is null"); this.partitionName = Objects.requireNonNull(partitionName, "partitionName is null"); this.updatePartitionStat = Objects.requireNonNull(statistics, "statistics is null"); this.merge = merge; @@ -486,9 +487,9 @@ public UpdateStatisticsTask(SimpleTableInfo tableInfo, Optional partitio public void run(HiveMetadataOps hiveOps) { if (partitionName.isPresent()) { - hiveOps.updatePartitionStatistics(tableInfo, partitionName.get(), this::updateStatistics); + hiveOps.updatePartitionStatistics(nameMapping, partitionName.get(), this::updateStatistics); } else { - hiveOps.updateTableStatistics(tableInfo, this::updateStatistics); + hiveOps.updateTableStatistics(nameMapping, this::updateStatistics); } done = true; } @@ -498,17 +499,17 @@ public void undo(HiveMetadataOps hmsOps) { return; } if (partitionName.isPresent()) { - hmsOps.updatePartitionStatistics(tableInfo, partitionName.get(), this::resetStatistics); + hmsOps.updatePartitionStatistics(nameMapping, partitionName.get(), this::resetStatistics); } else { - hmsOps.updateTableStatistics(tableInfo, this::resetStatistics); + hmsOps.updateTableStatistics(nameMapping, this::resetStatistics); } } public String getDescription() { if (partitionName.isPresent()) { - return "alter partition parameters " + tableInfo + " " + partitionName.get(); + return "alter partition parameters " + nameMapping.getFullLocalName() + " " + partitionName.get(); } else { - return "alter table parameters " + tableInfo; + return "alter table parameters " + nameMapping.getFullRemoteName(); } } @@ -545,11 +546,11 @@ public void addPartition(HivePartitionWithStatistics partition) { public void run(HiveMetadataOps hiveOps) { HivePartition firstPartition = partitions.get(0).getPartition(); - SimpleTableInfo tableInfo = firstPartition.getTableInfo(); + NameMapping nameMapping = firstPartition.getNameMapping(); List> batchedPartitions = Lists.partition(partitions, 20); for (List batch : batchedPartitions) { try { - hiveOps.addPartitions(tableInfo, batch); + hiveOps.addPartitions(nameMapping, batch); for (HivePartitionWithStatistics partition : batch) { createdPartitionValues.add(partition.getPartition().getPartitionValues()); } @@ -562,14 +563,14 @@ public void run(HiveMetadataOps hiveOps) { public List> rollback(HiveMetadataOps hiveOps) { HivePartition firstPartition = partitions.get(0).getPartition(); - SimpleTableInfo tableInfo = firstPartition.getTableInfo(); + NameMapping nameMapping = firstPartition.getNameMapping(); List> rollbackFailedPartitions = new ArrayList<>(); for (List createdPartitionValue : createdPartitionValues) { try { - hiveOps.dropPartition(tableInfo, createdPartitionValue, false); + hiveOps.dropPartition(nameMapping, createdPartitionValue, false); } catch (Throwable t) { LOG.warn("Failed to drop partition on {}.{} when rollback", - tableInfo, rollbackFailedPartitions); + nameMapping.getFullLocalName(), rollbackFailedPartitions); rollbackFailedPartitions.add(createdPartitionValue); } } @@ -899,10 +900,10 @@ public String toString() { } } - public synchronized Table getTable(SimpleTableInfo tableInfo) { - Action tableAction = tableActions.get(tableInfo); + private synchronized Table getTable(NameMapping nameMapping) { + Action tableAction = tableActions.get(nameMapping); if (tableAction == null) { - return hiveOps.getClient().getTable(tableInfo.getDbName(), tableInfo.getTbName()); + return hiveOps.getClient().getTable(nameMapping.getRemoteDbName(), nameMapping.getRemoteTblName()); } switch (tableAction.getType()) { case ADD: @@ -916,21 +917,21 @@ public synchronized Table getTable(SimpleTableInfo tableInfo) { default: throw new IllegalStateException("Unknown action type: " + tableAction.getType()); } - throw new RuntimeException("Not Found table: " + tableInfo); + throw new RuntimeException("Not Found table: " + nameMapping); } public synchronized void finishChangingExistingTable( ActionType actionType, - SimpleTableInfo tableInfo, + NameMapping nameMapping, String location, List fileNames, HivePartitionStatistics statisticsUpdate, THivePartitionUpdate hivePartitionUpdate) { - Action oldTableAction = tableActions.get(tableInfo); + Action oldTableAction = tableActions.get(nameMapping); if (oldTableAction == null) { - Table table = hiveOps.getClient().getTable(tableInfo.getDbName(), tableInfo.getTbName()); + Table table = hiveOps.getClient().getTable(nameMapping.getRemoteDbName(), nameMapping.getRemoteTblName()); tableActions.put( - tableInfo, + nameMapping, new Action<>( actionType, new TableAndMore( @@ -944,7 +945,7 @@ public synchronized void finishChangingExistingTable( switch (oldTableAction.getType()) { case DROP: - throw new RuntimeException("Not found table: " + tableInfo); + throw new RuntimeException("Not found table: " + nameMapping.getFullLocalName()); case ADD: case ALTER: case INSERT_EXISTING: @@ -960,28 +961,28 @@ public synchronized void finishChangingExistingTable( } public synchronized void createTable( - SimpleTableInfo tableInfo, + NameMapping nameMapping, Table table, String location, List fileNames, HivePartitionStatistics statistics, THivePartitionUpdate hivePartitionUpdate) { // When creating a table, it should never have partition actions. This is just a sanity check. - checkNoPartitionAction(tableInfo); - Action oldTableAction = tableActions.get(tableInfo); + checkNoPartitionAction(nameMapping); + Action oldTableAction = tableActions.get(nameMapping); TableAndMore tableAndMore = new TableAndMore(table, location, fileNames, statistics, hivePartitionUpdate); if (oldTableAction == null) { - tableActions.put(tableInfo, new Action<>(ActionType.ADD, tableAndMore)); + tableActions.put(nameMapping, new Action<>(ActionType.ADD, tableAndMore)); return; } switch (oldTableAction.getType()) { case DROP: - tableActions.put(tableInfo, new Action<>(ActionType.ALTER, tableAndMore)); + tableActions.put(nameMapping, new Action<>(ActionType.ALTER, tableAndMore)); return; case ADD: case ALTER: case INSERT_EXISTING: case MERGE: - throw new RuntimeException("Table already exists: " + tableInfo); + throw new RuntimeException("Table already exists: " + nameMapping.getFullLocalName()); case DROP_PRESERVE_DATA: break; default: @@ -990,17 +991,17 @@ public synchronized void createTable( } - public synchronized void dropTable(SimpleTableInfo tableInfo) { + public synchronized void dropTable(NameMapping nameMapping) { // Dropping table with partition actions requires cleaning up staging data, which is not implemented yet. - checkNoPartitionAction(tableInfo); - Action oldTableAction = tableActions.get(tableInfo); + checkNoPartitionAction(nameMapping); + Action oldTableAction = tableActions.get(nameMapping); if (oldTableAction == null || oldTableAction.getType() == ActionType.ALTER) { - tableActions.put(tableInfo, new Action<>(ActionType.DROP, null)); + tableActions.put(nameMapping, new Action<>(ActionType.DROP, null)); return; } switch (oldTableAction.getType()) { case DROP: - throw new RuntimeException("Not found table: " + tableInfo); + throw new RuntimeException("Not found table: " + nameMapping.getFullLocalName()); case ADD: case ALTER: case INSERT_EXISTING: @@ -1014,9 +1015,9 @@ public synchronized void dropTable(SimpleTableInfo tableInfo) { } - private void checkNoPartitionAction(SimpleTableInfo tableInfo) { + private void checkNoPartitionAction(NameMapping nameMapping) { Map, Action> partitionActionsForTable = - partitionActions.get(tableInfo); + partitionActions.get(nameMapping); if (partitionActionsForTable != null && !partitionActionsForTable.isEmpty()) { throw new RuntimeException( "Cannot make schema changes to a table with modified partitions in the same transaction"); @@ -1024,7 +1025,7 @@ private void checkNoPartitionAction(SimpleTableInfo tableInfo) { } public synchronized void addPartition( - SimpleTableInfo tableInfo, + NameMapping nameMapping, HivePartition partition, String currentLocation, String partitionName, @@ -1032,7 +1033,7 @@ public synchronized void addPartition( HivePartitionStatistics statistics, THivePartitionUpdate hivePartitionUpdate) { Map, Action> partitionActionsForTable = - partitionActions.computeIfAbsent(tableInfo, k -> new HashMap<>()); + partitionActions.computeIfAbsent(nameMapping, k -> new HashMap<>()); Action oldPartitionAction = partitionActionsForTable.get(partition.getPartitionValues()); if (oldPartitionAction == null) { partitionActionsForTable.put( @@ -1061,19 +1062,19 @@ public synchronized void addPartition( case MERGE: throw new RuntimeException( "Partition already exists for table: " - + tableInfo + ", partition values: " + partition + + nameMapping.getFullLocalName() + ", partition values: " + partition .getPartitionValues()); default: throw new IllegalStateException("Unknown action type: " + oldPartitionAction.getType()); } } - public synchronized void dropPartition( - SimpleTableInfo tableInfo, + private synchronized void dropPartition( + NameMapping nameMapping, List partitionValues, boolean deleteData) { Map, Action> partitionActionsForTable = - partitionActions.computeIfAbsent(tableInfo, k -> new HashMap<>()); + partitionActions.computeIfAbsent(nameMapping, k -> new HashMap<>()); Action oldPartitionAction = partitionActionsForTable.get(partitionValues); if (oldPartitionAction == null) { if (deleteData) { @@ -1087,7 +1088,7 @@ public synchronized void dropPartition( case DROP: case DROP_PRESERVE_DATA: throw new RuntimeException( - "Not found partition from partition actions for " + tableInfo + "Not found partition from partition actions for " + nameMapping.getFullLocalName() + ", partitions: " + partitionValues); case ADD: case ALTER: @@ -1095,7 +1096,7 @@ public synchronized void dropPartition( case MERGE: throw new RuntimeException( "Dropping a partition added in the same transaction is not supported: " - + tableInfo + ", partition values: " + partitionValues); + + nameMapping.getFullLocalName() + ", partition values: " + partitionValues); default: throw new IllegalStateException("Unknown action type: " + oldPartitionAction.getType()); } @@ -1152,11 +1153,11 @@ private void undoAddPartitionsTask() { } HivePartition firstPartition = addPartitionsTask.getPartitions().get(0).getPartition(); - SimpleTableInfo tableInfo = firstPartition.getTableInfo(); + NameMapping nameMapping = firstPartition.getNameMapping(); List> rollbackFailedPartitions = addPartitionsTask.rollback(hiveOps); if (!rollbackFailedPartitions.isEmpty()) { LOG.warn("Failed to rollback: add_partition for partition values {}.{}", - tableInfo, rollbackFailedPartitions); + nameMapping.getFullLocalName(), rollbackFailedPartitions); } addPartitionsTask.clear(); } @@ -1174,7 +1175,7 @@ private void waitForAsyncFileSystemTaskSuppressThrowable() { asyncFileSystemTaskFutures.clear(); } - public void prepareInsertExistingTable(SimpleTableInfo tableInfo, TableAndMore tableAndMore) { + public void prepareInsertExistingTable(NameMapping nameMapping, TableAndMore tableAndMore) { Table table = tableAndMore.getTable(); String targetPath = table.getSd().getLocation(); String writePath = tableAndMore.getCurrentLocation(); @@ -1195,14 +1196,14 @@ public void prepareInsertExistingTable(SimpleTableInfo tableInfo, TableAndMore t directoryCleanUpTasksForAbort.add(new DirectoryCleanUpTask(targetPath, false)); updateStatisticsTasks.add( new UpdateStatisticsTask( - tableInfo, + nameMapping, Optional.empty(), tableAndMore.getStatisticsUpdate(), true )); } - public void prepareAlterTable(SimpleTableInfo tableInfo, TableAndMore tableAndMore) { + public void prepareAlterTable(NameMapping nameMapping, TableAndMore tableAndMore) { Table table = tableAndMore.getTable(); String targetPath = table.getSd().getLocation(); String writePath = tableAndMore.getCurrentLocation(); @@ -1238,14 +1239,14 @@ public void prepareAlterTable(SimpleTableInfo tableInfo, TableAndMore tableAndMo } updateStatisticsTasks.add( new UpdateStatisticsTask( - tableInfo, + nameMapping, Optional.empty(), tableAndMore.getStatisticsUpdate(), false )); } - public void prepareAddPartition(SimpleTableInfo tableInfo, PartitionAndMore partitionAndMore) { + public void prepareAddPartition(NameMapping nameMapping, PartitionAndMore partitionAndMore) { HivePartition partition = partitionAndMore.getPartition(); String targetPath = partition.getPath(); @@ -1266,10 +1267,10 @@ public void prepareAddPartition(SimpleTableInfo tableInfo, PartitionAndMore part } } - StorageDescriptor sd = getTable(tableInfo).getSd(); + StorageDescriptor sd = getTable(nameMapping).getSd(); HivePartition hivePartition = new HivePartition( - tableInfo, + nameMapping, false, sd.getInputFormat(), targetPath, @@ -1288,7 +1289,7 @@ public void prepareAddPartition(SimpleTableInfo tableInfo, PartitionAndMore part addPartitionsTask.addPartition(partitionWithStats); } - public void prepareInsertExistPartition(SimpleTableInfo tableInfo, PartitionAndMore partitionAndMore) { + public void prepareInsertExistPartition(NameMapping nameMapping, PartitionAndMore partitionAndMore) { HivePartition partition = partitionAndMore.getPartition(); String targetPath = partition.getPath(); @@ -1312,7 +1313,7 @@ public void prepareInsertExistPartition(SimpleTableInfo tableInfo, PartitionAndM updateStatisticsTasks.add( new UpdateStatisticsTask( - tableInfo, + nameMapping, Optional.of(partitionAndMore.getPartitionName()), partitionAndMore.getStatisticsUpdate(), true)); @@ -1357,7 +1358,7 @@ private void runS3cleanWhenSuccess() { } } - public void prepareAlterPartition(SimpleTableInfo tableInfo, PartitionAndMore partitionAndMore) { + public void prepareAlterPartition(NameMapping nameMapping, PartitionAndMore partitionAndMore) { HivePartition partition = partitionAndMore.getPartition(); String targetPath = partition.getPath(); String writePath = partitionAndMore.getCurrentLocation(); @@ -1396,7 +1397,7 @@ public void prepareAlterPartition(SimpleTableInfo tableInfo, PartitionAndMore pa updateStatisticsTasks.add( new UpdateStatisticsTask( - tableInfo, + nameMapping, Optional.of(partitionAndMore.getPartitionName()), partitionAndMore.getStatisticsUpdate(), false diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveDlaTable.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveDlaTable.java index cb6d113458a5bc..eeccc021ae6f55 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveDlaTable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveDlaTable.java @@ -74,7 +74,7 @@ public MTMVSnapshotIf getPartitionSnapshot(String partitionName, MTMVRefreshCont HiveMetaStoreCache cache = Env.getCurrentEnv().getExtMetaCacheMgr() .getMetaStoreCache((HMSExternalCatalog) hmsTable.getCatalog()); HiveMetaStoreCache.HivePartitionValues hivePartitionValues = cache.getPartitionValues( - hmsTable.getDbName(), hmsTable.getName(), hmsTable.getPartitionColumnTypes(snapshot)); + hmsTable, hmsTable.getPartitionColumnTypes(snapshot)); Long partitionId = getPartitionIdByNameOrAnalysisException(partitionName, hivePartitionValues); HivePartition hivePartition = getHivePartitionByIdOrAnalysisException(partitionId, hivePartitionValues, cache); @@ -98,8 +98,8 @@ public MTMVSnapshotIf getTableSnapshot(Optional snapshot) throws A HiveMetaStoreCache cache = Env.getCurrentEnv().getExtMetaCacheMgr() .getMetaStoreCache((HMSExternalCatalog) hmsTable.getCatalog()); HiveMetaStoreCache.HivePartitionValues hivePartitionValues = cache.getPartitionValues( - hmsTable.getDbName(), hmsTable.getName(), hmsTable.getPartitionColumnTypes(snapshot)); - List partitionList = cache.getAllPartitionsWithCache(hmsTable.getDbName(), hmsTable.getName(), + hmsTable, hmsTable.getPartitionColumnTypes(snapshot)); + List partitionList = cache.getAllPartitionsWithCache(hmsTable, Lists.newArrayList(hivePartitionValues.getPartitionValuesMap().values())); if (CollectionUtils.isEmpty(partitionList)) { throw new AnalysisException("partitionList is empty, table name: " + hmsTable.getName()); @@ -132,7 +132,7 @@ private HivePartition getHivePartitionByIdOrAnalysisException(Long partitionId, if (CollectionUtils.isEmpty(partitionValues)) { throw new AnalysisException("can not find partitionValues: " + partitionId); } - HivePartition partition = cache.getHivePartition(hmsTable.getDbName(), hmsTable.getName(), partitionValues); + HivePartition partition = cache.getHivePartition(hmsTable, partitionValues); if (partition == null) { throw new AnalysisException("can not find partition: " + partitionId); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveMetaStoreCache.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveMetaStoreCache.java index eac8e1bea2f7a0..d24b5a643c4201 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveMetaStoreCache.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveMetaStoreCache.java @@ -39,6 +39,8 @@ import org.apache.doris.datasource.CacheException; import org.apache.doris.datasource.ExternalCatalog; import org.apache.doris.datasource.ExternalMetaCacheMgr; +import org.apache.doris.datasource.ExternalTable; +import org.apache.doris.datasource.NameMapping; import org.apache.doris.fs.DirectoryLister; import org.apache.doris.fs.FileSystemCache; import org.apache.doris.fs.FileSystemDirectoryLister; @@ -238,7 +240,9 @@ public Long getValue() { private HivePartitionValues loadPartitionValues(PartitionValueCacheKey key) { // partition name format: nation=cn/city=beijing,`listPartitionNames` returned string is the encoded string. - List partitionNames = catalog.getClient().listPartitionNames(key.dbName, key.tblName); + NameMapping nameMapping = key.nameMapping; + List partitionNames = catalog.getClient() + .listPartitionNames(nameMapping.getRemoteDbName(), nameMapping.getRemoteTblName()); if (LOG.isDebugEnabled()) { LOG.debug("load #{} partitions for {} in catalog {}", partitionNames.size(), key, catalog.getName()); } @@ -246,7 +250,8 @@ private HivePartitionValues loadPartitionValues(PartitionValueCacheKey key) { BiMap partitionNameToIdMap = HashBiMap.create(partitionNames.size()); Map> idToUniqueIdsMap = Maps.newHashMapWithExpectedSize(partitionNames.size()); for (String partitionName : partitionNames) { - long partitionId = Util.genIdByName(catalog.getName(), key.dbName, key.tblName, partitionName); + long partitionId = Util.genIdByName(catalog.getName(), nameMapping.getLocalDbName(), + nameMapping.getLocalTblName(), partitionName); ListPartitionItem listPartitionItem = toListPartitionItem(partitionName, key.types); idToPartitionItem.put(partitionId, listPartitionItem); partitionNameToIdMap.put(partitionName, partitionId); @@ -291,14 +296,16 @@ public ListPartitionItem toListPartitionItem(String partitionName, List ty } private HivePartition loadPartition(PartitionCacheKey key) { - Partition partition = catalog.getClient().getPartition(key.dbName, key.tblName, key.values); + NameMapping nameMapping = key.nameMapping; + Partition partition = catalog.getClient() + .getPartition(nameMapping.getRemoteDbName(), nameMapping.getRemoteTblName(), key.values); StorageDescriptor sd = partition.getSd(); if (LOG.isDebugEnabled()) { LOG.debug("load partition format: {}, location: {} for {} in catalog {}", sd.getInputFormat(), sd.getLocation(), key, catalog.getName()); } // TODO: more info? - return new HivePartition(key.dbName, key.tblName, false, sd.getInputFormat(), sd.getLocation(), key.values, + return new HivePartition(nameMapping, false, sd.getInputFormat(), sd.getLocation(), key.values, partition.getParameters()); } @@ -307,11 +314,13 @@ private Map loadPartitions(Iterable partitionColumns = ((HMSExternalTable) - (catalog.getDbNullable(dbName).getTableNullable(tblName))).getPartitionColumns(); + (catalog.getDbNullable(localDbName).getTableNullable(localTblName))).getPartitionColumns(); // A partitionName is like "country=China/city=Beijing" or "date=2023-02-01" List partitionNames = Streams.stream(keys).map(key -> { StringBuilder sb = new StringBuilder(); @@ -326,12 +335,15 @@ private Map loadPartitions(Iterable partitions = catalog.getClient().getPartitions(dbName, tblName, partitionNames); + + + List partitions = catalog.getClient() + .getPartitions(nameMapping.getRemoteDbName(), nameMapping.getRemoteTblName(), partitionNames); // Compose the return result map. for (Partition partition : partitions) { StorageDescriptor sd = partition.getSd(); - ret.put(new PartitionCacheKey(dbName, tblName, partition.getValues()), - new HivePartition(dbName, tblName, false, + ret.put(new PartitionCacheKey(nameMapping, partition.getValues()), + new HivePartition(nameMapping, false, sd.getInputFormat(), sd.getLocation(), partition.getValues(), partition.getParameters())); } return ret; @@ -417,8 +429,8 @@ private FileCacheValue loadFiles(FileCacheKey key, DirectoryLister directoryList } } - public HivePartitionValues getPartitionValues(String dbName, String tblName, List types) { - PartitionValueCacheKey key = new PartitionValueCacheKey(dbName, tblName, types); + public HivePartitionValues getPartitionValues(ExternalTable dorisTable, List types) { + PartitionValueCacheKey key = new PartitionValueCacheKey(dorisTable.getOrBuildNameMapping(), types); return getPartitionValues(key); } @@ -444,10 +456,16 @@ public List getFilesByPartitions(List partitions, DirectoryLister directoryLister, TableIf table) { long start = System.currentTimeMillis(); + if (partitions.isEmpty()) { + return Lists.newArrayList(); + } + HivePartition firstPartition = partitions.get(0); + long fileId = Util.genIdByName(firstPartition.getNameMapping().getLocalDbName(), + firstPartition.getNameMapping().getLocalTblName()); List keys = partitions.stream().map(p -> p.isDummyPartition() ? FileCacheKey.createDummyCacheKey( - p.getDbName(), p.getTblName(), p.getPath(), p.getInputFormat()) - : new FileCacheKey(p.getDbName(), p.getTblName(), p.getPath(), + fileId, p.getPath(), p.getInputFormat()) + : new FileCacheKey(fileId, p.getPath(), p.getInputFormat(), p.getPartitionValues())) .collect(Collectors.toList()); @@ -485,25 +503,26 @@ public List getFilesByPartitions(List partitions, return fileLists; } - public HivePartition getHivePartition(String dbName, String name, List partitionValues) { - return partitionCache.get(new PartitionCacheKey(dbName, name, partitionValues)); + public HivePartition getHivePartition(ExternalTable dorisTable, List partitionValues) { + return partitionCache.get(new PartitionCacheKey(dorisTable.getOrBuildNameMapping(), partitionValues)); } - public List getAllPartitionsWithCache(String dbName, String name, + public List getAllPartitionsWithCache(ExternalTable dorisTable, List> partitionValuesList) { - return getAllPartitions(dbName, name, partitionValuesList, true); + return getAllPartitions(dorisTable, partitionValuesList, true); } - public List getAllPartitionsWithoutCache(String dbName, String name, + public List getAllPartitionsWithoutCache(ExternalTable dorisTable, List> partitionValuesList) { - return getAllPartitions(dbName, name, partitionValuesList, false); + return getAllPartitions(dorisTable, partitionValuesList, false); } - private List getAllPartitions(String dbName, String name, List> partitionValuesList, + private List getAllPartitions(ExternalTable dorisTable, List> partitionValuesList, boolean withCache) { long start = System.currentTimeMillis(); + NameMapping nameMapping = dorisTable.getOrBuildNameMapping(); List keys = partitionValuesList.stream() - .map(p -> new PartitionCacheKey(dbName, name, p)) + .map(p -> new PartitionCacheKey(nameMapping, p)) .collect(Collectors.toList()); List partitions; @@ -521,14 +540,14 @@ private List getAllPartitions(String dbName, String name, List
  • { - if (k.isSameTable(dbName, tblName)) { + if (k.isSameTable(nameMapping.getLocalDbName(), nameMapping.getLocalTblName())) { partitionCache.invalidate(k); } }); - long id = Util.genIdByName(dbName, tblName); + long id = Util.genIdByName(nameMapping.getLocalDbName(), nameMapping.getLocalTblName()); LoadingCache fileCache = fileCacheRef.get(); fileCache.asMap().keySet().forEach(k -> { if (k.isSameTable(id)) { @@ -537,16 +556,18 @@ public void invalidateTableCache(String dbName, String tblName) { }); } - public void invalidatePartitionCache(String dbName, String tblName, String partitionName) { - PartitionValueCacheKey key = new PartitionValueCacheKey(dbName, tblName, null); + public void invalidatePartitionCache(ExternalTable dorisTable, String partitionName) { + NameMapping nameMapping = dorisTable.getOrBuildNameMapping(); + long id = Util.genIdByName(nameMapping.getLocalDbName(), nameMapping.getLocalTblName()); + PartitionValueCacheKey key = new PartitionValueCacheKey(nameMapping, null); HivePartitionValues partitionValues = partitionValuesCache.getIfPresent(key); if (partitionValues != null) { Long partitionId = partitionValues.partitionNameToIdMap.get(partitionName); List values = partitionValues.partitionValuesMap.get(partitionId); - PartitionCacheKey partKey = new PartitionCacheKey(dbName, tblName, values); + PartitionCacheKey partKey = new PartitionCacheKey(nameMapping, values); HivePartition partition = partitionCache.getIfPresent(partKey); if (partition != null) { - fileCacheRef.get().invalidate(new FileCacheKey(dbName, tblName, partition.getPath(), + fileCacheRef.get().invalidate(new FileCacheKey(id, partition.getPath(), null, partition.getPartitionValues())); partitionCache.invalidate(partKey); } @@ -557,8 +578,8 @@ public void invalidateDbCache(String dbName) { long start = System.currentTimeMillis(); Set keys = partitionValuesCache.asMap().keySet(); for (PartitionValueCacheKey key : keys) { - if (key.dbName.equals(dbName)) { - invalidateTableCache(dbName, key.tblName); + if (key.nameMapping.getLocalDbName().equals(dbName)) { + invalidateTableCache(key.nameMapping); } } if (LOG.isDebugEnabled()) { @@ -577,9 +598,9 @@ public void invalidateAll() { } // partition name format: nation=cn/city=beijing - public void addPartitionsCache(String dbName, String tblName, List partitionNames, + public void addPartitionsCache(NameMapping nameMapping, List partitionNames, List partitionColumnTypes) { - PartitionValueCacheKey key = new PartitionValueCacheKey(dbName, tblName, partitionColumnTypes); + PartitionValueCacheKey key = new PartitionValueCacheKey(nameMapping, partitionColumnTypes); HivePartitionValues partitionValues = partitionValuesCache.getIfPresent(key); if (partitionValues == null) { return; @@ -589,12 +610,14 @@ public void addPartitionsCache(String dbName, String tblName, List parti Map partitionNameToIdMapBefore = copy.getPartitionNameToIdMap(); Map> idToUniqueIdsMap = copy.getIdToUniqueIdsMap(); Map idToPartitionItem = new HashMap<>(); + String localDbName = nameMapping.getLocalDbName(); + String localTblName = nameMapping.getLocalTblName(); for (String partitionName : partitionNames) { if (partitionNameToIdMapBefore.containsKey(partitionName)) { - LOG.info("addPartitionsCache partitionName:[{}] has exist in table:[{}]", partitionName, tblName); + LOG.info("addPartitionsCache partitionName:[{}] has exist in table:[{}]", partitionName, localTblName); continue; } - long partitionId = Util.genIdByName(catalog.getName(), dbName, tblName, partitionName); + long partitionId = Util.genIdByName(catalog.getName(), localDbName, localTblName, partitionName); ListPartitionItem listPartitionItem = toListPartitionItem(partitionName, key.types); idToPartitionItemBefore.put(partitionId, listPartitionItem); idToPartitionItem.put(partitionId, listPartitionItem); @@ -631,9 +654,10 @@ public void addPartitionsCache(String dbName, String tblName, List parti } } - public void dropPartitionsCache(String dbName, String tblName, List partitionNames, + public void dropPartitionsCache(ExternalTable dorisTable, List partitionNames, boolean invalidPartitionCache) { - PartitionValueCacheKey key = new PartitionValueCacheKey(dbName, tblName, null); + NameMapping nameMapping = dorisTable.getOrBuildNameMapping(); + PartitionValueCacheKey key = new PartitionValueCacheKey(nameMapping, null); HivePartitionValues partitionValues = partitionValuesCache.getIfPresent(key); if (partitionValues == null) { return; @@ -649,7 +673,8 @@ public void dropPartitionsCache(String dbName, String tblName, List part Map> partitionValuesMap = copy.getPartitionValuesMap(); for (String partitionName : partitionNames) { if (!partitionNameToIdMapBefore.containsKey(partitionName)) { - LOG.info("dropPartitionsCache partitionName:[{}] not exist in table:[{}]", partitionName, tblName); + LOG.info("dropPartitionsCache partitionName:[{}] not exist in table:[{}]", + partitionName, nameMapping.getFullLocalName()); continue; } Long partitionId = partitionNameToIdMapBefore.remove(partitionName); @@ -673,7 +698,7 @@ public void dropPartitionsCache(String dbName, String tblName, List part } if (invalidPartitionCache) { - invalidatePartitionCache(dbName, tblName, partitionName); + invalidatePartitionCache(dorisTable, partitionName); } } HivePartitionValues partitionValuesCur = partitionValuesCache.getIfPresent(key); @@ -744,14 +769,12 @@ public List getFilesByTransaction(List partitions */ @Data public static class PartitionValueCacheKey { - private String dbName; - private String tblName; + private NameMapping nameMapping; // not in key private List types; - public PartitionValueCacheKey(String dbName, String tblName, List types) { - this.dbName = dbName; - this.tblName = tblName; + public PartitionValueCacheKey(NameMapping nameMapping, List types) { + this.nameMapping = nameMapping; this.types = types; } @@ -763,30 +786,28 @@ public boolean equals(Object obj) { if (!(obj instanceof PartitionValueCacheKey)) { return false; } - return dbName.equals(((PartitionValueCacheKey) obj).dbName) - && tblName.equals(((PartitionValueCacheKey) obj).tblName); + return nameMapping.equals(((PartitionValueCacheKey) obj).nameMapping); } @Override public int hashCode() { - return Objects.hash(dbName, tblName); + return nameMapping.hashCode(); } @Override public String toString() { - return "PartitionValueCacheKey{" + "dbName='" + dbName + '\'' + ", tblName='" + tblName + '\'' + '}'; + return "PartitionValueCacheKey{" + "dbName='" + nameMapping.getLocalDbName() + '\'' + ", tblName='" + + nameMapping.getLocalTblName() + '\'' + '}'; } } @Data public static class PartitionCacheKey { - private String dbName; - private String tblName; + private NameMapping nameMapping; private List values; - public PartitionCacheKey(String dbName, String tblName, List values) { - this.dbName = dbName; - this.tblName = tblName; + public PartitionCacheKey(NameMapping nameMapping, List values) { + this.nameMapping = nameMapping; this.values = values; } @@ -798,30 +819,31 @@ public boolean equals(Object obj) { if (!(obj instanceof PartitionCacheKey)) { return false; } - return dbName.equals(((PartitionCacheKey) obj).dbName) - && tblName.equals(((PartitionCacheKey) obj).tblName) + return nameMapping.equals(((PartitionCacheKey) obj).nameMapping) && Objects.equals(values, ((PartitionCacheKey) obj).values); } boolean isSameTable(String dbName, String tblName) { - return this.dbName.equals(dbName) && this.tblName.equals(tblName); + return this.nameMapping.getLocalDbName().equals(dbName) + && this.nameMapping.getLocalTblName().equals(tblName); } @Override public int hashCode() { - return Objects.hash(dbName, tblName, values); + return Objects.hash(nameMapping, values); } @Override public String toString() { - return "PartitionCacheKey{" + "dbName='" + dbName + '\'' + ", tblName='" + tblName + '\'' + ", values=" + return "PartitionCacheKey{" + "dbName='" + nameMapping.getLocalDbName() + '\'' + + ", tblName='" + nameMapping.getLocalTblName() + '\'' + ", values=" + values + '}'; } } @Data public static class FileCacheKey { - private String dummyKey; + private long dummyKey = 0; private String location; // not in key private String inputFormat; @@ -831,18 +853,18 @@ public static class FileCacheKey { protected List partitionValues; private long id; - public FileCacheKey(String dbName, String tblName, String location, String inputFormat, + public FileCacheKey(long id, String location, String inputFormat, List partitionValues) { this.location = location; this.inputFormat = inputFormat; this.partitionValues = partitionValues == null ? Lists.newArrayList() : partitionValues; - this.id = Util.genIdByName(dbName, tblName); + this.id = id; } - public static FileCacheKey createDummyCacheKey(String dbName, String tblName, String location, + public static FileCacheKey createDummyCacheKey(long id, String location, String inputFormat) { - FileCacheKey fileCacheKey = new FileCacheKey(dbName, tblName, location, inputFormat, null); - fileCacheKey.dummyKey = dbName + "." + tblName; + FileCacheKey fileCacheKey = new FileCacheKey(id, location, inputFormat, null); + fileCacheKey.dummyKey = id; return fileCacheKey; } @@ -854,8 +876,8 @@ public boolean equals(Object obj) { if (!(obj instanceof FileCacheKey)) { return false; } - if (dummyKey != null) { - return dummyKey.equals(((FileCacheKey) obj).dummyKey); + if (dummyKey != 0) { + return dummyKey == ((FileCacheKey) obj).dummyKey; } return location.equals(((FileCacheKey) obj).location) && Objects.equals(partitionValues, ((FileCacheKey) obj).partitionValues); @@ -867,7 +889,7 @@ boolean isSameTable(long id) { @Override public int hashCode() { - if (dummyKey != null) { + if (dummyKey != 0) { return Objects.hash(dummyKey); } return Objects.hash(location, partitionValues); diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveMetaStoreClientHelper.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveMetaStoreClientHelper.java index f59143961e6cc9..e21389b7f7d3bb 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveMetaStoreClientHelper.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveMetaStoreClientHelper.java @@ -718,7 +718,7 @@ public static Type hiveTypeToDorisType(String hiveType, int timeScale) { public static String showCreateTable(HMSExternalTable hmsTable) { // Always use the latest schema HMSExternalCatalog catalog = (HMSExternalCatalog) hmsTable.getCatalog(); - Table remoteTable = catalog.getClient().getTable(hmsTable.getDbName(), hmsTable.getRemoteName()); + Table remoteTable = catalog.getClient().getTable(hmsTable.getRemoteDbName(), hmsTable.getRemoteName()); StringBuilder output = new StringBuilder(); if (remoteTable.isSetViewOriginalText() || remoteTable.isSetViewExpandedText()) { output.append(String.format("CREATE VIEW `%s` AS ", remoteTable.getTableName())); diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveMetadataOps.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveMetadataOps.java index a7dc02f2af916d..3d79bc9c31efb2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveMetadataOps.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveMetadataOps.java @@ -17,10 +17,8 @@ package org.apache.doris.datasource.hive; -import org.apache.doris.analysis.CreateDbStmt; import org.apache.doris.analysis.CreateTableStmt; import org.apache.doris.analysis.DistributionDesc; -import org.apache.doris.analysis.DropTableStmt; import org.apache.doris.analysis.HashDistributionDesc; import org.apache.doris.analysis.PartitionDesc; import org.apache.doris.catalog.Column; @@ -31,12 +29,12 @@ import org.apache.doris.common.ErrorCode; import org.apache.doris.common.ErrorReport; import org.apache.doris.common.UserException; -import org.apache.doris.common.info.SimpleTableInfo; import org.apache.doris.common.security.authentication.HadoopAuthenticator; import org.apache.doris.datasource.ExternalDatabase; +import org.apache.doris.datasource.ExternalTable; +import org.apache.doris.datasource.NameMapping; import org.apache.doris.datasource.operations.ExternalMetadataOps; import org.apache.doris.datasource.property.constants.HMSProperties; -import org.apache.doris.nereids.trees.plans.commands.CreateDatabaseCommand; 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.DropBranchInfo; @@ -96,51 +94,21 @@ private static HMSCachedClient createCachedClient(HiveConf hiveConf, int thriftC } @Override - public void createDbImpl(CreateDbStmt stmt) throws DdlException { - String fullDbName = stmt.getFullDbName(); - Map properties = stmt.getProperties(); - long dbId = Env.getCurrentEnv().getNextId(); - if (databaseExist(fullDbName)) { - if (stmt.isSetIfNotExists()) { - LOG.info("create database[{}] which already exists", fullDbName); - return; - } else { - ErrorReport.reportDdlException(ErrorCode.ERR_DB_CREATE_EXISTS, fullDbName); - } - } - try { - HiveDatabaseMetadata catalogDatabase = new HiveDatabaseMetadata(); - catalogDatabase.setDbName(fullDbName); - if (properties.containsKey(LOCATION_URI_KEY)) { - catalogDatabase.setLocationUri(properties.get(LOCATION_URI_KEY)); - } - // remove it when set - properties.remove(LOCATION_URI_KEY); - catalogDatabase.setProperties(properties); - catalogDatabase.setComment(properties.getOrDefault("comment", "")); - client.createDatabase(catalogDatabase); - } catch (Exception e) { - throw new RuntimeException(e.getMessage(), e); - } - LOG.info("createDb dbName = " + fullDbName + ", id = " + dbId); - } - - @Override - public void createDbImpl(CreateDatabaseCommand command) throws DdlException { - String fullDbName = command.getDbName(); - Map properties = command.getProperties(); - long dbId = Env.getCurrentEnv().getNextId(); - if (databaseExist(fullDbName)) { - if (command.isIfNotExists()) { - LOG.info("create database[{}] which already exists", fullDbName); - return; + public boolean createDbImpl(String dbName, boolean ifNotExists, Map properties) + throws DdlException { + ExternalDatabase dorisDb = catalog.getDbNullable(dbName); + boolean exists = databaseExist(dbName); + if (dorisDb != null || exists) { + if (ifNotExists) { + LOG.info("create database[{}] which already exists", dbName); + return true; } else { - ErrorReport.reportDdlException(ErrorCode.ERR_DB_CREATE_EXISTS, fullDbName); + ErrorReport.reportDdlException(ErrorCode.ERR_DB_CREATE_EXISTS, dbName); } } try { HiveDatabaseMetadata catalogDatabase = new HiveDatabaseMetadata(); - catalogDatabase.setDbName(fullDbName); + catalogDatabase.setDbName(dbName); if (properties.containsKey(LOCATION_URI_KEY)) { catalogDatabase.setLocationUri(properties.get(LOCATION_URI_KEY)); } @@ -149,20 +117,22 @@ public void createDbImpl(CreateDatabaseCommand command) throws DdlException { catalogDatabase.setProperties(properties); catalogDatabase.setComment(properties.getOrDefault("comment", "")); client.createDatabase(catalogDatabase); + LOG.info("successfully create hive database: {}", dbName); + return false; } catch (Exception e) { throw new RuntimeException(e.getMessage(), e); } - LOG.info("createDb dbName = " + fullDbName + ", id = " + dbId); } @Override - public void afterCreateDb(String dbName) { + public void afterCreateDb() { catalog.onRefreshCache(true); } @Override public void dropDbImpl(String dbName, boolean ifExists, boolean force) throws DdlException { - if (!databaseExist(dbName)) { + ExternalDatabase dorisDb = catalog.getDbNullable(dbName); + if (dorisDb == null) { if (ifExists) { LOG.info("drop database[{}] which does not exist", dbName); return; @@ -173,15 +143,23 @@ public void dropDbImpl(String dbName, boolean ifExists, boolean force) throws Dd try { if (force) { // try to drop all tables in the database - List tables = listTableNames(dbName); - for (String table : tables) { - dropTableImpl(dbName, table, true); + List remoteTableNames = listTableNames(dorisDb.getRemoteName()); + for (String remoteTableName : remoteTableNames) { + ExternalTable tbl = null; + try { + tbl = (ExternalTable) dorisDb.getTableOrDdlException(remoteTableName); + } catch (DdlException e) { + LOG.warn("failed to get table when force drop database [{}], table[{}], error: {}", + dbName, remoteTableName, e.getMessage()); + continue; + } + dropTableImpl(tbl, true); } - if (!tables.isEmpty()) { - LOG.info("drop database[{}] with force, drop all tables, num: {}", dbName, tables.size()); + if (!remoteTableNames.isEmpty()) { + LOG.info("drop database[{}] with force, drop all tables, num: {}", dbName, remoteTableNames.size()); } } - client.dropDatabase(dbName); + client.dropDatabase(dorisDb.getRemoteName()); } catch (Exception e) { throw new RuntimeException(e.getMessage(), e); } @@ -200,7 +178,7 @@ public boolean createTableImpl(CreateTableStmt stmt) throws UserException { if (db == null) { throw new UserException("Failed to get database: '" + dbName + "' in catalog: " + catalog.getName()); } - if (tableExist(dbName, tblName)) { + if (tableExist(db.getRemoteName(), tblName)) { if (stmt.isSetIfNotExists()) { LOG.info("create table[{}] which already exists", tblName); return true; @@ -273,12 +251,12 @@ In hive, this table only can insert not update(not report error,but not actually if (bucketInfo != null) { if (Config.enable_create_hive_bucket_table) { if (bucketInfo instanceof HashDistributionDesc) { - hiveTableMeta = HiveTableMetadata.of(dbName, + hiveTableMeta = HiveTableMetadata.of(db.getRemoteName(), tblName, location, stmt.getColumns(), partitionColNames, - ((HashDistributionDesc) bucketInfo).getDistributionColumnNames(), + bucketInfo.getDistributionColumnNames(), bucketInfo.getBuckets(), ddlProps, fileFormat, @@ -291,7 +269,7 @@ In hive, this table only can insert not update(not report error,but not actually + " set enable_create_hive_bucket_table to true"); } } else { - hiveTableMeta = HiveTableMetadata.of(dbName, + hiveTableMeta = HiveTableMetadata.of(db.getRemoteName(), tblName, location, stmt.getColumns(), @@ -301,53 +279,37 @@ In hive, this table only can insert not update(not report error,but not actually comment); } client.createTable(hiveTableMeta, stmt.isSetIfNotExists()); + return false; } catch (Exception e) { throw new UserException(e.getMessage(), e); } - return false; } @Override public void afterCreateTable(String dbName, String tblName) { - ExternalDatabase db = catalog.getDbNullable(dbName); - if (db != null) { - db.setUnInitialized(true); - } - } - - @Override - public void dropTableImpl(DropTableStmt stmt) throws DdlException { - if (stmt == null) { - throw new DdlException("DropTableStmt is null"); + Optional> db = catalog.getDbForReplay(dbName); + if (db.isPresent()) { + db.get().setUnInitialized(true); } - dropTableImpl(stmt.getDbName(), stmt.getTableName(), stmt.isSetIfExists()); } @Override - public void dropTableImpl(String dbName, String tblName, boolean ifExists) throws DdlException { - ExternalDatabase db = catalog.getDbNullable(dbName); - if (db == null) { + public void dropTableImpl(ExternalTable dorisTable, boolean ifExists) throws DdlException { + if (!tableExist(dorisTable.getRemoteDbName(), dorisTable.getRemoteName())) { if (ifExists) { - LOG.info("database [{}] does not exist when drop table[{}]", dbName, tblName); + LOG.info("drop table[{}] which does not exist", dorisTable.getRemoteDbName()); return; } else { - ErrorReport.reportDdlException(ErrorCode.ERR_BAD_DB_ERROR, dbName); + ErrorReport.reportDdlException(ErrorCode.ERR_UNKNOWN_TABLE, + dorisTable.getRemoteName(), dorisTable.getRemoteDbName()); } } - if (!tableExist(dbName, tblName)) { - if (ifExists) { - LOG.info("drop table[{}] which does not exist", dbName); - return; - } else { - ErrorReport.reportDdlException(ErrorCode.ERR_UNKNOWN_TABLE, tblName, dbName); - } - } - if (AcidUtils.isTransactionalTable(client.getTable(dbName, tblName))) { + if (AcidUtils.isTransactionalTable(client.getTable(dorisTable.getRemoteDbName(), dorisTable.getRemoteName()))) { throw new DdlException("Not support drop hive transactional table."); } try { - client.dropTable(dbName, tblName); + client.dropTable(dorisTable.getRemoteDbName(), dorisTable.getRemoteName()); } catch (Exception e) { throw new DdlException(e.getMessage(), e); } @@ -355,21 +317,17 @@ public void dropTableImpl(String dbName, String tblName, boolean ifExists) throw @Override public void afterDropTable(String dbName, String tblName) { - ExternalDatabase db = catalog.getDbNullable(dbName); - if (db != null) { - db.setUnInitialized(true); + Optional> db = catalog.getDbForReplay(dbName); + if (db.isPresent()) { + db.get().setUnInitialized(true); } } @Override - public void truncateTableImpl(String dbName, String tblName, List partitions) + public void truncateTableImpl(ExternalTable dorisTable, List partitions) throws DdlException { - ExternalDatabase db = catalog.getDbNullable(dbName); - if (db == null) { - throw new DdlException("Failed to get database: '" + dbName + "' in catalog: " + catalog.getName()); - } try { - client.truncateTable(dbName, tblName, partitions); + client.truncateTable(dorisTable.getRemoteDbName(), dorisTable.getRemoteName(), partitions); } catch (Exception e) { throw new DdlException(e.getMessage(), e); } @@ -377,34 +335,42 @@ public void truncateTableImpl(String dbName, String tblName, List partit @Override public void afterTruncateTable(String dbName, String tblName) { - // Invalidate cache. - Env.getCurrentEnv().getExtMetaCacheMgr().invalidateTableCache(catalog.getId(), dbName, tblName); - ExternalDatabase db = catalog.getDbNullable(dbName); - if (db != null) { - db.setLastUpdateTime(System.currentTimeMillis()); - db.setUnInitialized(true); + try { + // Invalidate cache. + Optional> db = catalog.getDbForReplay(dbName); + if (db.isPresent()) { + Optional dorisTable = db.get().getTableForReplay(tblName); + if (dorisTable.isPresent()) { + Env.getCurrentEnv().getExtMetaCacheMgr().invalidateTableCache((ExternalTable) dorisTable.get()); + } + db.get().setLastUpdateTime(System.currentTimeMillis()); + db.get().setUnInitialized(true); + } + } catch (Exception e) { + LOG.warn("exception when calling afterTruncateTable for db: {}, table: {}, error: {}", + dbName, tblName, e.getMessage(), e); } } @Override - public void createOrReplaceBranchImpl(String dbName, String tblName, CreateOrReplaceBranchInfo branchInfo) + public void createOrReplaceBranchImpl(ExternalTable dorisTable, CreateOrReplaceBranchInfo branchInfo) throws UserException { throw new UserException("Not support create or replace branch in hive catalog."); } @Override - public void createOrReplaceTagImpl(String dbName, String tblName, CreateOrReplaceTagInfo tagInfo) + public void createOrReplaceTagImpl(ExternalTable dorisTable, CreateOrReplaceTagInfo tagInfo) throws UserException { throw new UserException("Not support create or replace tag in hive catalog."); } @Override - public void dropTagImpl(String dbName, String tblName, DropTagInfo tagInfo) throws UserException { + public void dropTagImpl(ExternalTable dorisTable, DropTagInfo tagInfo) throws UserException { throw new UserException("Not support drop tag in hive catalog."); } @Override - public void dropBranchImpl(String dbName, String tblName, DropBranchInfo branchInfo) throws UserException { + public void dropBranchImpl(ExternalTable dorisTable, DropBranchInfo branchInfo) throws UserException { throw new UserException("Not support drop branch in hive catalog."); } @@ -433,23 +399,25 @@ public List listDatabaseNames() { } public void updateTableStatistics( - SimpleTableInfo tableInfo, + NameMapping nameMapping, Function update) { - client.updateTableStatistics(tableInfo.getDbName(), tableInfo.getTbName(), update); + client.updateTableStatistics(nameMapping.getRemoteDbName(), nameMapping.getRemoteTblName(), update); } void updatePartitionStatistics( - SimpleTableInfo tableInfo, + NameMapping nameMapping, String partitionName, Function update) { - client.updatePartitionStatistics(tableInfo.getDbName(), tableInfo.getTbName(), partitionName, update); + client.updatePartitionStatistics(nameMapping.getRemoteDbName(), nameMapping.getRemoteTblName(), partitionName, + update); } - public void addPartitions(SimpleTableInfo tableInfo, List partitions) { - client.addPartitions(tableInfo.getDbName(), tableInfo.getTbName(), partitions); + public void addPartitions(NameMapping nameMapping, List partitions) { + client.addPartitions(nameMapping.getRemoteDbName(), nameMapping.getRemoteTblName(), partitions); } - public void dropPartition(SimpleTableInfo tableInfo, List partitionValues, boolean deleteData) { - client.dropPartition(tableInfo.getDbName(), tableInfo.getTbName(), partitionValues, deleteData); + public void dropPartition(NameMapping nameMapping, List partitionValues, boolean deleteData) { + client.dropPartition(nameMapping.getRemoteDbName(), nameMapping.getRemoteTblName(), partitionValues, + deleteData); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HivePartition.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HivePartition.java index a114c45d4ff127..59f5879ad5a18c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HivePartition.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HivePartition.java @@ -18,7 +18,7 @@ package org.apache.doris.datasource.hive; import org.apache.doris.catalog.Column; -import org.apache.doris.common.info.SimpleTableInfo; +import org.apache.doris.datasource.NameMapping; import com.google.common.base.Preconditions; import lombok.Data; @@ -32,7 +32,7 @@ public class HivePartition { public static final String LAST_MODIFY_TIME_KEY = "transient_lastDdlTime"; public static final String FILE_NUM_KEY = "numFiles"; - private SimpleTableInfo tableInfo; + private NameMapping nameMapping; private String inputFormat; private String path; private List partitionValues; @@ -43,9 +43,9 @@ public class HivePartition { private List columns; // If you want to read the data under a partition, you can use this constructor - public HivePartition(SimpleTableInfo tableInfo, boolean isDummyPartition, + public HivePartition(NameMapping nameMapping, boolean isDummyPartition, String inputFormat, String path, List partitionValues, Map parameters) { - this.tableInfo = tableInfo; + this.nameMapping = nameMapping; this.isDummyPartition = isDummyPartition; // eg: org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat this.inputFormat = inputFormat; @@ -56,29 +56,19 @@ public HivePartition(SimpleTableInfo tableInfo, boolean isDummyPartition, this.parameters = parameters; } - public HivePartition(String database, String tableName, boolean isDummyPartition, - String inputFormat, String path, List partitionValues, Map parameters) { - this(new SimpleTableInfo(database, tableName), isDummyPartition, inputFormat, path, partitionValues, - parameters); - } - // If you want to update hms with partition, then you can use this constructor, // as updating hms requires some additional information, such as outputFormat and so on - public HivePartition(SimpleTableInfo tableInfo, boolean isDummyPartition, + public HivePartition(NameMapping nameMapping, boolean isDummyPartition, String inputFormat, String path, List partitionValues, Map parameters, String outputFormat, String serde, List columns) { - this(tableInfo, isDummyPartition, inputFormat, path, partitionValues, parameters); + this(nameMapping, isDummyPartition, inputFormat, path, partitionValues, parameters); this.outputFormat = outputFormat; this.serde = serde; this.columns = columns; } - public String getDbName() { - return tableInfo.getDbName(); - } - - public String getTblName() { - return tableInfo.getTbName(); + public NameMapping getNameMapping() { + return nameMapping; } // return partition name like: nation=cn/city=beijing @@ -127,7 +117,7 @@ public long getFileNum() { @Override public String toString() { final StringBuilder sb = new StringBuilder("HivePartition{"); - sb.append("tableInfo=").append(tableInfo); + sb.append("nameMapping=").append(nameMapping); sb.append(", inputFormat='").append(inputFormat).append('\''); sb.append(", path='").append(path).append('\''); sb.append(", partitionValues=").append(partitionValues); diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveTransaction.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveTransaction.java index 7d62af29611447..81c730961c20ca 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveTransaction.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveTransaction.java @@ -63,8 +63,8 @@ public boolean isFullAcid() { public Map getValidWriteIds(HMSCachedClient client) { if (txnValidIds == null) { - TableName tableName = new TableName(hiveTable.getCatalog().getName(), hiveTable.getDbName(), - hiveTable.getName()); + TableName tableName = new TableName(hiveTable.getCatalog().getName(), hiveTable.getRemoteDbName(), + hiveTable.getRemoteName()); client.acquireSharedLock(queryId, txnId, user, tableName, partitionNames, 5000); txnValidIds = client.getValidWriteIds(tableName.getDb() + "." + tableName.getTbl(), txnId); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveUtil.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveUtil.java index ac7dcadbc265e7..059a2445d1b62c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveUtil.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveUtil.java @@ -363,8 +363,8 @@ public static Partition toMetastoreApiPartition(HivePartitionWithStatistics part public static Partition toMetastoreApiPartition(HivePartition hivePartition) { Partition result = new Partition(); - result.setDbName(hivePartition.getTableInfo().getDbName()); - result.setTableName(hivePartition.getTableInfo().getTbName()); + result.setDbName(hivePartition.getNameMapping().getRemoteDbName()); + result.setTableName(hivePartition.getNameMapping().getRemoteTblName()); result.setValues(hivePartition.getPartitionValues()); result.setSd(makeStorageDescriptorFromHivePartition(hivePartition)); result.setParameters(hivePartition.getParameters()); @@ -373,7 +373,7 @@ public static Partition toMetastoreApiPartition(HivePartition hivePartition) { public static StorageDescriptor makeStorageDescriptorFromHivePartition(HivePartition partition) { SerDeInfo serdeInfo = new SerDeInfo(); - serdeInfo.setName(partition.getTableInfo().getTbName()); + serdeInfo.setName(partition.getNameMapping().getRemoteTblName()); serdeInfo.setSerializationLib(partition.getSerde()); StorageDescriptor sd = new StorageDescriptor(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HudiDlaTable.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HudiDlaTable.java index 9a2a99093b3f1a..89f0ce8dc1b485 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HudiDlaTable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HudiDlaTable.java @@ -123,7 +123,7 @@ public HMSSchemaCacheValue getHudiSchemaCacheValue(Optional snapsh private HMSSchemaCacheValue getHudiSchemaCacheValue(long timestamp) { ExternalSchemaCache cache = Env.getCurrentEnv().getExtMetaCacheMgr().getSchemaCache(hmsTable.getCatalog()); Optional schemaCacheValue = cache.getSchemaValue( - new HudiSchemaCacheKey(hmsTable.getDbName(), hmsTable.getName(), timestamp)); + new HudiSchemaCacheKey(hmsTable.getOrBuildNameMapping(), timestamp)); if (!schemaCacheValue.isPresent()) { throw new CacheException("failed to getSchema for: %s.%s.%s.%s", null, hmsTable.getCatalog().getName(), hmsTable.getDbName(), hmsTable.getName(), timestamp); diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/IcebergDlaTable.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/IcebergDlaTable.java index ec08599fa4507c..11619452bea554 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/IcebergDlaTable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/IcebergDlaTable.java @@ -53,8 +53,7 @@ public IcebergDlaTable(HMSExternalTable table) { @Override public Map getAndCopyPartitionItems(Optional snapshot) { return Maps.newHashMap( - IcebergUtils.getOrFetchSnapshotCacheValue( - snapshot, hmsTable.getCatalog(), hmsTable.getRemoteDbName(), hmsTable.getRemoteName()) + IcebergUtils.getOrFetchSnapshotCacheValue(snapshot, hmsTable) .getPartitionInfo().getNameToPartitionItem()); } @@ -71,10 +70,9 @@ public Set getPartitionColumnNames(Optional snapshot) { @Override public List getPartitionColumns(Optional snapshot) { IcebergSnapshotCacheValue snapshotValue = - IcebergUtils.getOrFetchSnapshotCacheValue( - snapshot, hmsTable.getCatalog(), hmsTable.getRemoteDbName(), hmsTable.getRemoteName()); + IcebergUtils.getOrFetchSnapshotCacheValue(snapshot, hmsTable); IcebergSchemaCacheValue schemaValue = IcebergUtils.getSchemaCacheValue( - hmsTable.getCatalog(), hmsTable.getRemoteDbName(), hmsTable.getRemoteName(), + hmsTable, snapshotValue.getSnapshot().getSchemaId()); return schemaValue.getPartitionColumns(); } @@ -83,8 +81,7 @@ public List getPartitionColumns(Optional snapshot) { public MTMVSnapshotIf getPartitionSnapshot(String partitionName, MTMVRefreshContext context, Optional snapshot) throws AnalysisException { IcebergSnapshotCacheValue snapshotValue = - IcebergUtils.getOrFetchSnapshotCacheValue( - snapshot, hmsTable.getCatalog(), hmsTable.getRemoteDbName(), hmsTable.getRemoteName()); + IcebergUtils.getOrFetchSnapshotCacheValue(snapshot, hmsTable); long latestSnapshotId = snapshotValue.getPartitionInfo().getLatestSnapshotId(partitionName); if (latestSnapshotId <= 0) { throw new AnalysisException("can not find partition: " + partitionName); @@ -97,8 +94,7 @@ public MTMVSnapshotIf getTableSnapshot(MTMVRefreshContext context, Optional snapshot) throws AnalysisException { hmsTable.makeSureInitialized(); IcebergSnapshotCacheValue snapshotValue = - IcebergUtils.getOrFetchSnapshotCacheValue( - snapshot, hmsTable.getCatalog(), hmsTable.getRemoteDbName(), hmsTable.getRemoteName()); + IcebergUtils.getOrFetchSnapshotCacheValue(snapshot, hmsTable); return new MTMVSnapshotIdSnapshot(snapshotValue.getSnapshot().getSnapshotId()); } @@ -123,11 +118,7 @@ protected boolean isValidRelatedTable() { } isValidRelatedTable = false; Set allFields = Sets.newHashSet(); - Table table = IcebergUtils.getIcebergTable( - hmsTable.getCatalog(), - hmsTable.getRemoteDbName(), - hmsTable.getRemoteName() - ); + Table table = IcebergUtils.getIcebergTable(hmsTable); for (PartitionSpec spec : table.specs().values()) { if (spec == null) { isValidRelatedTableCached = true; diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/source/HiveScanNode.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/source/HiveScanNode.java index 1d4e8cfdbf076e..b92379967216ac 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/source/HiveScanNode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/source/HiveScanNode.java @@ -150,12 +150,11 @@ protected List getPartitions() throws AnalysisException { partitionValuesList.add( ((ListPartitionItem) item).getItems().get(0).getPartitionValuesAsStringListForHive()); } - resPartitions = cache.getAllPartitionsWithCache(hmsTable.getDbName(), hmsTable.getName(), - partitionValuesList); + resPartitions = cache.getAllPartitionsWithCache(hmsTable, partitionValuesList); } else { // non partitioned table, create a dummy partition to save location and inputformat, // so that we can unify the interface. - HivePartition dummyPartition = new HivePartition(hmsTable.getDbName(), hmsTable.getName(), true, + HivePartition dummyPartition = new HivePartition(hmsTable.getOrBuildNameMapping(), true, hmsTable.getRemoteTable().getSd().getInputFormat(), hmsTable.getRemoteTable().getSd().getLocation(), null, Maps.newHashMap()); this.totalPartitionNum = 1; diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/HudiSchemaCacheKey.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/HudiSchemaCacheKey.java index 5a5b0dc044eb81..2b0d125494533b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/HudiSchemaCacheKey.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/HudiSchemaCacheKey.java @@ -18,6 +18,7 @@ package org.apache.doris.datasource.hudi; import org.apache.doris.datasource.ExternalSchemaCache.SchemaCacheKey; +import org.apache.doris.datasource.NameMapping; import com.google.common.base.Objects; @@ -31,13 +32,12 @@ public class HudiSchemaCacheKey extends SchemaCacheKey { /** * Creates a new cache key for Hudi table schemas. * - * @param dbName The database name - * @param tableName The table name + * @param nameMapping * @param timestamp The timestamp for schema version * @throws IllegalArgumentException if dbName or tableName is null or empty */ - public HudiSchemaCacheKey(String dbName, String tableName, long timestamp) { - super(dbName, tableName); + public HudiSchemaCacheKey(NameMapping nameMapping, long timestamp) { + super(nameMapping); if (timestamp < 0) { throw new IllegalArgumentException("Timestamp cannot be negative"); } @@ -77,6 +77,6 @@ public int hashCode() { @Override public String toString() { return String.format("HudiSchemaCacheKey{dbName='%s', tableName='%s', timestamp=%d}", - getDbName(), getTblName(), timestamp); + getNameMapping().getLocalDbName(), getNameMapping().getLocalTblName(), timestamp); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/HudiUtils.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/HudiUtils.java index 1455c23b44c833..8edcf200488d82 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/HudiUtils.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/HudiUtils.java @@ -334,8 +334,7 @@ public static Map getSchemaInfo(InternalSchema internalSchema) public static HudiSchemaCacheValue getSchemaCacheValue(HMSExternalTable hmsTable, String queryInstant) { ExternalSchemaCache cache = Env.getCurrentEnv().getExtMetaCacheMgr().getSchemaCache(hmsTable.getCatalog()); - SchemaCacheKey key = new HudiSchemaCacheKey(hmsTable.getDbName(), hmsTable.getName(), - Long.parseLong(queryInstant)); + SchemaCacheKey key = new HudiSchemaCacheKey(hmsTable.getOrBuildNameMapping(), Long.parseLong(queryInstant)); Optional schemaCacheValue = cache.getSchemaValue(key); return (HudiSchemaCacheValue) schemaCacheValue.get(); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/HudiCachedFsViewProcessor.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/HudiCachedFsViewProcessor.java index 9516f8a0c30569..5c93f1650f992c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/HudiCachedFsViewProcessor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/HudiCachedFsViewProcessor.java @@ -20,6 +20,7 @@ import org.apache.doris.common.CacheFactory; import org.apache.doris.common.Config; import org.apache.doris.datasource.ExternalMetaCacheMgr; +import org.apache.doris.datasource.ExternalTable; import com.github.benmanes.caffeine.cache.LoadingCache; import com.google.common.collect.Maps; @@ -75,9 +76,9 @@ public void invalidateDbCache(String dbName) { }); } - public void invalidateTableCache(String dbName, String tbName) { + public void invalidateTableCache(ExternalTable dorisTable) { fsViewCache.asMap().forEach((k, v) -> { - if (k.getDbName().equals(dbName) && k.getTbName().equals(tbName)) { + if (k.getDbName().equals(dorisTable.getDbName()) && k.getTbName().equals(dorisTable.getName())) { fsViewCache.invalidate(k); } }); diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/HudiCachedMetaClientProcessor.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/HudiCachedMetaClientProcessor.java index 67bd6c72c81b99..140eed0d5ebea9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/HudiCachedMetaClientProcessor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/HudiCachedMetaClientProcessor.java @@ -20,6 +20,8 @@ import org.apache.doris.common.CacheFactory; import org.apache.doris.common.Config; import org.apache.doris.datasource.ExternalMetaCacheMgr; +import org.apache.doris.datasource.ExternalTable; +import org.apache.doris.datasource.NameMapping; import org.apache.doris.datasource.hive.HiveMetaStoreClientHelper; import com.github.benmanes.caffeine.cache.LoadingCache; @@ -54,7 +56,7 @@ public HudiCachedMetaClientProcessor(ExecutorService executor) { } private HoodieTableMetaClient createHoodieTableMetaClient(HudiCachedClientKey key) { - LOG.debug("create hudi table meta client for {}.{}", key.getDbName(), key.getTbName()); + LOG.debug("create hudi table meta client for {}.{}", key.getNameMapping().getFullLocalName()); HadoopStorageConfiguration hadoopStorageConfiguration = new HadoopStorageConfiguration(key.getConf()); return HiveMetaStoreClientHelper.ugiDoAs( key.getConf(), @@ -66,8 +68,8 @@ private HoodieTableMetaClient createHoodieTableMetaClient(HudiCachedClientKey ke } public HoodieTableMetaClient getHoodieTableMetaClient( - String dbName, String tbName, String hudiBasePath, Configuration conf) { - return hudiTableMetaClientCache.get(new HudiCachedClientKey(dbName, tbName, hudiBasePath, conf)); + NameMapping nameMapping, String hudiBasePath, Configuration conf) { + return hudiTableMetaClientCache.get(new HudiCachedClientKey(nameMapping, hudiBasePath, conf)); } public void cleanUp() { @@ -80,39 +82,34 @@ public void invalidateAll() { public void invalidateDbCache(String dbName) { hudiTableMetaClientCache.asMap().forEach((k, v) -> { - if (k.getDbName().equals(dbName)) { + if (k.getNameMapping().getLocalDbName().equals(dbName)) { hudiTableMetaClientCache.invalidate(k); } }); } - public void invalidateTableCache(String dbName, String tbName) { + public void invalidateTableCache(ExternalTable dorisTable) { hudiTableMetaClientCache.asMap().forEach((k, v) -> { - if (k.getDbName().equals(dbName) && k.getTbName().equals(tbName)) { + if (k.getNameMapping().getLocalDbName().equals(dorisTable.getDbName()) + && k.getNameMapping().getLocalTblName().equals(dorisTable.getName())) { hudiTableMetaClientCache.invalidate(k); } }); } private static class HudiCachedClientKey { - String dbName; - String tbName; + NameMapping nameMapping; String hudiBasePath; Configuration conf; - public HudiCachedClientKey(String dbName, String tbName, String hudiBasePath, Configuration conf) { - this.dbName = dbName; - this.tbName = tbName; + public HudiCachedClientKey(NameMapping nameMapping, String hudiBasePath, Configuration conf) { + this.nameMapping = nameMapping; this.hudiBasePath = hudiBasePath; this.conf = conf; } - public String getDbName() { - return dbName; - } - - public String getTbName() { - return tbName; + public NameMapping getNameMapping() { + return nameMapping; } public String getHudiBasePath() { @@ -132,13 +129,13 @@ public boolean equals(Object o) { return false; } HudiCachedClientKey that = (HudiCachedClientKey) o; - return Objects.equals(dbName, that.dbName) && Objects.equals(tbName, that.tbName) + return nameMapping.equals(that.nameMapping) && Objects.equals(hudiBasePath, that.hudiBasePath); } @Override public int hashCode() { - return Objects.hash(dbName, tbName, hudiBasePath); + return Objects.hash(nameMapping, hudiBasePath); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/HudiCachedPartitionProcessor.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/HudiCachedPartitionProcessor.java index 6ecc9e2efdd381..1db39c230a16ce 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/HudiCachedPartitionProcessor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/HudiCachedPartitionProcessor.java @@ -22,6 +22,7 @@ import org.apache.doris.common.util.Util; import org.apache.doris.datasource.CacheException; import org.apache.doris.datasource.ExternalMetaCacheMgr; +import org.apache.doris.datasource.ExternalTable; import org.apache.doris.datasource.TablePartitionValues; import org.apache.doris.datasource.TablePartitionValues.TablePartitionKey; import org.apache.doris.datasource.hive.HMSExternalCatalog; @@ -76,9 +77,10 @@ public void cleanDatabasePartitions(String dbName) { } @Override - public void cleanTablePartitions(String dbName, String tblName) { + public void cleanTablePartitions(ExternalTable dorisTable) { partitionCache.asMap().keySet().stream() - .filter(k -> k.getDbName().equals(dbName) && k.getTblName().equals(tblName)) + .filter(k -> k.getDbName().equals(dorisTable.getDbName()) + && k.getTblName().equals(dorisTable.getName())) .collect(Collectors.toList()) .forEach(partitionCache::invalidate); } @@ -150,7 +152,8 @@ public TablePartitionValues getPartitionValues(HMSExternalTable table, HoodieTab // so even if the metastore is not enabled in the Hudi table // (for example, if the Metastore is false for a Hudi table created with Flink), // we can still obtain the partition information through the HMS API. - partitionNames = catalog.getClient().listPartitionNames(table.getDbName(), table.getName()); + partitionNames = catalog.getClient() + .listPartitionNames(table.getRemoteDbName(), table.getRemoteName()); if (partitionNames.size() == 0) { LOG.warn("Failed to get partitions from hms api, switch it from hudi api."); partitionNames = getAllPartitionNames(tableMetaClient); diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/HudiMetadataCacheMgr.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/HudiMetadataCacheMgr.java index 8d921a2f3d339e..4d9e654092d6a5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/HudiMetadataCacheMgr.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/HudiMetadataCacheMgr.java @@ -18,6 +18,7 @@ package org.apache.doris.datasource.hudi.source; import org.apache.doris.datasource.CatalogIf; +import org.apache.doris.datasource.ExternalTable; import org.apache.doris.datasource.hive.HMSExternalCatalog; import com.google.common.collect.Maps; @@ -111,18 +112,19 @@ public void invalidateDbCache(long catalogId, String dbName) { } } - public void invalidateTableCache(long catalogId, String dbName, String tblName) { + public void invalidateTableCache(ExternalTable dorisTable) { + long catalogId = dorisTable.getCatalog().getId(); HudiPartitionProcessor processor = partitionProcessors.get(catalogId); if (processor != null) { - processor.cleanTablePartitions(dbName, tblName); + processor.cleanTablePartitions(dorisTable); } HudiCachedFsViewProcessor fsViewProcessor = fsViewProcessors.get(catalogId); if (fsViewProcessor != null) { - fsViewProcessor.invalidateTableCache(dbName, tblName); + fsViewProcessor.invalidateTableCache(dorisTable); } HudiCachedMetaClientProcessor metaClientProcessor = metaClientProcessors.get(catalogId); if (metaClientProcessor != null) { - metaClientProcessor.invalidateTableCache(dbName, tblName); + metaClientProcessor.invalidateTableCache(dorisTable); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/HudiPartitionProcessor.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/HudiPartitionProcessor.java index 0ab9fef951a378..cb5e2993a56c67 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/HudiPartitionProcessor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/HudiPartitionProcessor.java @@ -17,6 +17,8 @@ package org.apache.doris.datasource.hudi.source; +import org.apache.doris.datasource.ExternalTable; + import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieTimeline; @@ -36,7 +38,7 @@ public abstract class HudiPartitionProcessor { public abstract void cleanDatabasePartitions(String dbName); - public abstract void cleanTablePartitions(String dbName, String tblName); + public abstract void cleanTablePartitions(ExternalTable dorisTable); public String[] getPartitionColumns(HoodieTableMetaClient tableMetaClient) { return tableMetaClient.getTableConfig().getPartitionFields().get(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/HudiScanNode.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/HudiScanNode.java index 98101106578117..b80b869e1abd1e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/HudiScanNode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/HudiScanNode.java @@ -30,6 +30,7 @@ import org.apache.doris.common.util.FileFormatUtils; import org.apache.doris.common.util.LocationPath; import org.apache.doris.datasource.ExternalTable; +import org.apache.doris.datasource.NameMapping; import org.apache.doris.datasource.TableFormatType; import org.apache.doris.datasource.hive.HivePartition; import org.apache.doris.datasource.hive.source.HiveScanNode; @@ -302,14 +303,13 @@ private boolean canUseNativeReader() { } private List getPrunedPartitions(HoodieTableMetaClient metaClient) { + NameMapping nameMapping = hmsTable.getOrBuildNameMapping(); List partitionColumnTypes = hmsTable.getPartitionColumnTypes(MvccUtil.getSnapshotFromContext(hmsTable)); if (!partitionColumnTypes.isEmpty()) { this.totalPartitionNum = selectedPartitions.totalPartitionNum; Map prunedPartitions = selectedPartitions.selectedPartitions; this.selectedPartitionNum = prunedPartitions.size(); - String dbName = hmsTable.getDbName(); - String tblName = hmsTable.getName(); String inputFormat = hmsTable.getRemoteTable().getSd().getInputFormat(); String basePath = metaClient.getBasePathV2().toString(); @@ -318,7 +318,7 @@ private List getPrunedPartitions(HoodieTableMetaClient metaClient (key, value) -> { String path = basePath + "/" + key; hivePartitions.add(new HivePartition( - dbName, tblName, false, inputFormat, path, + nameMapping, false, inputFormat, path, ((ListPartitionItem) value).getItems().get(0).getPartitionValuesAsStringList(), Maps.newHashMap())); } @@ -328,7 +328,7 @@ private List getPrunedPartitions(HoodieTableMetaClient metaClient // unpartitioned table, create a dummy partition to save location and // inputformat, // so that we can unify the interface. - HivePartition dummyPartition = new HivePartition(hmsTable.getDbName(), hmsTable.getName(), true, + HivePartition dummyPartition = new HivePartition(nameMapping, true, hmsTable.getRemoteTable().getSd().getInputFormat(), hmsTable.getRemoteTable().getSd().getLocation(), null, Maps.newHashMap()); this.totalPartitionNum = 1; diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergDLFExternalCatalog.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergDLFExternalCatalog.java index 7f9052447bacbc..05fcf1d19d4c90 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergDLFExternalCatalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergDLFExternalCatalog.java @@ -17,9 +17,8 @@ package org.apache.doris.datasource.iceberg; -import org.apache.doris.analysis.CreateDbStmt; import org.apache.doris.analysis.CreateTableStmt; -import org.apache.doris.analysis.DropTableStmt; +import org.apache.doris.analysis.PartitionNames; import org.apache.doris.common.DdlException; import org.apache.doris.common.UserException; import org.apache.doris.datasource.CatalogProperty; @@ -27,8 +26,6 @@ import org.apache.doris.datasource.property.PropertyConverter; import org.apache.doris.datasource.property.constants.HMSProperties; import org.apache.doris.nereids.exceptions.NotSupportedException; -import org.apache.doris.nereids.trees.plans.commands.CreateDatabaseCommand; -import org.apache.doris.nereids.trees.plans.commands.TruncateTableCommand; import java.util.Map; @@ -55,12 +52,7 @@ protected void initCatalog() { } @Override - public void createDb(CreateDbStmt stmt) throws DdlException { - throw new NotSupportedException("iceberg catalog with dlf type not supports 'create database'"); - } - - @Override - public void createDb(CreateDatabaseCommand command) throws DdlException { + public void createDb(String dbName, boolean ifNotExists, Map properties) throws DdlException { throw new NotSupportedException("iceberg catalog with dlf type not supports 'create database'"); } @@ -74,11 +66,6 @@ public boolean createTable(CreateTableStmt stmt) throws UserException { throw new NotSupportedException("iceberg catalog with dlf type not supports 'create table'"); } - @Override - public void dropTable(DropTableStmt stmt) throws DdlException { - throw new NotSupportedException("iceberg catalog with dlf type not supports 'drop table'"); - } - @Override public void dropTable(String dbName, String tableName, boolean isView, boolean isMtmv, boolean ifExists, boolean force) throws DdlException { @@ -86,7 +73,8 @@ public void dropTable(String dbName, String tableName, boolean isView, boolean i } @Override - public void truncateTable(TruncateTableCommand command) throws DdlException { + public void truncateTable(String dbName, String tableName, PartitionNames partitionNames, boolean forceDrop, + String rawTruncateSql) throws DdlException { throw new NotSupportedException("iceberg catalog with dlf type not supports 'truncate table'"); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergExternalTable.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergExternalTable.java index 3ea824d747a891..e639a6946a7c9d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergExternalTable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergExternalTable.java @@ -97,8 +97,7 @@ public void setTable(Table table) { @Override public Optional initSchema(SchemaCacheKey key) { boolean isView = isView(); - return IcebergUtils.loadSchemaCacheValue( - getCatalog(), getRemoteDbName(), getRemoteName(), ((IcebergSchemaCacheKey) key).getSchemaId(), isView); + return IcebergUtils.loadSchemaCacheValue(this, ((IcebergSchemaCacheKey) key).getSchemaId(), isView); } @Override @@ -128,12 +127,12 @@ public BaseAnalysisTask createAnalysisTask(AnalysisInfo info) { @Override public long fetchRowCount() { makeSureInitialized(); - long rowCount = IcebergUtils.getIcebergRowCount(getCatalog(), getRemoteDbName(), getRemoteName()); + long rowCount = IcebergUtils.getIcebergRowCount(this); return rowCount > 0 ? rowCount : UNKNOWN_ROW_COUNT; } public Table getIcebergTable() { - return IcebergUtils.getIcebergTable(getCatalog(), getRemoteDbName(), getRemoteName()); + return IcebergUtils.getIcebergTable(this); } @Override @@ -143,13 +142,13 @@ public void beforeMTMVRefresh(MTMV mtmv) throws DdlException { @Override public Map getAndCopyPartitionItems(Optional snapshot) { return Maps.newHashMap( - IcebergUtils.getOrFetchSnapshotCacheValue(snapshot, getCatalog(), getRemoteDbName(), getRemoteName()) + IcebergUtils.getOrFetchSnapshotCacheValue(snapshot, this) .getPartitionInfo().getNameToPartitionItem()); } @Override public Map getNameToPartitionItems(Optional snapshot) { - return IcebergUtils.getOrFetchSnapshotCacheValue(snapshot, getCatalog(), getRemoteDbName(), getRemoteName()) + return IcebergUtils.getOrFetchSnapshotCacheValue(snapshot, this) .getPartitionInfo().getNameToPartitionItem(); } @@ -166,9 +165,9 @@ public Set getPartitionColumnNames(Optional snapshot) thro @Override public List getPartitionColumns(Optional snapshot) { IcebergSnapshotCacheValue snapshotValue = - IcebergUtils.getOrFetchSnapshotCacheValue(snapshot, getCatalog(), getRemoteDbName(), getRemoteName()); + IcebergUtils.getOrFetchSnapshotCacheValue(snapshot, this); IcebergSchemaCacheValue schemaValue = IcebergUtils.getSchemaCacheValue( - getCatalog(), getRemoteDbName(), getRemoteName(), snapshotValue.getSnapshot().getSchemaId()); + this, snapshotValue.getSnapshot().getSchemaId()); return schemaValue.getPartitionColumns(); } @@ -176,7 +175,7 @@ public List getPartitionColumns(Optional snapshot) { public MTMVSnapshotIf getPartitionSnapshot(String partitionName, MTMVRefreshContext context, Optional snapshot) throws AnalysisException { IcebergSnapshotCacheValue snapshotValue = - IcebergUtils.getOrFetchSnapshotCacheValue(snapshot, getCatalog(), getRemoteDbName(), getRemoteName()); + IcebergUtils.getOrFetchSnapshotCacheValue(snapshot, this); long latestSnapshotId = snapshotValue.getPartitionInfo().getLatestSnapshotId(partitionName); if (latestSnapshotId <= 0) { throw new AnalysisException("can not find partition: " + partitionName); @@ -193,15 +192,14 @@ public MTMVSnapshotIf getTableSnapshot(MTMVRefreshContext context, Optional snapshot) throws AnalysisException { makeSureInitialized(); - IcebergSnapshotCacheValue snapshotValue = - IcebergUtils.getOrFetchSnapshotCacheValue(snapshot, getCatalog(), getRemoteDbName(), getRemoteName()); + IcebergSnapshotCacheValue snapshotValue = IcebergUtils.getOrFetchSnapshotCacheValue(snapshot, this); return new MTMVSnapshotIdSnapshot(snapshotValue.getSnapshot().getSnapshotId()); } @Override public long getNewestUpdateVersionOrTime() { - return IcebergUtils.getIcebergSnapshotCacheValue(Optional.empty(), getCatalog(), getRemoteDbName(), - getRemoteName(), Optional.empty()).getPartitionInfo().getNameToIcebergPartition().values().stream() + return IcebergUtils.getIcebergSnapshotCacheValue(Optional.empty(), this, Optional.empty()) + .getPartitionInfo().getNameToIcebergPartition().values().stream() .mapToLong(IcebergPartition::getLastUpdateTime).max().orElse(0); } @@ -256,13 +254,13 @@ public MvccSnapshot loadSnapshot(Optional tableSnapshot, Optional return new EmptyMvccSnapshot(); } else { return new IcebergMvccSnapshot(IcebergUtils.getIcebergSnapshotCacheValue( - tableSnapshot, getCatalog(), getRemoteDbName(), getRemoteName(), scanParams)); + tableSnapshot, this, scanParams)); } } @Override public List getFullSchema() { - return IcebergUtils.getIcebergSchema(this, getCatalog(), getRemoteDbName(), getRemoteName()); + return IcebergUtils.getIcebergSchema(this); } @Override @@ -299,7 +297,7 @@ public boolean isView() { public String getViewText() { try { return catalog.getPreExecutionAuthenticator().execute(() -> { - View icebergView = IcebergUtils.getIcebergView(getCatalog(), getRemoteDbName(), getRemoteName()); + View icebergView = IcebergUtils.getIcebergView(this); ViewVersion viewVersion = icebergView.currentVersion(); if (viewVersion == null) { throw new RuntimeException(String.format("Cannot get view version for view '%s'", icebergView)); @@ -326,7 +324,7 @@ public String getViewText() { public String getSqlDialect() { try { return catalog.getPreExecutionAuthenticator().execute(() -> { - View icebergView = IcebergUtils.getIcebergView(getCatalog(), getRemoteDbName(), getRemoteName()); + View icebergView = IcebergUtils.getIcebergView(this); ViewVersion viewVersion = icebergView.currentVersion(); if (viewVersion == null) { throw new RuntimeException(String.format("Cannot get view version for view '%s'", icebergView)); @@ -347,7 +345,7 @@ public String getSqlDialect() { } public View getIcebergView() { - return IcebergUtils.getIcebergView(getCatalog(), getRemoteDbName(), getRemoteName()); + return IcebergUtils.getIcebergView(this); } /** diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergMetadataCache.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergMetadataCache.java index 67cf1d3855a390..b4373218cdac12 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergMetadataCache.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergMetadataCache.java @@ -17,12 +17,16 @@ package org.apache.doris.datasource.iceberg; +import org.apache.doris.catalog.Env; +import org.apache.doris.catalog.TableIf; import org.apache.doris.common.AnalysisException; import org.apache.doris.common.CacheFactory; import org.apache.doris.common.Config; import org.apache.doris.datasource.CatalogIf; import org.apache.doris.datasource.ExternalCatalog; import org.apache.doris.datasource.ExternalMetaCacheMgr; +import org.apache.doris.datasource.ExternalTable; +import org.apache.doris.datasource.NameMapping; import org.apache.doris.datasource.hive.HMSExternalCatalog; import org.apache.doris.mtmv.MTMVRelatedTableIf; @@ -32,7 +36,6 @@ import com.google.common.collect.Maps; import org.apache.commons.lang3.exception.ExceptionUtils; import org.apache.iceberg.ManifestFiles; -import org.apache.iceberg.SerializableTable; import org.apache.iceberg.Snapshot; import org.apache.iceberg.Table; import org.apache.iceberg.view.View; @@ -41,7 +44,6 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.Objects; import java.util.OptionalLong; import java.util.concurrent.ExecutorService; @@ -79,28 +81,23 @@ public IcebergMetadataCache(ExecutorService executor) { this.viewCache = tableCacheFactory.buildCache(key -> loadView(key), null, executor); } - public Table getIcebergTable(CatalogIf catalog, String dbName, String tbName) { - IcebergMetadataCacheKey key = IcebergMetadataCacheKey.of(catalog, dbName, tbName); + public Table getIcebergTable(ExternalTable dorisTable) { + IcebergMetadataCacheKey key = new IcebergMetadataCacheKey(dorisTable.getOrBuildNameMapping()); return tableCache.get(key); } - public Table getAndCloneTable(CatalogIf catalog, String dbName, String tbName) { - Table restTable; - synchronized (this) { - Table table = getIcebergTable(catalog, dbName, tbName); - restTable = SerializableTable.copyOf(table); - } - return restTable; + public Table getIcebergTable(IcebergMetadataCacheKey key) { + return tableCache.get(key); } - public IcebergSnapshotCacheValue getSnapshotCache(CatalogIf catalog, String dbName, String tbName) { - IcebergMetadataCacheKey key = IcebergMetadataCacheKey.of(catalog, dbName, tbName); + public IcebergSnapshotCacheValue getSnapshotCache(ExternalTable dorisTable) { + IcebergMetadataCacheKey key = new IcebergMetadataCacheKey(dorisTable.getOrBuildNameMapping()); return snapshotCache.get(key); } @NotNull private List loadSnapshots(IcebergMetadataCacheKey key) { - Table icebergTable = getIcebergTable(key.catalog, key.dbName, key.tableName); + Table icebergTable = getIcebergTable(key); List snaps = Lists.newArrayList(); Iterables.addAll(snaps, icebergTable.snapshots()); return snaps; @@ -108,17 +105,23 @@ private List loadSnapshots(IcebergMetadataCacheKey key) { @NotNull private Table loadTable(IcebergMetadataCacheKey key) { + NameMapping nameMapping = key.nameMapping; + CatalogIf catalog = Env.getCurrentEnv().getCatalogMgr().getCatalog(nameMapping.getCtlId()); + if (catalog == null) { + throw new RuntimeException(String.format("Cannot find catalog %d when loading table %s/%s.", + nameMapping.getCtlId(), nameMapping.getLocalDbName(), nameMapping.getLocalTblName())); + } IcebergMetadataOps ops; - if (key.catalog instanceof HMSExternalCatalog) { - ops = ((HMSExternalCatalog) key.catalog).getIcebergMetadataOps(); - } else if (key.catalog instanceof IcebergExternalCatalog) { - ops = (IcebergMetadataOps) (((IcebergExternalCatalog) key.catalog).getMetadataOps()); + if (catalog instanceof HMSExternalCatalog) { + ops = ((HMSExternalCatalog) catalog).getIcebergMetadataOps(); + } else if (catalog instanceof IcebergExternalCatalog) { + ops = (IcebergMetadataOps) (((IcebergExternalCatalog) catalog).getMetadataOps()); } else { throw new RuntimeException("Only support 'hms' and 'iceberg' type for iceberg table"); } try { - return ((ExternalCatalog) key.catalog).getPreExecutionAuthenticator().execute(() - -> ops.loadTable(key.dbName, key.tableName)); + return ((ExternalCatalog) catalog).getPreExecutionAuthenticator().execute(() + -> ops.loadTable(nameMapping.getRemoteDbName(), nameMapping.getRemoteTblName())); } catch (Exception e) { throw new RuntimeException(ExceptionUtils.getRootCauseMessage(e), e); } @@ -127,52 +130,65 @@ private Table loadTable(IcebergMetadataCacheKey key) { @NotNull private IcebergSnapshotCacheValue loadSnapshot(IcebergMetadataCacheKey key) throws AnalysisException { - MTMVRelatedTableIf table = (MTMVRelatedTableIf) key.catalog.getDbOrAnalysisException(key.dbName) - .getTableOrAnalysisException(key.tableName); - IcebergSnapshot lastedIcebergSnapshot = IcebergUtils.getLastedIcebergSnapshot( - (ExternalCatalog) key.catalog, key.dbName, key.tableName); + NameMapping nameMapping = key.nameMapping; + TableIf dorisTable = Env.getCurrentEnv().getCatalogMgr().getCatalogOrAnalysisException(nameMapping.getCtlId()) + .getDbOrAnalysisException(nameMapping.getLocalDbName()) + .getTableOrAnalysisException(nameMapping.getLocalTblName()); + + if (!(dorisTable instanceof MTMVRelatedTableIf)) { + throw new AnalysisException(String.format("Table %s.%s is not a valid MTMV related table.", + nameMapping.getLocalDbName(), nameMapping.getLocalTblName())); + } + + MTMVRelatedTableIf table = (MTMVRelatedTableIf) dorisTable; + IcebergSnapshot lastedIcebergSnapshot = IcebergUtils.getLastedIcebergSnapshot((ExternalTable) table); IcebergPartitionInfo icebergPartitionInfo; if (!table.isValidRelatedTable()) { icebergPartitionInfo = IcebergPartitionInfo.empty(); } else { - icebergPartitionInfo = IcebergUtils.loadPartitionInfo( - (ExternalCatalog) key.catalog, key.dbName, key.tableName, lastedIcebergSnapshot.getSnapshotId()); + icebergPartitionInfo = IcebergUtils.loadPartitionInfo((ExternalTable) table, + lastedIcebergSnapshot.getSnapshotId()); } return new IcebergSnapshotCacheValue(icebergPartitionInfo, lastedIcebergSnapshot); } public void invalidateCatalogCache(long catalogId) { snapshotListCache.asMap().keySet().stream() - .filter(key -> key.catalog.getId() == catalogId) + .filter(key -> key.nameMapping.getCtlId() == catalogId) .forEach(snapshotListCache::invalidate); tableCache.asMap().entrySet().stream() - .filter(entry -> entry.getKey().catalog.getId() == catalogId) + .filter(entry -> entry.getKey().nameMapping.getCtlId() == catalogId) .forEach(entry -> { ManifestFiles.dropCache(entry.getValue().io()); tableCache.invalidate(entry.getKey()); }); snapshotCache.asMap().keySet().stream() - .filter(key -> key.catalog.getId() == catalogId) + .filter(key -> key.nameMapping.getCtlId() == catalogId) .forEach(snapshotCache::invalidate); viewCache.asMap().entrySet().stream() - .filter(entry -> entry.getKey().catalog.getId() == catalogId) + .filter(entry -> entry.getKey().nameMapping.getCtlId() == catalogId) .forEach(entry -> viewCache.invalidate(entry.getKey())); } - public void invalidateTableCache(long catalogId, String dbName, String tblName) { + public void invalidateTableCache(ExternalTable dorisTable) { + long catalogId = dorisTable.getCatalog().getId(); + String dbName = dorisTable.getDbName(); + String tblName = dorisTable.getName(); snapshotListCache.asMap().keySet().stream() - .filter(key -> key.catalog.getId() == catalogId && key.dbName.equals(dbName) && key.tableName.equals( - tblName)) + .filter(key -> key.nameMapping.getCtlId() == catalogId + && key.nameMapping.getLocalDbName().equals(dbName) + && key.nameMapping.getLocalTblName().equals(tblName)) .forEach(snapshotListCache::invalidate); tableCache.asMap().entrySet().stream() .filter(entry -> { IcebergMetadataCacheKey key = entry.getKey(); - return key.catalog.getId() == catalogId && key.dbName.equals(dbName) && key.tableName.equals( - tblName); + return key.nameMapping.getCtlId() == catalogId + && key.nameMapping.getLocalDbName().equals(dbName) + && key.nameMapping.getLocalTblName().equals(tblName); }) .forEach(entry -> { ManifestFiles.dropCache(entry.getValue().io()); @@ -180,27 +196,31 @@ public void invalidateTableCache(long catalogId, String dbName, String tblName) }); snapshotCache.asMap().keySet().stream() - .filter(key -> key.catalog.getId() == catalogId && key.dbName.equals(dbName) && key.tableName.equals( - tblName)) + .filter(key -> key.nameMapping.getCtlId() == catalogId + && key.nameMapping.getLocalDbName().equals(dbName) + && key.nameMapping.getLocalTblName().equals(tblName)) .forEach(snapshotCache::invalidate); viewCache.asMap().entrySet().stream() .filter(entry -> { IcebergMetadataCacheKey key = entry.getKey(); - return key.catalog.getId() == catalogId && key.dbName.equals(dbName) && key.tableName.equals( - tblName); + return key.nameMapping.getCtlId() == catalogId + && key.nameMapping.getLocalDbName().equals(dbName) + && key.nameMapping.getLocalTblName().equals(tblName); }) .forEach(entry -> viewCache.invalidate(entry.getKey())); } public void invalidateDbCache(long catalogId, String dbName) { snapshotListCache.asMap().keySet().stream() - .filter(key -> key.catalog.getId() == catalogId && key.dbName.equals(dbName)) + .filter(key -> key.nameMapping.getCtlId() == catalogId + && key.nameMapping.getLocalDbName().equals(dbName)) .forEach(snapshotListCache::invalidate); tableCache.asMap().entrySet().stream() .filter(entry -> { IcebergMetadataCacheKey key = entry.getKey(); - return key.catalog.getId() == catalogId && key.dbName.equals(dbName); + return key.nameMapping.getCtlId() == catalogId + && key.nameMapping.getLocalDbName().equals(dbName); }) .forEach(entry -> { ManifestFiles.dropCache(entry.getValue().io()); @@ -208,12 +228,14 @@ public void invalidateDbCache(long catalogId, String dbName) { }); snapshotCache.asMap().keySet().stream() - .filter(key -> key.catalog.getId() == catalogId && key.dbName.equals(dbName)) + .filter(key -> key.nameMapping.getCtlId() == catalogId + && key.nameMapping.getLocalDbName().equals(dbName)) .forEach(snapshotCache::invalidate); viewCache.asMap().entrySet().stream() .filter(entry -> { IcebergMetadataCacheKey key = entry.getKey(); - return key.catalog.getId() == catalogId && key.dbName.equals(dbName); + return key.nameMapping.getCtlId() == catalogId + && key.nameMapping.getLocalDbName().equals(dbName); }) .forEach(entry -> viewCache.invalidate(entry.getKey())); } @@ -233,18 +255,10 @@ private static void initIcebergTableFileIO(Table table, Map prop } static class IcebergMetadataCacheKey { - CatalogIf catalog; - String dbName; - String tableName; - - public IcebergMetadataCacheKey(CatalogIf catalog, String dbName, String tableName) { - this.catalog = catalog; - this.dbName = dbName; - this.tableName = tableName; - } + NameMapping nameMapping; - static IcebergMetadataCacheKey of(CatalogIf catalog, String dbName, String tableName) { - return new IcebergMetadataCacheKey(catalog, dbName, tableName); + private IcebergMetadataCacheKey(NameMapping nameMapping) { + this.nameMapping = nameMapping; } @Override @@ -256,14 +270,12 @@ public boolean equals(Object o) { return false; } IcebergMetadataCacheKey that = (IcebergMetadataCacheKey) o; - return catalog.getId() == that.catalog.getId() - && Objects.equals(dbName, that.dbName) - && Objects.equals(tableName, that.tableName); + return nameMapping.equals(that.nameMapping); } @Override public int hashCode() { - return Objects.hash(catalog.getId(), dbName, tableName); + return nameMapping.hashCode(); } } @@ -280,22 +292,22 @@ public Map> getCacheStats() { private View loadView(IcebergMetadataCacheKey key) { IcebergMetadataOps ops; - if (key.catalog instanceof IcebergExternalCatalog) { - ops = (IcebergMetadataOps) (((IcebergExternalCatalog) key.catalog).getMetadataOps()); + CatalogIf catalog = Env.getCurrentEnv().getCatalogMgr().getCatalog(key.nameMapping.getCtlId()); + if (catalog instanceof IcebergExternalCatalog) { + ops = (IcebergMetadataOps) (((IcebergExternalCatalog) catalog).getMetadataOps()); } else { return null; } try { - return ((ExternalCatalog) key.catalog).getPreExecutionAuthenticator().execute(() -> - ops.loadView(key.dbName, key.tableName)); + return ((ExternalCatalog) catalog).getPreExecutionAuthenticator().execute(() -> + ops.loadView(key.nameMapping.getRemoteDbName(), key.nameMapping.getRemoteTblName())); } catch (Exception e) { throw new RuntimeException(ExceptionUtils.getRootCauseMessage(e), e); } - } - public View getIcebergView(CatalogIf catalog, String dbName, String tbName) { - IcebergMetadataCacheKey key = IcebergMetadataCacheKey.of(catalog, dbName, tbName); + public View getIcebergView(ExternalTable dorisTable) { + IcebergMetadataCacheKey key = new IcebergMetadataCacheKey(dorisTable.getOrBuildNameMapping()); return viewCache.get(key); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergMetadataCacheMgr.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergMetadataCacheMgr.java index 69a29cc2885e83..6e72dcf2fac0c5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergMetadataCacheMgr.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergMetadataCacheMgr.java @@ -17,6 +17,8 @@ package org.apache.doris.datasource.iceberg; +import org.apache.doris.datasource.ExternalTable; + import java.util.concurrent.ExecutorService; public class IcebergMetadataCacheMgr { @@ -39,8 +41,8 @@ public void invalidateCatalogCache(long catalogId) { icebergMetadataCache.invalidateCatalogCache(catalogId); } - public void invalidateTableCache(long catalogId, String dbName, String tblName) { - icebergMetadataCache.invalidateTableCache(catalogId, dbName, tblName); + public void invalidateTableCache(ExternalTable dorisTable) { + icebergMetadataCache.invalidateTableCache(dorisTable); } public void invalidateDbCache(long catalogId, String dbName) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergMetadataOps.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergMetadataOps.java index 3363602e237bbc..059e12376fb2ee 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergMetadataOps.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergMetadataOps.java @@ -17,12 +17,12 @@ package org.apache.doris.datasource.iceberg; -import org.apache.doris.analysis.CreateDbStmt; import org.apache.doris.analysis.CreateTableStmt; -import org.apache.doris.analysis.DropTableStmt; import org.apache.doris.catalog.Column; +import org.apache.doris.catalog.Env; import org.apache.doris.catalog.StructField; import org.apache.doris.catalog.StructType; +import org.apache.doris.catalog.TableIf; import org.apache.doris.common.DdlException; import org.apache.doris.common.ErrorCode; import org.apache.doris.common.ErrorReport; @@ -34,7 +34,6 @@ 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.CreateDatabaseCommand; import org.apache.doris.nereids.trees.plans.commands.info.BranchOptions; import org.apache.doris.nereids.trees.plans.commands.info.CreateOrReplaceBranchInfo; import org.apache.doris.nereids.trees.plans.commands.info.CreateOrReplaceTagInfo; @@ -135,7 +134,6 @@ public List listDatabaseNames() { } } - @Override public List listTableNames(String dbName) { try { @@ -166,66 +164,28 @@ public List listTableNames(String dbName) { } @Override - public void createDbImpl(CreateDbStmt stmt) throws DdlException { - try { - preExecutionAuthenticator.execute(() -> { - performCreateDb(stmt); - return null; - }); - } catch (Exception e) { - throw new DdlException("Failed to create database: " - + stmt.getFullDbName() + ": " + Util.getRootCauseMessage(e), e); - } - } - - @Override - public void createDbImpl(CreateDatabaseCommand command) throws DdlException { + public boolean createDbImpl(String dbName, boolean ifNotExists, Map properties) + throws DdlException { try { - preExecutionAuthenticator.execute(() -> { - performCreateDb(command); - return null; - }); + return preExecutionAuthenticator.execute(() -> performCreateDb(dbName, ifNotExists, properties)); } catch (Exception e) { throw new DdlException("Failed to create database: " - + command.getDbName() + ": " + Util.getRootCauseMessage(e), e); + + dbName + ": " + Util.getRootCauseMessage(e), e); } } @Override - public void afterCreateDb(String dbName) { + public void afterCreateDb() { dorisCatalog.onRefreshCache(true); } - private void performCreateDb(CreateDbStmt stmt) throws DdlException { + private boolean performCreateDb(String dbName, boolean ifNotExists, Map properties) + throws DdlException { SupportsNamespaces nsCatalog = (SupportsNamespaces) catalog; - String dbName = stmt.getFullDbName(); - Map properties = stmt.getProperties(); if (databaseExist(dbName)) { - if (stmt.isSetIfNotExists()) { + if (ifNotExists) { LOG.info("create database[{}] which already exists", dbName); - return; - } else { - ErrorReport.reportDdlException(ErrorCode.ERR_DB_CREATE_EXISTS, dbName); - } - } - if (!properties.isEmpty() && dorisCatalog instanceof IcebergExternalCatalog) { - String icebergCatalogType = ((IcebergExternalCatalog) dorisCatalog).getIcebergCatalogType(); - if (!IcebergExternalCatalog.ICEBERG_HMS.equals(icebergCatalogType)) { - throw new DdlException( - "Not supported: create database with properties for iceberg catalog type: " + icebergCatalogType); - } - } - nsCatalog.createNamespace(getNamespace(dbName), properties); - } - - private void performCreateDb(CreateDatabaseCommand command) throws DdlException { - SupportsNamespaces nsCatalog = (SupportsNamespaces) catalog; - String dbName = command.getDbName(); - Map properties = command.getProperties(); - if (databaseExist(dbName)) { - if (command.isIfNotExists()) { - LOG.info("create database[{}] which already exists", dbName); - return; + return true; } else { ErrorReport.reportDdlException(ErrorCode.ERR_DB_CREATE_EXISTS, dbName); } @@ -238,6 +198,7 @@ private void performCreateDb(CreateDatabaseCommand command) throws DdlException } } nsCatalog.createNamespace(getNamespace(dbName), properties); + return false; } @Override @@ -254,7 +215,8 @@ public void dropDbImpl(String dbName, boolean ifExists, boolean force) throws Dd } private void preformDropDb(String dbName, boolean ifExists, boolean force) throws DdlException { - if (!databaseExist(dbName)) { + ExternalDatabase dorisDb = dorisCatalog.getDbNullable(dbName); + if (dorisDb == null) { if (ifExists) { LOG.info("drop database[{}] which does not exist", dbName); return; @@ -264,15 +226,23 @@ private void preformDropDb(String dbName, boolean ifExists, boolean force) throw } if (force) { // try to drop all tables in the database - List tables = listTableNames(dbName); - for (String table : tables) { - performDropTable(dbName, table, true); + List remoteTableNames = listTableNames(dorisDb.getRemoteName()); + for (String remoteTableName : remoteTableNames) { + performDropTable(dorisDb.getRemoteName(), remoteTableName, true); + } + if (!remoteTableNames.isEmpty()) { + LOG.info("drop database[{}] with force, drop all tables, num: {}", dbName, remoteTableNames.size()); + } + // try to drop all views in the database + List remoteViewNames = listViewNames(dorisDb.getRemoteName()); + for (String remoteViewName : remoteViewNames) { + performDropView(dorisDb.getRemoteName(), remoteViewName); } - if (!tables.isEmpty()) { - LOG.info("drop database[{}] with force, drop all tables, num: {}", dbName, tables.size()); + if (!remoteViewNames.isEmpty()) { + LOG.info("drop database[{}] with force, drop all views, num: {}", dbName, remoteViewNames.size()); } } - nsCatalog.dropNamespace(getNamespace(dbName)); + nsCatalog.dropNamespace(getNamespace(dorisDb.getRemoteName())); } @Override @@ -283,12 +253,11 @@ public void afterDropDb(String dbName) { @Override public boolean createTableImpl(CreateTableStmt stmt) throws UserException { try { - preExecutionAuthenticator.execute(() -> performCreateTable(stmt)); + return preExecutionAuthenticator.execute(() -> performCreateTable(stmt)); } catch (Exception e) { throw new DdlException( "Failed to create table: " + stmt.getTableName() + ", error message is:" + e.getMessage(), e); } - return false; } public boolean performCreateTable(CreateTableStmt stmt) throws UserException { @@ -298,7 +267,24 @@ public boolean performCreateTable(CreateTableStmt stmt) throws UserException { throw new UserException("Failed to get database: '" + dbName + "' in catalog: " + dorisCatalog.getName()); } String tableName = stmt.getTableName(); - if (tableExist(dbName, tableName)) { + // 1. first, check if table exist in remote + if (tableExist(db.getRemoteName(), tableName)) { + if (stmt.isSetIfNotExists()) { + LOG.info("create table[{}] which already exists", tableName); + return true; + } else { + ErrorReport.reportDdlException(ErrorCode.ERR_TABLE_EXISTS_ERROR, tableName); + } + } + // 2. second, check fi table exist in local. + // This is because case sensibility issue, eg: + // 1. lower_case_table_name = 1 + // 2. create table tbl1; + // 3. create table TBL1; TBL1 does not exist in remote because the remote system is case-sensitive. + // but because lower_case_table_name = 1, the table can not be created in Doris because it is conflict with + // tbl1 + ExternalTable dorisTable = db.getTableNullable(tableName); + if (dorisTable != null) { if (stmt.isSetIfNotExists()) { LOG.info("create table[{}] which already exists", tableName); return true; @@ -323,82 +309,59 @@ public boolean performCreateTable(CreateTableStmt stmt) throws UserException { @Override public void afterCreateTable(String dbName, String tblName) { - ExternalDatabase db = dorisCatalog.getDbNullable(dbName); - if (db != null) { - db.setUnInitialized(true); + Optional> db = dorisCatalog.getDbForReplay(dbName); + if (db.isPresent()) { + db.get().setUnInitialized(true); } } @Override - public void dropTableImpl(DropTableStmt stmt) throws DdlException { - if (stmt == null) { - throw new DdlException("DropTableStmt is null"); - } - dropTableImpl(stmt.getDbName(), stmt.getTableName(), stmt.isSetIfExists()); - } - - public void dropTableImpl(String dbName, String tableName, boolean ifExists) throws DdlException { + public void dropTableImpl(ExternalTable dorisTable, boolean ifExists) throws DdlException { try { preExecutionAuthenticator.execute(() -> { - if (getExternalCatalog().getMetadataOps().viewExists(dbName, tableName)) { - performDropView(dbName, tableName, ifExists); + if (getExternalCatalog().getMetadataOps() + .viewExists(dorisTable.getRemoteDbName(), dorisTable.getRemoteName())) { + performDropView(dorisTable.getRemoteDbName(), dorisTable.getRemoteName()); } else { - performDropTable(dbName, tableName, ifExists); + performDropTable(dorisTable.getRemoteDbName(), dorisTable.getRemoteName(), ifExists); } return null; }); } catch (Exception e) { throw new DdlException( - "Failed to drop table: " + tableName + ", error message is:" + e.getMessage(), e); + "Failed to drop table: " + dorisTable.getName() + ", error message is:" + e.getMessage(), e); } } @Override public void afterDropTable(String dbName, String tblName) { - ExternalDatabase db = dorisCatalog.getDbNullable(dbName); - if (db != null) { - db.setUnInitialized(true); - } - } - - private void performDropTable(DropTableStmt stmt) throws DdlException { - if (stmt == null) { - throw new DdlException("DropTableStmt is null"); + Optional> db = dorisCatalog.getDbForReplay(dbName); + if (db.isPresent()) { + db.get().setUnInitialized(true); } - performDropTable(stmt.getDbName(), stmt.getTableName(), stmt.isSetIfExists()); } - private void performDropTable(String dbName, String tableName, boolean ifExists) throws DdlException { - ExternalDatabase db = dorisCatalog.getDbNullable(dbName); - if (db == null) { + private void performDropTable(String remoteDbName, String remoteTblName, boolean ifExists) throws DdlException { + if (!tableExist(remoteDbName, remoteTblName)) { if (ifExists) { - LOG.info("database [{}] does not exist when drop table[{}]", dbName, tableName); + LOG.info("drop table[{}] which does not exist", remoteTblName); return; } else { - ErrorReport.reportDdlException(ErrorCode.ERR_BAD_DB_ERROR, dbName); + ErrorReport.reportDdlException(ErrorCode.ERR_UNKNOWN_TABLE, remoteTblName, remoteDbName); } } - - if (!tableExist(dbName, tableName)) { - if (ifExists) { - LOG.info("drop table[{}] which does not exist", tableName); - return; - } else { - ErrorReport.reportDdlException(ErrorCode.ERR_UNKNOWN_TABLE, tableName, dbName); - } - } - catalog.dropTable(getTableIdentifier(dbName, tableName), true); + catalog.dropTable(getTableIdentifier(remoteDbName, remoteTblName), true); } @Override - public void truncateTableImpl(String dbName, String tblName, List partitions) { + public void truncateTableImpl(ExternalTable dorisTable, List partitions) { throw new UnsupportedOperationException("Truncate Iceberg table is not supported."); } @Override - public void createOrReplaceBranchImpl(String dbName, String tblName, CreateOrReplaceBranchInfo branchInfo) + public void createOrReplaceBranchImpl(ExternalTable dorisTable, CreateOrReplaceBranchInfo branchInfo) throws UserException { - Table icebergTable = IcebergUtils.getIcebergTable(dorisCatalog, dbName, tblName); + Table icebergTable = IcebergUtils.getIcebergTable(dorisTable); BranchOptions branchOptions = branchInfo.getBranchOptions(); Long snapshotId = branchOptions.getSnapshotId() @@ -453,19 +416,21 @@ public void createOrReplaceBranchImpl(String dbName, String tblName, CreateOrRep @Override public void afterOperateOnBranchOrTag(String dbName, String tblName) { - ExternalDatabase db = dorisCatalog.getDbNullable(dbName); - if (db != null) { - ExternalTable tbl = db.getTableNullable(tblName); - if (tbl != null) { - tbl.unsetObjectCreated(); + Optional> db = dorisCatalog.getDbForReplay(dbName); + if (db.isPresent()) { + Optional tbl = db.get().getTableForReplay(tblName); + if (tbl.isPresent()) { + Env.getCurrentEnv().getRefreshManager() + .refreshTableInternal(dorisCatalog, db.get(), (TableIf) tbl.get(), + System.currentTimeMillis()); } } } @Override - public void createOrReplaceTagImpl(String dbName, String tblName, CreateOrReplaceTagInfo tagInfo) + public void createOrReplaceTagImpl(ExternalTable dorisTable, CreateOrReplaceTagInfo tagInfo) throws UserException { - Table icebergTable = IcebergUtils.getIcebergTable(dorisCatalog, dbName, tblName); + Table icebergTable = IcebergUtils.getIcebergTable(dorisTable); TagOptions tagOptions = tagInfo.getTagOptions(); Long snapshotId = tagOptions.getSnapshotId() .orElse( @@ -507,10 +472,10 @@ public void createOrReplaceTagImpl(String dbName, String tblName, CreateOrReplac } @Override - public void dropTagImpl(String dbName, String tblName, DropTagInfo tagInfo) throws UserException { + public void dropTagImpl(ExternalTable dorisTable, DropTagInfo tagInfo) throws UserException { String tagName = tagInfo.getTagName(); boolean ifExists = tagInfo.getIfExists(); - Table icebergTable = IcebergUtils.getIcebergTable(dorisCatalog, dbName, tblName); + Table icebergTable = IcebergUtils.getIcebergTable(dorisTable); SnapshotRef snapshotRef = icebergTable.refs().get(tagName); if (snapshotRef != null || !ifExists) { @@ -526,10 +491,10 @@ public void dropTagImpl(String dbName, String tblName, DropTagInfo tagInfo) thro } @Override - public void dropBranchImpl(String dbName, String tblName, DropBranchInfo branchInfo) throws UserException { + public void dropBranchImpl(ExternalTable dorisTable, DropBranchInfo branchInfo) throws UserException { String branchName = branchInfo.getBranchName(); boolean ifExists = branchInfo.getIfExists(); - Table icebergTable = IcebergUtils.getIcebergTable(dorisCatalog, dbName, tblName); + Table icebergTable = IcebergUtils.getIcebergTable(dorisTable); SnapshotRef snapshotRef = icebergTable.refs().get(branchName); if (snapshotRef != null || !ifExists) { @@ -558,13 +523,13 @@ public Table loadTable(String dbName, String tblName) { } @Override - public boolean viewExists(String dbName, String viewName) { + public boolean viewExists(String remoteDbName, String remoteViewName) { if (!(catalog instanceof ViewCatalog)) { return false; } try { return preExecutionAuthenticator.execute(() -> - ((ViewCatalog) catalog).viewExists(getTableIdentifier(dbName, viewName))); + ((ViewCatalog) catalog).viewExists(getTableIdentifier(remoteDbName, remoteViewName))); } catch (Exception e) { throw new RuntimeException("Failed to check view exist, error message is:" + e.getMessage(), e); @@ -618,30 +583,12 @@ public ThreadPoolExecutor getThreadPoolWithPreAuth() { return dorisCatalog.getThreadPoolExecutor(); } - private void performDropView(String dbName, String viewName, boolean ifExists) throws DdlException { + private void performDropView(String remoteDbName, String remoteViewName) throws DdlException { if (!(catalog instanceof ViewCatalog)) { throw new DdlException("Drop Iceberg view is not supported with not view catalog."); } - ExternalDatabase db = dorisCatalog.getDbNullable(dbName); - if (db == null) { - if (ifExists) { - LOG.info("database [{}] does not exist when drop view[{}]", dbName, viewName); - return; - } else { - ErrorReport.reportDdlException(ErrorCode.ERR_BAD_DB_ERROR, dbName); - } - } ViewCatalog viewCatalog = (ViewCatalog) catalog; - if (!viewExists(dbName, viewName)) { - if (ifExists) { - LOG.info("drop view[{}] which does not exist", viewName); - return; - } else { - ErrorReport.reportDdlException(ErrorCode.ERR_UNKNOWN_TABLE, viewName, dbName); - } - } - viewCatalog.dropView(getTableIdentifier(dbName, viewName)); - db.setUnInitialized(true); + viewCatalog.dropView(getTableIdentifier(remoteDbName, remoteViewName)); } - } + diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergSchemaCacheKey.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergSchemaCacheKey.java index 7931d91831fcec..0f5d15e4f6fb48 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergSchemaCacheKey.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergSchemaCacheKey.java @@ -18,14 +18,15 @@ package org.apache.doris.datasource.iceberg; import org.apache.doris.datasource.ExternalSchemaCache.SchemaCacheKey; +import org.apache.doris.datasource.NameMapping; import com.google.common.base.Objects; public class IcebergSchemaCacheKey extends SchemaCacheKey { private final long schemaId; - public IcebergSchemaCacheKey(String dbName, String tableName, long schemaId) { - super(dbName, tableName); + public IcebergSchemaCacheKey(NameMapping nameMapping, long schemaId) { + super(nameMapping); this.schemaId = schemaId; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergTransaction.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergTransaction.java index 797caea0deaf1d..95cf6c36a3b37c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergTransaction.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergTransaction.java @@ -21,8 +21,8 @@ package org.apache.doris.datasource.iceberg; import org.apache.doris.common.UserException; -import org.apache.doris.common.info.SimpleTableInfo; -import org.apache.doris.datasource.ExternalCatalog; +import org.apache.doris.datasource.ExternalTable; +import org.apache.doris.datasource.NameMapping; import org.apache.doris.datasource.iceberg.helper.IcebergWriterHelper; import org.apache.doris.nereids.trees.plans.commands.insert.BaseExternalTableInsertCommandContext; import org.apache.doris.nereids.trees.plans.commands.insert.InsertCommandContext; @@ -48,7 +48,6 @@ import java.util.Arrays; import java.util.Collections; import java.util.List; -import java.util.Objects; import java.util.Optional; public class IcebergTransaction implements Transaction { @@ -56,7 +55,6 @@ public class IcebergTransaction implements Transaction { private static final Logger LOG = LogManager.getLogger(IcebergTransaction.class); private final IcebergMetadataOps ops; - private SimpleTableInfo tableInfo; private Table table; @@ -73,23 +71,22 @@ public void updateIcebergCommitData(List commitDataList) { } } - public void beginInsert(SimpleTableInfo tableInfo) throws UserException { + public void beginInsert(ExternalTable dorisTable) throws UserException { try { ops.getPreExecutionAuthenticator().execute(() -> { // create and start the iceberg transaction - this.tableInfo = tableInfo; - this.table = getNativeTable(tableInfo); + this.table = IcebergUtils.getIcebergTable(dorisTable); this.transaction = table.newTransaction(); }); } catch (Exception e) { - throw new UserException("Failed to begin insert for iceberg table " + tableInfo, e); + throw new UserException("Failed to begin insert for iceberg table " + dorisTable.getName(), e); } } - public void finishInsert(SimpleTableInfo tableInfo, Optional insertCtx) { + public void finishInsert(NameMapping nameMapping, Optional insertCtx) { if (LOG.isDebugEnabled()) { - LOG.info("iceberg table {} insert table finished!", tableInfo); + LOG.info("iceberg table {} insert table finished!", nameMapping.getFullLocalName()); } try { ops.getPreExecutionAuthenticator().execute(() -> { @@ -104,7 +101,7 @@ public void finishInsert(SimpleTableInfo tableInfo, Optional pendingResults) { // commit append files. AppendFiles appendFiles = table.newAppend().scanManifestsWith(ops.getThreadPoolWithPreAuth()); diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergUtils.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergUtils.java index 73d4a2fa76a0f4..27fc50ba11865f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergUtils.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergUtils.java @@ -51,11 +51,11 @@ import org.apache.doris.catalog.Type; import org.apache.doris.common.AnalysisException; import org.apache.doris.common.UserException; -import org.apache.doris.common.info.SimpleTableInfo; import org.apache.doris.common.util.TimeUtils; import org.apache.doris.datasource.CacheException; import org.apache.doris.datasource.ExternalCatalog; import org.apache.doris.datasource.ExternalSchemaCache; +import org.apache.doris.datasource.ExternalTable; import org.apache.doris.datasource.SchemaCacheValue; import org.apache.doris.datasource.iceberg.source.IcebergTableQueryInfo; import org.apache.doris.datasource.mvcc.MvccSnapshot; @@ -601,49 +601,28 @@ public static Type icebergTypeToDorisType(org.apache.iceberg.types.Type type) { } } - - public static org.apache.iceberg.Table getIcebergTable(ExternalCatalog catalog, String dbName, String tblName) { - return getIcebergTableInternal(catalog, dbName, tblName, false); - } - - public static org.apache.iceberg.Table getAndCloneTable(ExternalCatalog catalog, SimpleTableInfo tableInfo) { - return getIcebergTableInternal(catalog, tableInfo.getDbName(), tableInfo.getTbName(), true); - } - - public static org.apache.iceberg.Table getRemoteTable(ExternalCatalog catalog, SimpleTableInfo tableInfo) { + public static org.apache.iceberg.Table getIcebergTable(ExternalTable dorisTable) { return Env.getCurrentEnv() .getExtMetaCacheMgr() - .getIcebergMetadataCache() - .getIcebergTable(catalog, tableInfo.getDbName(), tableInfo.getTbName()); - } - - private static org.apache.iceberg.Table getIcebergTableInternal(ExternalCatalog catalog, String dbName, - String tblName, - boolean isClone) { - IcebergMetadataCache metadataCache = Env.getCurrentEnv() - .getExtMetaCacheMgr() - .getIcebergMetadataCache(); - return isClone ? metadataCache.getAndCloneTable(catalog, dbName, tblName) - : metadataCache.getIcebergTable(catalog, dbName, tblName); + .getIcebergMetadataCache().getIcebergTable(dorisTable); } /** * Get iceberg schema from catalog and convert them to doris schema */ - public static List getSchema( - ExternalCatalog catalog, String dbName, String name, long schemaId, boolean isView) { + private static List getSchema(ExternalTable dorisTable, long schemaId, boolean isView) { try { - return catalog.getPreExecutionAuthenticator().execute(() -> { + return dorisTable.getCatalog().getPreExecutionAuthenticator().execute(() -> { Schema schema; if (isView) { - View icebergView = getIcebergView(catalog, dbName, name); + View icebergView = getIcebergView(dorisTable); if (schemaId == NEWEST_SCHEMA_ID) { schema = icebergView.schema(); } else { schema = icebergView.schemas().get((int) schemaId); } } else { - Table icebergTable = getIcebergTable(catalog, dbName, name); + Table icebergTable = getIcebergTable(dorisTable); if (schemaId == NEWEST_SCHEMA_ID || icebergTable.currentSnapshot() == null) { schema = icebergTable.schema(); } else { @@ -652,7 +631,8 @@ public static List getSchema( } String type = isView ? "view" : "table"; Preconditions.checkNotNull(schema, - "Schema for " + type + " " + catalog.getName() + "." + dbName + "." + name + " is null"); + "Schema for " + type + " " + dorisTable.getCatalog().getName() + + "." + dorisTable.getDbName() + "." + dorisTable.getName() + " is null"); return parseSchema(schema); }); } catch (Exception e) { @@ -681,23 +661,25 @@ public static List parseSchema(Schema schema) { * * @return estimated row count */ - public static long getIcebergRowCount(ExternalCatalog catalog, String dbName, String tbName) { + public static long getIcebergRowCount(ExternalTable tbl) { // the table may be null when the iceberg metadata cache is not loaded.But I don't think it's a problem, // because the NPE would be caught in the caller and return the default value -1. // Meanwhile, it will trigger iceberg metadata cache to load the table, so we can get it next time. Table icebergTable = Env.getCurrentEnv() .getExtMetaCacheMgr() .getIcebergMetadataCache() - .getIcebergTable(catalog, dbName, tbName); + .getIcebergTable(tbl); Snapshot snapshot = icebergTable.currentSnapshot(); if (snapshot == null) { - LOG.info("Iceberg table {}.{}.{} is empty, return -1.", catalog.getName(), dbName, tbName); + LOG.info("Iceberg table {}.{}.{} is empty, return -1.", + tbl.getCatalog().getName(), tbl.getDbName(), tbl.getName()); // empty table return TableIf.UNKNOWN_ROW_COUNT; } Map summary = snapshot.summary(); long rows = Long.parseLong(summary.get(TOTAL_RECORDS)) - Long.parseLong(summary.get(TOTAL_POSITION_DELETES)); - LOG.info("Iceberg table {}.{}.{} row count in summary is {}", catalog.getName(), dbName, tbName, rows); + LOG.info("Iceberg table {}.{}.{} row count in summary is {}", + tbl.getCatalog().getName(), tbl.getDbName(), tbl.getName(), rows); return rows; } @@ -909,38 +891,37 @@ public static boolean isIcebergBranchOrTag(Optional scanParams) } // read schema from external schema cache - public static IcebergSchemaCacheValue getSchemaCacheValue( - ExternalCatalog catalog, String dbName, String name, long schemaId) { - ExternalSchemaCache cache = Env.getCurrentEnv().getExtMetaCacheMgr().getSchemaCache(catalog); + public static IcebergSchemaCacheValue getSchemaCacheValue(ExternalTable dorisTable, long schemaId) { + ExternalSchemaCache cache = Env.getCurrentEnv().getExtMetaCacheMgr().getSchemaCache(dorisTable.getCatalog()); Optional schemaCacheValue = cache.getSchemaValue( - new IcebergSchemaCacheKey(dbName, name, schemaId)); + new IcebergSchemaCacheKey(dorisTable.getOrBuildNameMapping(), schemaId)); if (!schemaCacheValue.isPresent()) { throw new CacheException("failed to getSchema for: %s.%s.%s.%s", - null, catalog.getName(), dbName, name, schemaId); + null, dorisTable.getCatalog().getName(), dorisTable.getDbName(), dorisTable.getName(), schemaId); } return (IcebergSchemaCacheValue) schemaCacheValue.get(); } - public static IcebergSnapshot getLastedIcebergSnapshot(ExternalCatalog catalog, String dbName, String tbName) { - Table table = IcebergUtils.getIcebergTable(catalog, dbName, tbName); + public static IcebergSnapshot getLastedIcebergSnapshot(ExternalTable dorisTable) { + Table table = IcebergUtils.getIcebergTable(dorisTable); Snapshot snapshot = table.currentSnapshot(); long snapshotId = snapshot == null ? IcebergUtils.UNKNOWN_SNAPSHOT_ID : snapshot.snapshotId(); return new IcebergSnapshot(snapshotId, table.schema().schemaId()); } - public static IcebergPartitionInfo loadPartitionInfo( - ExternalCatalog catalog, String dbName, String tbName, long snapshotId) throws AnalysisException { + public static IcebergPartitionInfo loadPartitionInfo(ExternalTable dorisTable, long snapshotId) + throws AnalysisException { // snapshotId == UNKNOWN_SNAPSHOT_ID means this is an empty table, haven't contained any snapshot yet. if (snapshotId == IcebergUtils.UNKNOWN_SNAPSHOT_ID) { return IcebergPartitionInfo.empty(); } - Table table = getIcebergTable(catalog, dbName, tbName); + Table table = getIcebergTable(dorisTable); List icebergPartitions = loadIcebergPartition(table, snapshotId); Map nameToPartition = Maps.newHashMap(); Map nameToPartitionItem = Maps.newHashMap(); List partitionColumns = IcebergUtils.getSchemaCacheValue( - catalog, dbName, tbName, table.snapshot(snapshotId).schemaId()).getPartitionColumns(); + dorisTable, table.snapshot(snapshotId).schemaId()).getPartitionColumns(); for (IcebergPartition partition : icebergPartitions) { nameToPartition.put(partition.getPartitionName(), partition); String transform = table.specs().get(partition.getSpecId()).fields().get(0).transform().toString(); @@ -1156,16 +1137,12 @@ public int compare(Map.Entry p1, Map.Entry tableSnapshot, - ExternalCatalog catalog, - String dbName, - String tbName, + ExternalTable dorisTable, Optional scanParams) { - IcebergSnapshotCacheValue snapshotCache = Env.getCurrentEnv().getExtMetaCacheMgr().getIcebergMetadataCache() - .getSnapshotCache(catalog, dbName, tbName); if (tableSnapshot.isPresent() || IcebergUtils.isIcebergBranchOrTag(scanParams)) { // If a snapshot is specified, // use the specified snapshot and the corresponding schema(not the latest schema). - Table icebergTable = getIcebergTable(catalog, dbName, tbName); + Table icebergTable = getIcebergTable(dorisTable); IcebergTableQueryInfo info; try { info = getQuerySpecSnapshot(icebergTable, tableSnapshot, scanParams); @@ -1177,53 +1154,41 @@ public static IcebergSnapshotCacheValue getIcebergSnapshotCacheValue( new IcebergSnapshot(info.getSnapshotId(), info.getSchemaId())); } else { // Otherwise, use the latest snapshot and the latest schema. - return snapshotCache; + return Env.getCurrentEnv().getExtMetaCacheMgr().getIcebergMetadataCache() + .getSnapshotCache(dorisTable); } } - public static List getIcebergSchema( - TableIf tableIf, - ExternalCatalog catalog, - String dbName, - String tbName) { - Optional snapshotFromContext = MvccUtil.getSnapshotFromContext(tableIf); + public static List getIcebergSchema(ExternalTable dorisTable) { + Optional snapshotFromContext = MvccUtil.getSnapshotFromContext(dorisTable); IcebergSnapshotCacheValue cacheValue = - IcebergUtils.getOrFetchSnapshotCacheValue(snapshotFromContext, catalog, dbName, tbName); - return IcebergUtils.getSchemaCacheValue( - catalog, dbName, tbName, cacheValue.getSnapshot().getSchemaId()) + IcebergUtils.getOrFetchSnapshotCacheValue(snapshotFromContext, dorisTable); + return IcebergUtils.getSchemaCacheValue(dorisTable, cacheValue.getSnapshot().getSchemaId()) .getSchema(); } public static IcebergSnapshotCacheValue getOrFetchSnapshotCacheValue( Optional snapshot, - ExternalCatalog catalog, - String dbName, - String tbName) { + ExternalTable dorisTable) { if (snapshot.isPresent()) { return ((IcebergMvccSnapshot) snapshot.get()).getSnapshotCacheValue(); } else { - return IcebergUtils.getIcebergSnapshotCacheValue( - Optional.empty(), catalog, dbName, tbName, Optional.empty()); + return IcebergUtils.getIcebergSnapshotCacheValue(Optional.empty(), dorisTable, Optional.empty()); } } - public static org.apache.iceberg.view.View getIcebergView(ExternalCatalog catalog, String dbName, String tblName) { - return getIcebergViewInternal(catalog, dbName, tblName); - } - - private static org.apache.iceberg.view.View getIcebergViewInternal(ExternalCatalog catalog, String dbName, - String tblName) { + public static org.apache.iceberg.view.View getIcebergView(ExternalTable dorisTable) { IcebergMetadataCache metadataCache = Env.getCurrentEnv().getExtMetaCacheMgr().getIcebergMetadataCache(); - return metadataCache.getIcebergView(catalog, dbName, tblName); + return metadataCache.getIcebergView(dorisTable); } public static Optional loadSchemaCacheValue( - ExternalCatalog catalog, String dbName, String tbName, long schemaId, boolean isView) { - List schema = IcebergUtils.getSchema(catalog, dbName, tbName, schemaId, isView); + ExternalTable dorisTable, long schemaId, boolean isView) { + List schema = IcebergUtils.getSchema(dorisTable, schemaId, isView); List tmpColumns = Lists.newArrayList(); if (!isView) { // get table partition column info - Table table = IcebergUtils.getIcebergTable(catalog, dbName, tbName); + Table table = IcebergUtils.getIcebergTable(dorisTable); PartitionSpec spec = table.spec(); for (PartitionField field : spec.fields()) { Types.NestedField col = table.schema().findField(field.sourceId()); diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/source/IcebergApiSource.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/source/IcebergApiSource.java index 7915fec174382f..78f7ece1a7a33e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/source/IcebergApiSource.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/source/IcebergApiSource.java @@ -56,9 +56,7 @@ public IcebergApiSource(IcebergExternalTable table, TupleDescriptor desc, this.icebergExtTable = table; this.originTable = Env.getCurrentEnv().getExtMetaCacheMgr().getIcebergMetadataCache().getIcebergTable( - icebergExtTable.getCatalog(), - icebergExtTable.getRemoteDbName(), - icebergExtTable.getRemoteName()); + icebergExtTable); this.desc = desc; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/source/IcebergHMSSource.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/source/IcebergHMSSource.java index 5fc6b400d1c027..2e68c775a3eabd 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/source/IcebergHMSSource.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/source/IcebergHMSSource.java @@ -25,26 +25,19 @@ import org.apache.doris.datasource.ExternalCatalog; import org.apache.doris.datasource.hive.HMSExternalTable; import org.apache.doris.datasource.iceberg.IcebergUtils; -import org.apache.doris.planner.ColumnRange; - -import java.util.Map; public class IcebergHMSSource implements IcebergSource { private final HMSExternalTable hmsTable; private final TupleDescriptor desc; - private final Map columnNameToRange; private final org.apache.iceberg.Table icebergTable; - public IcebergHMSSource(HMSExternalTable hmsTable, TupleDescriptor desc, - Map columnNameToRange) { + public IcebergHMSSource(HMSExternalTable hmsTable, TupleDescriptor desc) { this.hmsTable = hmsTable; this.desc = desc; - this.columnNameToRange = columnNameToRange; this.icebergTable = Env.getCurrentEnv().getExtMetaCacheMgr().getIcebergMetadataCache() - .getIcebergTable(hmsTable.getCatalog(), - hmsTable.getRemoteDbName(), hmsTable.getRemoteName()); + .getIcebergTable(hmsTable); } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/source/IcebergScanNode.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/source/IcebergScanNode.java index ccd2a06fb5d2bd..ee27143d550eda 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/source/IcebergScanNode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/source/IcebergScanNode.java @@ -121,7 +121,7 @@ public IcebergScanNode(PlanNodeId id, TupleDescriptor desc, boolean needCheckCol ExternalTable table = (ExternalTable) desc.getTable(); if (table instanceof HMSExternalTable) { - source = new IcebergHMSSource((HMSExternalTable) table, desc, columnNameToRange); + source = new IcebergHMSSource((HMSExternalTable) table, desc); } else if (table instanceof IcebergExternalTable) { String catalogType = ((IcebergExternalTable) table).getIcebergCatalogType(); switch (catalogType) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/maxcompute/MaxComputeMetadataCacheMgr.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/maxcompute/MaxComputeMetadataCacheMgr.java index 270c47f0df6f6c..c8b87519a78a05 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/maxcompute/MaxComputeMetadataCacheMgr.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/maxcompute/MaxComputeMetadataCacheMgr.java @@ -17,6 +17,8 @@ package org.apache.doris.datasource.maxcompute; +import org.apache.doris.datasource.ExternalTable; + import com.google.common.collect.Maps; import java.util.Map; @@ -58,10 +60,10 @@ public void invalidateDbCache(long catalogId, String dbName) { } } - public void invalidateTableCache(long catalogId, String dbName, String tblName) { - MaxComputeMetadataCache cache = maxComputeMetadataCaches.get(catalogId); + public void invalidateTableCache(ExternalTable dorisTable) { + MaxComputeMetadataCache cache = maxComputeMetadataCaches.get(dorisTable.getCatalog().getId()); if (cache != null) { - cache.cleanTableCache(dbName, tblName); + cache.cleanTableCache(dorisTable.getDbName(), dorisTable.getName()); } } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/metacache/MetaCache.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/metacache/MetaCache.java index dc2670a83c6ba9..3cc47d26d3de4b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/metacache/MetaCache.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/metacache/MetaCache.java @@ -103,6 +103,14 @@ public Optional getMetaObj(String name, long id) { return val; } + public Optional tryGetMetaObj(String name) { + Optional val = metaObjCache.getIfPresent(name); + if (val == null || !val.isPresent()) { + return Optional.empty(); + } + return val; + } + public Optional getMetaObjById(long id) { String name = idToName.get(id); return name == null ? Optional.empty() : getMetaObj(name, id); @@ -144,4 +152,10 @@ public void invalidateAll() { public LoadingCache> getMetaObjCache() { return metaObjCache; } + + @VisibleForTesting + public void addObjForTest(long id, String name, T db) { + idToName.put(id, name); + metaObjCache.put(name, Optional.of(db)); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/operations/ExternalMetadataOps.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/operations/ExternalMetadataOps.java index 30dbab3e07056b..5a31011f142a93 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/operations/ExternalMetadataOps.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/operations/ExternalMetadataOps.java @@ -17,13 +17,10 @@ package org.apache.doris.datasource.operations; -import org.apache.doris.analysis.CreateDbStmt; import org.apache.doris.analysis.CreateTableStmt; -import org.apache.doris.analysis.DropDbStmt; -import org.apache.doris.analysis.DropTableStmt; import org.apache.doris.common.DdlException; import org.apache.doris.common.UserException; -import org.apache.doris.nereids.trees.plans.commands.CreateDatabaseCommand; +import org.apache.doris.datasource.ExternalTable; 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.DropBranchInfo; @@ -33,6 +30,7 @@ import java.util.Collections; import java.util.List; +import java.util.Map; /** * all external metadata operations use this interface @@ -41,53 +39,47 @@ public interface ExternalMetadataOps { /** * create db in external metastore - * @param stmt + * @param dbName + * @param properties + * @return false means db does not exist and is created this time * @throws DdlException */ - default void createDb(CreateDbStmt stmt) throws DdlException { - createDbImpl(stmt); - afterCreateDb(stmt.getFullDbName()); + default boolean createDb(String dbName, boolean ifNotExists, Map properties) throws DdlException { + boolean res = createDbImpl(dbName, ifNotExists, properties); + if (!res) { + afterCreateDb(); + } + return res; } /** - * create db in external metastore - * @param command + * create db in external metastore for nereids + * + * @param dbName the remote name that will be created in remote metastore + * @param ifNotExists + * @param properties + * @return false means db does not exist and is created this time * @throws DdlException */ - default void createDb(CreateDatabaseCommand command) throws DdlException { - createDbImpl(command); - afterCreateDb(command.getDbName()); - } - - void createDbImpl(CreateDbStmt stmt) throws DdlException; + boolean createDbImpl(String dbName, boolean ifNotExists, Map properties) throws DdlException; - void createDbImpl(CreateDatabaseCommand command) throws DdlException; - - default void afterCreateDb(String dbName) { + default void afterCreateDb() { } + /** * drop db in external metastore - * @param stmt + * + * @param dbName the local db name in Doris + * @param ifExists + * @param force * @throws DdlException */ - default void dropDb(DropDbStmt stmt) throws DdlException { - dropDbImpl(stmt.getDbName(), stmt.isSetIfExists(), stmt.isForceDrop()); - afterDropDb(stmt.getCtlName()); - } - - default void dropDb(String ctlName, String dbName, boolean ifExists, boolean force) throws DdlException { + default void dropDb(String dbName, boolean ifExists, boolean force) throws DdlException { dropDbImpl(dbName, ifExists, force); - afterDropDb(ctlName); + afterDropDb(dbName); } - /** - * drop db in external metastore for nereids - * @param dbName - * @param ifExists - * @param force - * @throws DdlException - */ void dropDbImpl(String dbName, boolean ifExists, boolean force) throws DdlException; void afterDropDb(String dbName); @@ -95,7 +87,7 @@ default void dropDb(String ctlName, String dbName, boolean ifExists, boolean for /** * * @param stmt - * @return if set isExists is true, return true if table exists, otherwise return false + * @return return false means table does not exist and is created this time * @throws UserException */ default boolean createTable(CreateTableStmt stmt) throws UserException { @@ -111,24 +103,12 @@ default boolean createTable(CreateTableStmt stmt) throws UserException { default void afterCreateTable(String dbName, String tblName) { } - /** - * - * @param stmt - * @throws DdlException - */ - default void dropTable(DropTableStmt stmt) throws DdlException { - dropTableImpl(stmt); - afterDropTable(stmt.getDbName(), stmt.getTableName()); - } - - default void dropTable(String dbName, String tableName, boolean ifExists) throws DdlException { - dropTableImpl(dbName, tableName, ifExists); - afterDropTable(dbName, tableName); + default void dropTable(ExternalTable dorisTable, boolean ifExists) throws DdlException { + dropTableImpl(dorisTable, ifExists); + afterDropTable(dorisTable.getDbName(), dorisTable.getName()); } - void dropTableImpl(DropTableStmt stmt) throws DdlException; - - void dropTableImpl(String dbName, String tableName, boolean ifExists) throws DdlException; + void dropTableImpl(ExternalTable dorisTable, boolean ifExists) throws DdlException; default void afterDropTable(String dbName, String tblName) { } @@ -136,16 +116,15 @@ default void afterDropTable(String dbName, String tblName) { /** * truncate table in external metastore * - * @param dbName - * @param tblName + * @param dorisTable * @param partitions */ - default void truncateTable(String dbName, String tblName, List partitions) throws DdlException { - truncateTableImpl(dbName, tblName, partitions); - afterTruncateTable(dbName, tblName); + default void truncateTable(ExternalTable dorisTable, List partitions) throws DdlException { + truncateTableImpl(dorisTable, partitions); + afterTruncateTable(dorisTable.getDbName(), dorisTable.getName()); } - void truncateTableImpl(String dbName, String tblName, List partitions) throws DdlException; + void truncateTableImpl(ExternalTable dorisTable, List partitions) throws DdlException; default void afterTruncateTable(String dbName, String tblName) { } @@ -153,18 +132,17 @@ default void afterTruncateTable(String dbName, String tblName) { /** * create or replace branch in external metastore * - * @param dbName - * @param tblName + * @param dorisTable * @param branchInfo * @throws UserException */ - default void createOrReplaceBranch(String dbName, String tblName, CreateOrReplaceBranchInfo branchInfo) + default void createOrReplaceBranch(ExternalTable dorisTable, CreateOrReplaceBranchInfo branchInfo) throws UserException { - createOrReplaceBranchImpl(dbName, tblName, branchInfo); - afterOperateOnBranchOrTag(dbName, tblName); + createOrReplaceBranchImpl(dorisTable, branchInfo); + afterOperateOnBranchOrTag(dorisTable.getDbName(), dorisTable.getName()); } - void createOrReplaceBranchImpl(String dbName, String tblName, CreateOrReplaceBranchInfo branchInfo) + void createOrReplaceBranchImpl(ExternalTable dorisTable, CreateOrReplaceBranchInfo branchInfo) throws UserException; default void afterOperateOnBranchOrTag(String dbName, String tblName) { @@ -173,51 +151,48 @@ default void afterOperateOnBranchOrTag(String dbName, String tblName) { /** * create or replace tag in external metastore * - * @param dbName - * @param tblName + * @param dorisTable * @param tagInfo * @throws UserException */ - default void createOrReplaceTag(String dbName, String tblName, CreateOrReplaceTagInfo tagInfo) + default void createOrReplaceTag(ExternalTable dorisTable, CreateOrReplaceTagInfo tagInfo) throws UserException { - createOrReplaceTagImpl(dbName, tblName, tagInfo); - afterOperateOnBranchOrTag(dbName, tblName); + createOrReplaceTagImpl(dorisTable, tagInfo); + afterOperateOnBranchOrTag(dorisTable.getDbName(), dorisTable.getName()); } - void createOrReplaceTagImpl(String dbName, String tblName, CreateOrReplaceTagInfo tagInfo) + void createOrReplaceTagImpl(ExternalTable dorisTable, CreateOrReplaceTagInfo tagInfo) throws UserException; /** * drop tag in external metastore * - * @param dbName - * @param tblName + * @param dorisTable * @param tagInfo * @throws UserException */ - default void dropTag(String dbName, String tblName, DropTagInfo tagInfo) + default void dropTag(ExternalTable dorisTable, DropTagInfo tagInfo) throws UserException { - dropTagImpl(dbName, tblName, tagInfo); - afterOperateOnBranchOrTag(dbName, tblName); + dropTagImpl(dorisTable, tagInfo); + afterOperateOnBranchOrTag(dorisTable.getDbName(), dorisTable.getName()); } - void dropTagImpl(String dbName, String tblName, DropTagInfo tagInfo) throws UserException; + void dropTagImpl(ExternalTable dorisTable, DropTagInfo tagInfo) throws UserException; /** * drop branch in external metastore * - * @param dbName - * @param tblName + * @param dorisTable * @param branchInfo * @throws UserException */ - default void dropBranch(String dbName, String tblName, DropBranchInfo branchInfo) + default void dropBranch(ExternalTable dorisTable, DropBranchInfo branchInfo) throws UserException { - dropBranchImpl(dbName, tblName, branchInfo); - afterOperateOnBranchOrTag(dbName, tblName); + dropBranchImpl(dorisTable, branchInfo); + afterOperateOnBranchOrTag(dorisTable.getDbName(), dorisTable.getName()); } - void dropBranchImpl(String dbName, String tblName, DropBranchInfo branchInfo) throws UserException; + void dropBranchImpl(ExternalTable dorisTable, DropBranchInfo branchInfo) throws UserException; /** * diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonExternalCatalog.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonExternalCatalog.java index fd332f8216f32e..7483ef2e477cf5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonExternalCatalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonExternalCatalog.java @@ -23,6 +23,7 @@ import org.apache.doris.datasource.CatalogProperty; import org.apache.doris.datasource.ExternalCatalog; import org.apache.doris.datasource.InitCatalogLog; +import org.apache.doris.datasource.NameMapping; import org.apache.doris.datasource.SessionContext; import org.apache.doris.datasource.property.PropertyConverter; import org.apache.doris.datasource.property.constants.HMSProperties; @@ -128,23 +129,26 @@ public List listTableNames(SessionContext ctx, String dbName) { } } - public org.apache.paimon.table.Table getPaimonTable(String dbName, String tblName) { + public org.apache.paimon.table.Table getPaimonTable(NameMapping nameMapping) { makeSureInitialized(); try { - return hadoopAuthenticator.doAs(() -> catalog.getTable(Identifier.create(dbName, tblName))); + return hadoopAuthenticator.doAs(() -> catalog.getTable( + Identifier.create(nameMapping.getRemoteDbName(), nameMapping.getRemoteTblName()))); } catch (Exception e) { throw new RuntimeException("Failed to get Paimon table:" + getName() + "." - + dbName + "." + tblName + ", because " + e.getMessage(), e); + + nameMapping.getLocalDbName() + "." + nameMapping.getLocalTblName() + ", because " + + e.getMessage(), e); } } - public List getPaimonPartitions(String dbName, String tblName) { + public List getPaimonPartitions(NameMapping nameMapping) { makeSureInitialized(); try { return hadoopAuthenticator.doAs(() -> { List partitions = new ArrayList<>(); try { - partitions = catalog.listPartitions(Identifier.create(dbName, tblName)); + partitions = catalog.listPartitions( + Identifier.create(nameMapping.getRemoteDbName(), nameMapping.getRemoteTblName())); } catch (Catalog.TableNotExistException e) { LOG.warn("TableNotExistException", e); } @@ -152,7 +156,8 @@ public List getPaimonPartitions(String dbName, String tblName) { }); } catch (IOException e) { throw new RuntimeException("Failed to get Paimon table partitions:" + getName() + "." - + dbName + "." + tblName + ", because " + e.getMessage(), e); + + nameMapping.getRemoteDbName() + "." + nameMapping.getRemoteTblName() + + ", because " + e.getMessage(), e); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonExternalTable.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonExternalTable.java index 635f3038bcf9a6..24698d220ca95d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonExternalTable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonExternalTable.java @@ -27,7 +27,6 @@ import org.apache.doris.common.AnalysisException; import org.apache.doris.common.DdlException; import org.apache.doris.datasource.CacheException; -import org.apache.doris.datasource.ExternalSchemaCache; import org.apache.doris.datasource.ExternalSchemaCache.SchemaCacheKey; import org.apache.doris.datasource.ExternalTable; import org.apache.doris.datasource.SchemaCacheValue; @@ -78,7 +77,7 @@ public class PaimonExternalTable extends ExternalTable implements MTMVRelatedTab public PaimonExternalTable(long id, String name, String remoteName, PaimonExternalCatalog catalog, PaimonExternalDatabase db) { super(id, name, remoteName, catalog, db, TableType.PAIMON_EXTERNAL_TABLE); - this.paimonTable = catalog.getPaimonTable(dbName, name); + this.paimonTable = catalog.getPaimonTable(getOrBuildNameMapping()); } public String getPaimonCatalogType() { @@ -96,21 +95,10 @@ public Table getPaimonTable(Optional snapshot) { return getOrFetchSnapshotCacheValue(snapshot).getSnapshot().getTable(); } - public PaimonSchemaCacheValue getPaimonSchemaCacheValue(long schemaId) { - ExternalSchemaCache cache = Env.getCurrentEnv().getExtMetaCacheMgr().getSchemaCache(catalog); - Optional schemaCacheValue = cache.getSchemaValue( - new PaimonSchemaCacheKey(dbName, name, schemaId)); - if (!schemaCacheValue.isPresent()) { - throw new CacheException("failed to getSchema for: %s.%s.%s.%s", - null, catalog.getName(), dbName, name, schemaId); - } - return (PaimonSchemaCacheValue) schemaCacheValue.get(); - } - private PaimonSnapshotCacheValue getPaimonSnapshotCacheValue() { makeSureInitialized(); return Env.getCurrentEnv().getExtMetaCacheMgr().getPaimonMetadataCache() - .getPaimonSnapshot(catalog, dbName, name); + .getPaimonSnapshot(this); } @Override @@ -251,7 +239,7 @@ public Optional initSchema(SchemaCacheKey key) { makeSureInitialized(); PaimonSchemaCacheKey paimonSchemaCacheKey = (PaimonSchemaCacheKey) key; try { - Table table = ((PaimonExternalCatalog) getCatalog()).getPaimonTable(key.getDbName(), name); + Table table = ((PaimonExternalCatalog) getCatalog()).getPaimonTable(getOrBuildNameMapping()); TableSchema tableSchema = ((DataTable) table).schemaManager().schema(paimonSchemaCacheKey.getSchemaId()); List columns = tableSchema.fields(); List dorisColumns = Lists.newArrayListWithCapacity(columns.size()); @@ -269,15 +257,16 @@ public Optional initSchema(SchemaCacheKey key) { return Optional.of(new PaimonSchemaCacheValue(dorisColumns, partitionColumns, tableSchema)); } catch (Exception e) { throw new CacheException("failed to initSchema for: %s.%s.%s.%s", - null, getCatalog().getName(), key.getDbName(), key.getTblName(), + null, getCatalog().getName(), key.getNameMapping().getLocalDbName(), + key.getNameMapping().getLocalTblName(), paimonSchemaCacheKey.getSchemaId()); } - } private PaimonSchemaCacheValue getPaimonSchemaCacheValue(Optional snapshot) { PaimonSnapshotCacheValue snapshotCacheValue = getOrFetchSnapshotCacheValue(snapshot); - return getPaimonSchemaCacheValue(snapshotCacheValue.getSnapshot().getSchemaId()); + return Env.getCurrentEnv().getExtMetaCacheMgr().getPaimonMetadataCache() + .getPaimonSchemaCacheValue(getOrBuildNameMapping(), snapshotCacheValue.getSnapshot().getSchemaId()); } private PaimonSnapshotCacheValue getOrFetchSnapshotCacheValue(Optional snapshot) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonMetadataCache.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonMetadataCache.java index b16715e823800d..b36c532d2ed150 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonMetadataCache.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonMetadataCache.java @@ -18,12 +18,17 @@ package org.apache.doris.datasource.paimon; import org.apache.doris.catalog.Column; +import org.apache.doris.catalog.Env; import org.apache.doris.common.AnalysisException; import org.apache.doris.common.CacheFactory; import org.apache.doris.common.Config; import org.apache.doris.datasource.CacheException; -import org.apache.doris.datasource.CatalogIf; +import org.apache.doris.datasource.ExternalCatalog; import org.apache.doris.datasource.ExternalMetaCacheMgr; +import org.apache.doris.datasource.ExternalSchemaCache; +import org.apache.doris.datasource.ExternalTable; +import org.apache.doris.datasource.NameMapping; +import org.apache.doris.datasource.SchemaCacheValue; import com.github.benmanes.caffeine.cache.LoadingCache; import com.google.common.collect.Maps; @@ -58,32 +63,54 @@ public PaimonMetadataCache(ExecutorService executor) { @NotNull private PaimonSnapshotCacheValue loadSnapshot(PaimonSnapshotCacheKey key) { + NameMapping nameMapping = key.getNameMapping(); try { PaimonSnapshot latestSnapshot = loadLatestSnapshot(key); - PaimonExternalTable table = (PaimonExternalTable) key.getCatalog().getDbOrAnalysisException(key.getDbName()) - .getTableOrAnalysisException(key.getTableName()); - List partitionColumns = table.getPaimonSchemaCacheValue(latestSnapshot.getSchemaId()) - .getPartitionColumns(); + List partitionColumns = getPaimonSchemaCacheValue(nameMapping, + latestSnapshot.getSchemaId()).getPartitionColumns(); PaimonPartitionInfo partitionInfo = loadPartitionInfo(key, partitionColumns); return new PaimonSnapshotCacheValue(partitionInfo, latestSnapshot); - } catch (IOException | AnalysisException e) { - throw new CacheException("failed to loadSnapshot for: %s.%s.%s", - e, key.getCatalog().getName(), key.getDbName(), key.getTableName()); + } catch (Exception e) { + throw new CacheException("failed to load snapshot: %s.%s.%s with schema id: %s", + null, nameMapping.getCtlId(), nameMapping.getLocalDbName(), nameMapping.getLocalTblName()); } } + public PaimonSchemaCacheValue getPaimonSchemaCacheValue(NameMapping nameMapping, long schemaId) { + ExternalCatalog catalog = (ExternalCatalog) Env.getCurrentEnv().getCatalogMgr() + .getCatalog(nameMapping.getCtlId()); + if (catalog == null) { + throw new CacheException("catalog %s not found when getting paimon schema cache value", + null, nameMapping.getCtlId()); + } + ExternalSchemaCache cache = Env.getCurrentEnv().getExtMetaCacheMgr().getSchemaCache(catalog); + Optional schemaCacheValue = cache.getSchemaValue( + new PaimonSchemaCacheKey(nameMapping, schemaId)); + if (!schemaCacheValue.isPresent()) { + throw new CacheException("failed to get paimon schema cache value for: %s.%s.%s with schema id: %s", + null, nameMapping.getCtlId(), nameMapping.getLocalDbName(), nameMapping.getLocalTblName(), + schemaId); + } + return (PaimonSchemaCacheValue) schemaCacheValue.get(); + } + private PaimonPartitionInfo loadPartitionInfo(PaimonSnapshotCacheKey key, List partitionColumns) - throws IOException, AnalysisException { + throws AnalysisException { if (CollectionUtils.isEmpty(partitionColumns)) { return PaimonPartitionInfo.EMPTY; } - List paimonPartitions = ((PaimonExternalCatalog) key.getCatalog()) - .getPaimonPartitions(key.getDbName(), key.getTableName()); + NameMapping nameMapping = key.getNameMapping(); + PaimonExternalCatalog externalCatalog = (PaimonExternalCatalog) Env.getCurrentEnv().getCatalogMgr() + .getCatalogOrAnalysisException(nameMapping.getCtlId()); + List paimonPartitions = externalCatalog.getPaimonPartitions(nameMapping); return PaimonUtil.generatePartitionInfo(partitionColumns, paimonPartitions); } private PaimonSnapshot loadLatestSnapshot(PaimonSnapshotCacheKey key) throws IOException { - Table table = ((PaimonExternalCatalog) key.getCatalog()).getPaimonTable(key.getDbName(), key.getTableName()); + NameMapping nameMapping = key.getNameMapping(); + PaimonExternalCatalog externalCatalog = (PaimonExternalCatalog) Env.getCurrentEnv().getCatalogMgr() + .getCatalogOrException(nameMapping.getCtlId(), id -> new IOException("Catalog not found: " + id)); + Table table = externalCatalog.getPaimonTable(nameMapping); Table snapshotTable = table; // snapshotId and schemaId Long latestSnapshotId = PaimonSnapshot.INVALID_SNAPSHOT_ID; @@ -100,26 +127,27 @@ private PaimonSnapshot loadLatestSnapshot(PaimonSnapshotCacheKey key) throws IOE public void invalidateCatalogCache(long catalogId) { snapshotCache.asMap().keySet().stream() - .filter(key -> key.getCatalog().getId() == catalogId) + .filter(key -> key.getNameMapping().getCtlId() == catalogId) .forEach(snapshotCache::invalidate); } - public void invalidateTableCache(long catalogId, String dbName, String tblName) { + public void invalidateTableCache(ExternalTable dorisTable) { snapshotCache.asMap().keySet().stream() - .filter(key -> key.getCatalog().getId() == catalogId && key.getDbName().equals(dbName) - && key.getTableName().equals( - tblName)) + .filter(key -> key.getNameMapping().getCtlId() == dorisTable.getCatalog().getId() + && key.getNameMapping().getLocalDbName().equals(dorisTable.getDbName()) + && key.getNameMapping().getLocalTblName().equals(dorisTable.getName())) .forEach(snapshotCache::invalidate); } public void invalidateDbCache(long catalogId, String dbName) { snapshotCache.asMap().keySet().stream() - .filter(key -> key.getCatalog().getId() == catalogId && key.getDbName().equals(dbName)) + .filter(key -> key.getNameMapping().getCtlId() == catalogId + && key.getNameMapping().getLocalTblName().equals(dbName)) .forEach(snapshotCache::invalidate); } - public PaimonSnapshotCacheValue getPaimonSnapshot(CatalogIf catalog, String dbName, String tbName) { - PaimonSnapshotCacheKey key = new PaimonSnapshotCacheKey(catalog, dbName, tbName); + public PaimonSnapshotCacheValue getPaimonSnapshot(ExternalTable dorisTable) { + PaimonSnapshotCacheKey key = new PaimonSnapshotCacheKey(dorisTable.getOrBuildNameMapping()); return snapshotCache.get(key); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonMetadataCacheMgr.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonMetadataCacheMgr.java index a282fde665b197..4493f91893944e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonMetadataCacheMgr.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonMetadataCacheMgr.java @@ -17,6 +17,8 @@ package org.apache.doris.datasource.paimon; +import org.apache.doris.datasource.ExternalTable; + import java.util.concurrent.ExecutorService; public class PaimonMetadataCacheMgr { @@ -39,8 +41,8 @@ public void invalidateCatalogCache(long catalogId) { paimonMetadataCache.invalidateCatalogCache(catalogId); } - public void invalidateTableCache(long catalogId, String dbName, String tblName) { - paimonMetadataCache.invalidateTableCache(catalogId, dbName, tblName); + public void invalidateTableCache(ExternalTable dorisTable) { + paimonMetadataCache.invalidateTableCache(dorisTable); } public void invalidateDbCache(long catalogId, String dbName) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonSchemaCacheKey.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonSchemaCacheKey.java index f74555b369b380..2e5456c4432695 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonSchemaCacheKey.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonSchemaCacheKey.java @@ -18,14 +18,15 @@ package org.apache.doris.datasource.paimon; import org.apache.doris.datasource.ExternalSchemaCache.SchemaCacheKey; +import org.apache.doris.datasource.NameMapping; import com.google.common.base.Objects; public class PaimonSchemaCacheKey extends SchemaCacheKey { private final long schemaId; - public PaimonSchemaCacheKey(String dbName, String tableName, long schemaId) { - super(dbName, tableName); + public PaimonSchemaCacheKey(NameMapping nameMapping, long schemaId) { + super(nameMapping); this.schemaId = schemaId; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonSnapshotCacheKey.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonSnapshotCacheKey.java index 970f111a72133f..6154d607f0b2b3 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonSnapshotCacheKey.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonSnapshotCacheKey.java @@ -17,32 +17,19 @@ package org.apache.doris.datasource.paimon; -import org.apache.doris.datasource.CatalogIf; +import org.apache.doris.datasource.NameMapping; -import java.util.Objects; import java.util.StringJoiner; public class PaimonSnapshotCacheKey { - private final CatalogIf catalog; - private final String dbName; - private final String tableName; + private final NameMapping nameMapping; - public PaimonSnapshotCacheKey(CatalogIf catalog, String dbName, String tableName) { - this.catalog = catalog; - this.dbName = dbName; - this.tableName = tableName; + public PaimonSnapshotCacheKey(NameMapping nameMapping) { + this.nameMapping = nameMapping; } - public CatalogIf getCatalog() { - return catalog; - } - - public String getDbName() { - return dbName; - } - - public String getTableName() { - return tableName; + public NameMapping getNameMapping() { + return nameMapping; } @Override @@ -54,22 +41,20 @@ public boolean equals(Object o) { return false; } PaimonSnapshotCacheKey that = (PaimonSnapshotCacheKey) o; - return catalog.getId() == that.catalog.getId() - && Objects.equals(dbName, that.dbName) - && Objects.equals(tableName, that.tableName); + return nameMapping.equals(that.nameMapping); } @Override public int hashCode() { - return Objects.hash(catalog.getId(), dbName, tableName); + return nameMapping.hashCode(); } @Override public String toString() { return new StringJoiner(", ", PaimonSnapshotCacheKey.class.getSimpleName() + "[", "]") - .add("catalog=" + catalog) - .add("dbName='" + dbName + "'") - .add("tableName='" + tableName + "'") + .add("catalog=" + nameMapping.getCtlId()) + .add("dbName='" + nameMapping.getLocalDbName() + "'") + .add("tableName='" + nameMapping.getLocalTblName() + "'") .toString(); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/source/PaimonScanNode.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/source/PaimonScanNode.java index d8e0d1f72d555b..00449c4d9e47cc 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/source/PaimonScanNode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/source/PaimonScanNode.java @@ -18,6 +18,7 @@ package org.apache.doris.datasource.paimon.source; import org.apache.doris.analysis.TupleDescriptor; +import org.apache.doris.catalog.Env; import org.apache.doris.catalog.TableIf; import org.apache.doris.common.DdlException; import org.apache.doris.common.MetaNotFoundException; @@ -189,7 +190,8 @@ protected Optional getSerializedTable() { private Map getSchemaInfo(Long schemaId) { PaimonExternalTable table = (PaimonExternalTable) source.getTargetTable(); - TableSchema tableSchema = table.getPaimonSchemaCacheValue(schemaId).getTableSchema(); + TableSchema tableSchema = Env.getCurrentEnv().getExtMetaCacheMgr().getPaimonMetadataCache() + .getPaimonSchemaCacheValue(table.getOrBuildNameMapping(), schemaId).getTableSchema(); Map columnIdToName = new HashMap<>(tableSchema.fields().size()); for (DataField dataField : tableSchema.fields()) { columnIdToName.put(dataField.id(), dataField.name().toLowerCase()); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowCreateDatabaseCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowCreateDatabaseCommand.java index 29186e9e5cddee..364d4183e36dad 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowCreateDatabaseCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowCreateDatabaseCommand.java @@ -27,6 +27,7 @@ import org.apache.doris.common.ErrorReport; import org.apache.doris.common.util.PrintableMap; import org.apache.doris.datasource.CatalogIf; +import org.apache.doris.datasource.ExternalDatabase; import org.apache.doris.datasource.hive.HMSExternalCatalog; import org.apache.doris.datasource.iceberg.IcebergExternalCatalog; import org.apache.doris.datasource.iceberg.IcebergExternalDatabase; @@ -85,9 +86,10 @@ public ShowResultSet doRun(ConnectContext ctx, StmtExecutor executor) throws Exc CatalogIf catalog = Env.getCurrentEnv().getCatalogMgr().getCatalogOrAnalysisException(ctlgName); if (catalog instanceof HMSExternalCatalog) { String simpleDBName = ClusterNamespace.getNameFromFullName(databaseName); + ExternalDatabase dorisDb = ((HMSExternalCatalog) catalog).getDbOrAnalysisException(simpleDBName); org.apache.hadoop.hive.metastore.api.Database db = ((HMSExternalCatalog) catalog).getClient() - .getDatabase(simpleDBName); - sb.append("CREATE DATABASE `").append(simpleDBName).append("`") + .getDatabase(dorisDb.getRemoteName()); + sb.append("CREATE DATABASE `").append(dorisDb.getRemoteName()).append("`") .append(" LOCATION '") .append(db.getLocationUri()) .append("'"); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowPartitionsCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowPartitionsCommand.java index e8b06ce5a64f20..558117ed209594 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowPartitionsCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowPartitionsCommand.java @@ -39,6 +39,7 @@ import org.apache.doris.common.proc.ProcService; import org.apache.doris.common.util.OrderByPair; import org.apache.doris.datasource.CatalogIf; +import org.apache.doris.datasource.ExternalTable; import org.apache.doris.datasource.hive.HMSExternalCatalog; import org.apache.doris.datasource.hive.HMSExternalTable; import org.apache.doris.datasource.iceberg.IcebergExternalCatalog; @@ -369,11 +370,15 @@ private ShowResultSet handleShowHMSTablePartitions() throws AnalysisException { // example: insert into tmp partition(pt="1=3/3") values( xxx ); // show partitions from tmp: pt=1%3D3%2F3 // Need to consider whether to call `HiveUtil.toPartitionColNameAndValues` method + ExternalTable dorisTable = hmsCatalog.getDbOrAnalysisException(dbName) + .getTableOrAnalysisException(tableName.getTbl()); if (limit >= 0 && offset == 0 && (orderByPairs == null || !orderByPairs.get(0).isDesc())) { - partitionNames = hmsCatalog.getClient().listPartitionNames(dbName, tableName.getTbl(), limit); + partitionNames = hmsCatalog.getClient() + .listPartitionNames(dorisTable.getRemoteDbName(), dorisTable.getRemoteName(), limit); } else { - partitionNames = hmsCatalog.getClient().listPartitionNames(dbName, tableName.getTbl()); + partitionNames = hmsCatalog.getClient() + .listPartitionNames(dorisTable.getRemoteDbName(), dorisTable.getRemoteName()); } /* Filter add rows */ diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/HiveInsertExecutor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/HiveInsertExecutor.java index 99464ccfc01a90..64f68454d8449d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/HiveInsertExecutor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/HiveInsertExecutor.java @@ -18,8 +18,8 @@ package org.apache.doris.nereids.trees.plans.commands.insert; import org.apache.doris.common.UserException; -import org.apache.doris.common.info.SimpleTableInfo; import org.apache.doris.common.util.DebugUtil; +import org.apache.doris.datasource.ExternalTable; import org.apache.doris.datasource.hive.HMSExternalTable; import org.apache.doris.datasource.hive.HMSTransaction; import org.apache.doris.nereids.NereidsPlanner; @@ -64,9 +64,7 @@ protected void beforeExec() throws UserException { protected void doBeforeCommit() throws UserException { HMSTransaction transaction = (HMSTransaction) transactionManager.getTransaction(txnId); loadedRows = transaction.getUpdateCnt(); - String dbName = ((HMSExternalTable) table).getDbName(); - String tbName = table.getName(); - transaction.finishInsertTable(new SimpleTableInfo(dbName, tbName)); + transaction.finishInsertTable(((ExternalTable) table).getOrBuildNameMapping()); } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/IcebergInsertExecutor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/IcebergInsertExecutor.java index 2d08a66cd3c7e8..1eee1a525be777 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/IcebergInsertExecutor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/IcebergInsertExecutor.java @@ -18,7 +18,7 @@ package org.apache.doris.nereids.trees.plans.commands.insert; import org.apache.doris.common.UserException; -import org.apache.doris.common.info.SimpleTableInfo; +import org.apache.doris.datasource.NameMapping; import org.apache.doris.datasource.iceberg.IcebergExternalTable; import org.apache.doris.datasource.iceberg.IcebergTransaction; import org.apache.doris.nereids.NereidsPlanner; @@ -48,21 +48,19 @@ public IcebergInsertExecutor(ConnectContext ctx, IcebergExternalTable table, @Override protected void beforeExec() throws UserException { - String dbName = ((IcebergExternalTable) table).getRemoteDbName(); - String tbName = ((IcebergExternalTable) table).getRemoteName(); - SimpleTableInfo tableInfo = new SimpleTableInfo(dbName, tbName); IcebergTransaction transaction = (IcebergTransaction) transactionManager.getTransaction(txnId); - transaction.beginInsert(tableInfo); + transaction.beginInsert((IcebergExternalTable) table); } @Override protected void doBeforeCommit() throws UserException { - String dbName = ((IcebergExternalTable) table).getRemoteDbName(); - String tbName = ((IcebergExternalTable) table).getRemoteName(); - SimpleTableInfo tableInfo = new SimpleTableInfo(dbName, tbName); + IcebergExternalTable dorisTable = (IcebergExternalTable) table; + NameMapping nameMapping = new NameMapping(dorisTable.getCatalog().getId(), + dorisTable.getDbName(), dorisTable.getName(), + dorisTable.getRemoteDbName(), dorisTable.getRemoteName()); IcebergTransaction transaction = (IcebergTransaction) transactionManager.getTransaction(txnId); this.loadedRows = transaction.getUpdateCnt(); - transaction.finishInsert(tableInfo, insertCtx); + transaction.finishInsert(nameMapping, insertCtx); } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/HiveTableSink.java b/fe/fe-core/src/main/java/org/apache/doris/planner/HiveTableSink.java index 8818481a01b1bd..a232ef8b8b75f1 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/planner/HiveTableSink.java +++ b/fe/fe-core/src/main/java/org/apache/doris/planner/HiveTableSink.java @@ -197,7 +197,7 @@ private void setPartitionValues(THiveTableSink tSink) throws AnalysisException { List partitions = new ArrayList<>(); List hivePartitions = ((HMSExternalCatalog) targetTable.getCatalog()) - .getClient().listPartitions(targetTable.getDbName(), targetTable.getName()); + .getClient().listPartitions(targetTable.getRemoteDbName(), targetTable.getRemoteName()); for (org.apache.hadoop.hive.metastore.api.Partition partition : hivePartitions) { THivePartition hivePartition = new THivePartition(); StorageDescriptor sd = partition.getSd(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/tablefunction/IcebergTableValuedFunction.java b/fe/fe-core/src/main/java/org/apache/doris/tablefunction/IcebergTableValuedFunction.java index 25a3ad793429c7..a3b88cbb15d774 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/tablefunction/IcebergTableValuedFunction.java +++ b/fe/fe-core/src/main/java/org/apache/doris/tablefunction/IcebergTableValuedFunction.java @@ -20,12 +20,14 @@ import org.apache.doris.analysis.TableName; import org.apache.doris.catalog.Column; import org.apache.doris.catalog.Env; +import org.apache.doris.catalog.TableIf; import org.apache.doris.common.AnalysisException; import org.apache.doris.common.ErrorCode; import org.apache.doris.common.ErrorReport; import org.apache.doris.common.security.authentication.PreExecutionAuthenticator; import org.apache.doris.datasource.CatalogIf; import org.apache.doris.datasource.ExternalCatalog; +import org.apache.doris.datasource.ExternalTable; import org.apache.doris.datasource.iceberg.IcebergUtils; import org.apache.doris.mysql.privilege.PrivPredicate; import org.apache.doris.qe.ConnectContext; @@ -109,23 +111,21 @@ public IcebergTableValuedFunction(TableName icebergTableName, String queryType) ExternalCatalog externalCatalog = (ExternalCatalog) catalog; hadoopProps = externalCatalog.getCatalogProperty().getHadoopProperties(); preExecutionAuthenticator = externalCatalog.getPreExecutionAuthenticator(); - Table table; - try { - table = preExecutionAuthenticator.execute(() -> { - return IcebergUtils.getIcebergTable(externalCatalog, icebergTableName.getDb(), - icebergTableName.getTbl()); - }); - } catch (Exception e) { - throw new RuntimeException(ExceptionUtils.getRootCauseMessage(e)); + + TableIf dorisTable = externalCatalog.getDbOrAnalysisException(icebergTableName.getDb()) + .getTableOrAnalysisException(icebergTableName.getTbl()); + if (!(dorisTable instanceof ExternalTable)) { + throw new AnalysisException("Table " + icebergTableName + " is not an iceberg table"); } - if (table == null) { + Table icebergTable = IcebergUtils.getIcebergTable((ExternalTable) dorisTable); + if (icebergTable == null) { throw new AnalysisException("Iceberg table " + icebergTableName + " does not exist"); } MetadataTableType tableType = MetadataTableType.from(queryType); if (tableType == null) { throw new AnalysisException("Unrecognized queryType for iceberg metadata: " + queryType); } - this.sysTable = MetadataTableUtils.createMetadataTableInstance(table, tableType); + this.sysTable = MetadataTableUtils.createMetadataTableInstance(icebergTable, tableType); this.schema = IcebergUtils.parseSchema(sysTable.schema()); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/tablefunction/MetadataGenerator.java b/fe/fe-core/src/main/java/org/apache/doris/tablefunction/MetadataGenerator.java index 3c9428b44ed067..a3fa1a53937065 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/tablefunction/MetadataGenerator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/tablefunction/MetadataGenerator.java @@ -53,6 +53,7 @@ import org.apache.doris.datasource.CatalogMgr; import org.apache.doris.datasource.ExternalCatalog; import org.apache.doris.datasource.ExternalMetaCacheMgr; +import org.apache.doris.datasource.ExternalTable; import org.apache.doris.datasource.InternalCatalog; import org.apache.doris.datasource.TablePartitionValues; import org.apache.doris.datasource.hive.HMSExternalCatalog; @@ -355,10 +356,27 @@ private static TFetchSchemaTableDataResult hudiMetadataResult(TMetadataTableRequ if (catalog == null) { return errorResult("The specified catalog does not exist:" + hudiMetadataParams.getCatalog()); } + if (!(catalog instanceof ExternalCatalog)) { + return errorResult("The specified catalog is not an external catalog: " + + hudiMetadataParams.getCatalog()); + } + + ExternalTable dorisTable; + try { + dorisTable = (ExternalTable) catalog.getDbOrAnalysisException(hudiMetadataParams.getDatabase()) + .getTableOrAnalysisException(hudiMetadataParams.getTable()); + } catch (AnalysisException e) { + return errorResult("The specified db or table does not exist"); + } + + if (!(dorisTable instanceof HMSExternalTable)) { + return errorResult("The specified table is not a hudi table: " + hudiMetadataParams.getTable()); + } + HudiCachedMetaClientProcessor hudiMetadataCache = Env.getCurrentEnv().getExtMetaCacheMgr() .getHudiMetadataCacheMgr().getHudiMetaClientProcessor(catalog); String hudiBasePathString = ((HMSExternalCatalog) catalog).getClient() - .getTable(hudiMetadataParams.getDatabase(), hudiMetadataParams.getTable()).getSd().getLocation(); + .getTable(dorisTable.getRemoteDbName(), dorisTable.getRemoteName()).getSd().getLocation(); Configuration conf = ((HMSExternalCatalog) catalog).getConfiguration(); List dataBatch = Lists.newArrayList(); @@ -366,8 +384,8 @@ private static TFetchSchemaTableDataResult hudiMetadataResult(TMetadataTableRequ switch (hudiQueryType) { case TIMELINE: - HoodieTimeline timeline = hudiMetadataCache.getHoodieTableMetaClient(hudiMetadataParams.getDatabase(), - hudiMetadataParams.getTable(), hudiBasePathString, conf).getActiveTimeline(); + HoodieTimeline timeline = hudiMetadataCache.getHoodieTableMetaClient(dorisTable.getOrBuildNameMapping(), + hudiBasePathString, conf).getActiveTimeline(); for (HoodieInstant instant : timeline.getInstants()) { TRow trow = new TRow(); trow.addToColumnValue(new TCell().setStringVal(instant.getTimestamp())); @@ -995,9 +1013,9 @@ private static TFetchSchemaTableDataResult partitionMetadataResult(TMetadataTabl if (catalog instanceof InternalCatalog) { return dealInternalCatalog((Database) db, table); } else if (catalog instanceof MaxComputeExternalCatalog) { - return dealMaxComputeCatalog((MaxComputeExternalCatalog) catalog, dbName, tableName); + return dealMaxComputeCatalog((MaxComputeExternalCatalog) catalog, (ExternalTable) table); } else if (catalog instanceof HMSExternalCatalog) { - return dealHMSCatalog((HMSExternalCatalog) catalog, dbName, tableName); + return dealHMSCatalog((HMSExternalCatalog) catalog, (ExternalTable) table); } if (LOG.isDebugEnabled()) { @@ -1006,10 +1024,10 @@ private static TFetchSchemaTableDataResult partitionMetadataResult(TMetadataTabl return errorResult("not support catalog: " + catalogName); } - private static TFetchSchemaTableDataResult dealHMSCatalog(HMSExternalCatalog catalog, String dbName, - String tableName) { + private static TFetchSchemaTableDataResult dealHMSCatalog(HMSExternalCatalog catalog, ExternalTable table) { List dataBatch = Lists.newArrayList(); - List partitionNames = catalog.getClient().listPartitionNames(dbName, tableName); + List partitionNames = catalog.getClient() + .listPartitionNames(table.getRemoteDbName(), table.getRemoteName()); for (String partition : partitionNames) { TRow trow = new TRow(); trow.addToColumnValue(new TCell().setStringVal(partition)); @@ -1021,10 +1039,10 @@ private static TFetchSchemaTableDataResult dealHMSCatalog(HMSExternalCatalog cat return result; } - private static TFetchSchemaTableDataResult dealMaxComputeCatalog(MaxComputeExternalCatalog catalog, String dbName, - String tableName) { + private static TFetchSchemaTableDataResult dealMaxComputeCatalog(MaxComputeExternalCatalog catalog, + ExternalTable table) { List dataBatch = Lists.newArrayList(); - List partitionNames = catalog.listPartitionNames(dbName, tableName); + List partitionNames = catalog.listPartitionNames(table.getRemoteDbName(), table.getRemoteName()); for (String partition : partitionNames) { TRow trow = new TRow(); trow.addToColumnValue(new TCell().setStringVal(partition)); @@ -1689,7 +1707,7 @@ private static List partitionValuesMetadataResultForHmsTable(HMSExternalTa HiveMetaStoreCache cache = Env.getCurrentEnv().getExtMetaCacheMgr() .getMetaStoreCache((HMSExternalCatalog) tbl.getCatalog()); HiveMetaStoreCache.HivePartitionValues hivePartitionValues = cache.getPartitionValues( - tbl.getDbName(), tbl.getName(), tbl.getPartitionColumnTypes(MvccUtil.getSnapshotFromContext(tbl))); + tbl, tbl.getPartitionColumnTypes(MvccUtil.getSnapshotFromContext(tbl))); Map> valuesMap = hivePartitionValues.getPartitionValuesMap(); List dataBatch = Lists.newArrayList(); for (Map.Entry> entry : valuesMap.entrySet()) { diff --git a/fe/fe-core/src/test/java/org/apache/doris/catalog/constraint/ConstraintPersistTest.java b/fe/fe-core/src/test/java/org/apache/doris/catalog/constraint/ConstraintPersistTest.java index c169c7bc341a45..e33b4ae417d953 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/catalog/constraint/ConstraintPersistTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/catalog/constraint/ConstraintPersistTest.java @@ -18,17 +18,14 @@ package org.apache.doris.catalog.constraint; import org.apache.doris.catalog.Column; -import org.apache.doris.catalog.DatabaseIf; import org.apache.doris.catalog.Env; import org.apache.doris.catalog.PrimitiveType; import org.apache.doris.catalog.Table; import org.apache.doris.catalog.TableIf; import org.apache.doris.common.Config; -import org.apache.doris.common.Pair; +import org.apache.doris.common.FeConstants; import org.apache.doris.datasource.ExternalTable; -import org.apache.doris.datasource.es.EsExternalCatalog; -import org.apache.doris.datasource.es.EsExternalDatabase; -import org.apache.doris.datasource.es.EsExternalTable; +import org.apache.doris.datasource.test.TestExternalCatalog; import org.apache.doris.journal.JournalEntity; import org.apache.doris.nereids.util.PlanPatternMatchSupported; import org.apache.doris.nereids.util.RelationUtil; @@ -38,9 +35,8 @@ import org.apache.doris.utframe.TestWithFeService; import com.google.common.collect.ImmutableList; -import mockit.Mock; -import mockit.MockUp; -import org.apache.hadoop.util.Lists; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; @@ -55,7 +51,6 @@ import java.util.List; import java.util.Map; import java.util.Optional; -import java.util.concurrent.CopyOnWriteArrayList; class ConstraintPersistTest extends TestWithFeService implements PlanPatternMatchSupported { @@ -147,52 +142,6 @@ void dropConstraintLogPersistTest() throws Exception { Assertions.assertTrue(tableIf.getConstraintsMap().isEmpty()); } - @Test - void replayDropConstraintLogTest() throws Exception { - Config.edit_log_type = "local"; - ByteArrayOutputStream outputStream = new ByteArrayOutputStream(); - DataOutput output = new DataOutputStream(outputStream); - List> logs = new CopyOnWriteArrayList<>(); - EditLog editLog = new EditLog(""); - new MockUp() { - @Mock - public void logAddConstraint(AlterConstraintLog log) { - logs.add(Pair.of(OperationType.OP_ADD_CONSTRAINT, log)); - } - - @Mock - public void logDropConstraint(AlterConstraintLog log) { - logs.add(Pair.of(OperationType.OP_DROP_CONSTRAINT, log)); - } - }; - new MockUp() { - @Mock - public EditLog getEditLog() { - return editLog; - } - }; - addConstraint("alter table t1 add constraint pk primary key (k1)"); - addConstraint("alter table t2 add constraint pk primary key (k1)"); - addConstraint("alter table t1 add constraint uk unique (k1)"); - addConstraint("alter table t1 add constraint fk foreign key (k1) references t2(k1)"); - TableIf tableIf = RelationUtil.getTable( - RelationUtil.getQualifierName(connectContext, Lists.newArrayList("test", "t1")), - connectContext.getEnv(), Optional.empty()); - Assertions.assertEquals(3, tableIf.getConstraintsMap().size()); - dropConstraint("alter table t1 drop constraint uk"); - dropConstraint("alter table t1 drop constraint pk"); - dropConstraint("alter table t2 drop constraint pk"); - Assertions.assertEquals(0, tableIf.getConstraintsMap().size()); - for (Pair log : logs) { - JournalEntity journalEntity = new JournalEntity(); - journalEntity.setData(log.second); - journalEntity.setOpCode(log.first); - journalEntity.write(output); - } - Assertions.assertEquals(0, tableIf.getConstraintsMap().size()); - Assertions.assertEquals(0, tableIf.getConstraintsMap().size()); - } - @Test void constraintWithTablePersistTest() throws Exception { addConstraint("alter table t1 add constraint pk primary key (k1)"); @@ -235,47 +184,24 @@ void externalTableTest() throws Exception { @Test void addConstraintLogPersistForExternalTableTest() throws Exception { Config.edit_log_type = "local"; - createCatalog("create catalog es properties('type' = 'es', 'elasticsearch.hosts' = 'http://192.168.0.1'," - + " 'elasticsearch.username' = 'user1');"); - - Env.getCurrentEnv().changeCatalog(connectContext, "es"); - EsExternalCatalog esCatalog = (EsExternalCatalog) getCatalog("es"); - EsExternalDatabase db = new EsExternalDatabase(esCatalog, 10002, "es_db1", "es_db1"); - EsExternalTable tbl = new EsExternalTable(10003, "es_tbl1", "es_db1", esCatalog, db); - ImmutableList schema = ImmutableList.of(new Column("k1", PrimitiveType.INT)); - tbl.setNewFullSchema(schema); - db.addTableForTest(tbl); - esCatalog.addDatabaseForTest(db); - Env.getCurrentEnv().getExtMetaCacheMgr().getSchemaCache(esCatalog).addSchemaForTest(db.getFullName(), tbl.getName(), schema); - new MockUp() { - @Mock - public TableIf getTable(List qualifierName, Env env) { - return tbl; - } - }; + FeConstants.runningUnitTest = true; + createCatalog("create catalog extCtl1 properties(\n" + + " \"type\" = \"test\",\n" + + " \"catalog_provider.class\" " + + "= \"org.apache.doris.datasource.RefreshCatalogTest$RefreshCatalogProvider\"" + + ");"); - new MockUp() { - @Mock - public DatabaseIf getDatabase() { - return db; - } - }; - - new MockUp() { - @Mock - public TableIf toTableIf() { - return tbl; - } - }; - - addConstraint("alter table es.es_db1.es_tbl1 add constraint pk primary key (k1)"); - addConstraint("alter table es.es_db1.es_tbl1 add constraint uk unique (k1)"); TableIf tableIf = RelationUtil.getTable( - RelationUtil.getQualifierName(connectContext, Lists.newArrayList("test", "t1")), + RelationUtil.getQualifierName(connectContext, Lists.newArrayList("extCtl1", "db1", "tbl11")), connectContext.getEnv(), Optional.empty()); + + // add constraints + addConstraint("alter table extCtl1.db1.tbl11 add constraint pk primary key (a11)"); + addConstraint("alter table extCtl1.db1.tbl11 add constraint uk unique (a11)"); + Assertions.assertEquals(2, tableIf.getConstraintsMap().size()); + // clear the constraints Map constraintMap = tableIf.getConstraintsMap(); - tableIf.getConstraintsMapUnsafe().clear(); - Assertions.assertTrue(tableIf.getConstraintsMap().isEmpty()); + // save constraints map in edit log ByteArrayOutputStream outputStream = new ByteArrayOutputStream(); DataOutput output = new DataOutputStream(outputStream); for (Constraint value : new ArrayList<>(constraintMap.values())) { @@ -284,6 +210,10 @@ public TableIf toTableIf() { journalEntity.setOpCode(OperationType.OP_ADD_CONSTRAINT); journalEntity.write(output); } + // clear constraints map manually + tableIf.getConstraintsMapUnsafe().clear(); + Assertions.assertTrue(tableIf.getConstraintsMap().isEmpty()); + // add constraints back from edit log InputStream inputStream = new ByteArrayInputStream(outputStream.toByteArray()); DataInput input = new DataInputStream(inputStream); for (int i = 0; i < constraintMap.values().size(); i++) { @@ -291,50 +221,30 @@ public TableIf toTableIf() { journalEntity.readFields(input); EditLog.loadJournal(Env.getCurrentEnv(), 0L, journalEntity); } - Assertions.assertEquals(tableIf.getConstraintsMap(), constraintMap); - Env.getCurrentEnv().changeCatalog(connectContext, "internal"); + Assertions.assertEquals(2, tableIf.getConstraintsMap().size()); } @Test void dropConstraintLogPersistForExternalTest() throws Exception { Config.edit_log_type = "local"; - createCatalog("create catalog es2 properties('type' = 'es', 'elasticsearch.hosts' = 'http://192.168.0.1'," - + " 'elasticsearch.username' = 'user1');"); - - Env.getCurrentEnv().changeCatalog(connectContext, "es2"); - EsExternalCatalog esCatalog = (EsExternalCatalog) getCatalog("es2"); - EsExternalDatabase db = new EsExternalDatabase(esCatalog, 10002, "es_db1", "es_db1"); - EsExternalTable tbl = new EsExternalTable(10003, "es_tbl1", "es_db1", esCatalog, db); - ImmutableList schema = ImmutableList.of(new Column("k1", PrimitiveType.INT)); - tbl.setNewFullSchema(schema); - db.addTableForTest(tbl); - esCatalog.addDatabaseForTest(db); - Env.getCurrentEnv().getExtMetaCacheMgr().getSchemaCache(esCatalog).addSchemaForTest(db.getFullName(), tbl.getName(), schema); - new MockUp() { - @Mock - public TableIf getTable(List qualifierName, Env env) { - return tbl; - } - }; + FeConstants.runningUnitTest = true; + createCatalog("create catalog extCtl2 properties(\n" + + " \"type\" = \"test\",\n" + + " \"catalog_provider.class\" " + + "= \"org.apache.doris.datasource.RefreshCatalogTest$RefreshCatalogProvider\"" + + ");"); - new MockUp() { - @Mock - public DatabaseIf getDatabase() { - return db; - } - }; - - new MockUp() { - @Mock - public TableIf toTableIf() { - return tbl; - } - }; - addConstraint("alter table es.es_db1.es_tbl1 add constraint pk primary key (k1)"); - addConstraint("alter table es.es_db1.es_tbl1 add constraint uk unique (k1)"); TableIf tableIf = RelationUtil.getTable( - RelationUtil.getQualifierName(connectContext, Lists.newArrayList("test", "t1")), + RelationUtil.getQualifierName(connectContext, Lists.newArrayList("extCtl2", "db1", "tbl11")), connectContext.getEnv(), Optional.empty()); + + // add constraints + addConstraint("alter table extCtl2.db1.tbl11 add constraint pk primary key (a11)"); + addConstraint("alter table extCtl2.db1.tbl11 add constraint uk unique (a11)"); + Assertions.assertEquals(2, tableIf.getConstraintsMap().size()); + // drop it + // dropConstraint("alter table extCtl2.db1.tbl11 drop constraint pk"); + // dropConstraint("alter table extCtl2.db1.tbl11 drop constraint uk"); Map constraintMap = tableIf.getConstraintsMap(); ByteArrayOutputStream outputStream = new ByteArrayOutputStream(); DataOutput output = new DataOutputStream(outputStream); @@ -352,6 +262,38 @@ public TableIf toTableIf() { EditLog.loadJournal(Env.getCurrentEnv(), 0L, journalEntity); } Assertions.assertTrue(tableIf.getConstraintsMap().isEmpty()); + Env.getCurrentEnv().changeCatalog(connectContext, "internal"); } + + public static class RefreshCatalogProvider implements TestExternalCatalog.TestCatalogProvider { + public static final Map>> MOCKED_META; + + static { + MOCKED_META = Maps.newHashMap(); + Map> tblSchemaMap1 = Maps.newHashMap(); + // db1 + tblSchemaMap1.put("tbl11", Lists.newArrayList( + new Column("a11", PrimitiveType.BIGINT), + new Column("a12", PrimitiveType.STRING), + new Column("a13", PrimitiveType.FLOAT))); + tblSchemaMap1.put("tbl12", Lists.newArrayList( + new Column("b21", PrimitiveType.BIGINT), + new Column("b22", PrimitiveType.STRING), + new Column("b23", PrimitiveType.FLOAT))); + MOCKED_META.put("db1", tblSchemaMap1); + // db2 + Map> tblSchemaMap2 = Maps.newHashMap(); + tblSchemaMap2.put("tbl21", Lists.newArrayList( + new Column("c11", PrimitiveType.BIGINT), + new Column("c12", PrimitiveType.STRING), + new Column("c13", PrimitiveType.FLOAT))); + MOCKED_META.put("db2", tblSchemaMap2); + } + + @Override + public Map>> getMetadata() { + return MOCKED_META; + } + } } diff --git a/fe/fe-core/src/test/java/org/apache/doris/datasource/TestHMSCachedClient.java b/fe/fe-core/src/test/java/org/apache/doris/datasource/TestHMSCachedClient.java index 5d4474d77ec21e..d5bd0c3c6d03fa 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/datasource/TestHMSCachedClient.java +++ b/fe/fe-core/src/test/java/org/apache/doris/datasource/TestHMSCachedClient.java @@ -18,7 +18,6 @@ package org.apache.doris.datasource; import org.apache.doris.analysis.TableName; -import org.apache.doris.common.info.SimpleTableInfo; import org.apache.doris.datasource.hive.HMSCachedClient; import org.apache.doris.datasource.hive.HiveDatabaseMetadata; import org.apache.doris.datasource.hive.HivePartitionStatistics; @@ -47,7 +46,7 @@ public class TestHMSCachedClient implements HMSCachedClient { - public Map> partitions = new ConcurrentHashMap<>(); + public Map> partitions = new ConcurrentHashMap<>(); public Map> tables = new HashMap<>(); public List dbs = new ArrayList<>(); @@ -231,7 +230,7 @@ public void dropDatabase(String dbName) { public void dropTable(String dbName, String tableName) { Table table = getTable(dbName, tableName); this.tables.get(dbName).remove(table); - this.partitions.remove(new SimpleTableInfo(dbName, tableName)); + this.partitions.remove(NameMapping.createForTest(dbName, tableName)); } @Override @@ -247,8 +246,7 @@ public void createTable(TableMetadata tbl, boolean ignoreIfExists) { List tableList = getTableList(tbl.getDbName()); tableList.add(HiveUtil.toHiveTable((HiveTableMetadata) tbl)); - SimpleTableInfo key = new SimpleTableInfo(dbName, tbName); - partitions.put(key, new ArrayList<>()); + partitions.put(NameMapping.createForTest(dbName, tbName), new ArrayList<>()); } @Override @@ -321,10 +319,10 @@ public void dropPartition(String dbName, String tableName, List partitio } public List getPartitionList(String dbName, String tableName) { - SimpleTableInfo key = new SimpleTableInfo(dbName, tableName); - List partitionList = this.partitions.get(key); + NameMapping nameMapping = NameMapping.createForTest(dbName, tableName); + List partitionList = this.partitions.get(NameMapping.createForTest(dbName, tableName)); if (partitionList == null) { - throw new RuntimeException("can't found table: " + key); + throw new RuntimeException("can't found table: " + nameMapping.getFullLocalName()); } return partitionList; } diff --git a/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HiveAcidTest.java b/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HiveAcidTest.java index 937dedfb10bb7b..2a879a24142fa0 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HiveAcidTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HiveAcidTest.java @@ -17,7 +17,7 @@ package org.apache.doris.datasource.hive; -import org.apache.doris.common.info.SimpleTableInfo; +import org.apache.doris.datasource.NameMapping; import org.apache.doris.datasource.hive.HiveMetaStoreCache.FileCacheValue; import org.apache.doris.datasource.property.storage.LocalProperties; import org.apache.doris.datasource.property.storage.StorageProperties; @@ -70,7 +70,7 @@ public void testOriginalDeltas() throws Exception { AcidUtil.VALID_WRITEIDS_KEY, new ValidReaderWriteIdList("tbl:100:" + Long.MAX_VALUE + ":").writeToString()); - HivePartition partition = new HivePartition(new SimpleTableInfo("", "tbl"), + HivePartition partition = new HivePartition(NameMapping.createForTest("", "tbl"), false, "", "file://" + tempPath.toAbsolutePath() + "", new ArrayList<>(), new HashMap<>()); try { @@ -99,7 +99,7 @@ public void testObsoleteOriginals() throws Exception { new ValidReaderWriteIdList("tbl:150:" + Long.MAX_VALUE + ":").writeToString()); HivePartition partition = new HivePartition( - new SimpleTableInfo("", "tbl"), + NameMapping.createForTest("", "tbl"), false, "", "file://" + tempPath.toAbsolutePath() + "", @@ -135,7 +135,7 @@ public void testOverlapingDelta() throws Exception { new ValidReaderWriteIdList("tbl:100:" + Long.MAX_VALUE + ":").writeToString()); HivePartition partition = new HivePartition( - new SimpleTableInfo("", "tbl"), + NameMapping.createForTest("", "tbl"), false, "", "file://" + tempPath.toAbsolutePath() + "", @@ -186,7 +186,7 @@ public void testOverlapingDelta2() throws Exception { new ValidReaderWriteIdList("tbl:100:" + Long.MAX_VALUE + ":").writeToString()); HivePartition partition = new HivePartition( - new SimpleTableInfo("", "tbl"), + NameMapping.createForTest("", "tbl"), false, "", "file://" + tempPath.toAbsolutePath() + "", @@ -233,7 +233,7 @@ public void deltasWithOpenTxnInRead() throws Exception { new ValidReaderWriteIdList("tbl:100:4:4").writeToString()); HivePartition partition = new HivePartition( - new SimpleTableInfo("", "tbl"), + NameMapping.createForTest("", "tbl"), false, "", "file://" + tempPath.toAbsolutePath() + "", @@ -280,7 +280,7 @@ public void deltasWithOpenTxnInRead2() throws Exception { new ValidReaderWriteIdList("tbl:100:4:4").writeToString()); HivePartition partition = new HivePartition( - new SimpleTableInfo("", "tbl"), + NameMapping.createForTest("", "tbl"), false, "", "file://" + tempPath.toAbsolutePath() + "", @@ -333,7 +333,7 @@ public void testBaseWithDeleteDeltas() throws Exception { // AcidUtils.AcidOperationalProperties.getDefault().toString()); HivePartition partition = new HivePartition( - new SimpleTableInfo("", "tbl"), + NameMapping.createForTest("", "tbl"), false, "", "file://" + tempPath.toAbsolutePath() + "", @@ -403,7 +403,7 @@ public void testOverlapingDeltaAndDeleteDelta() throws Exception { // AcidUtils.AcidOperationalProperties.getDefault().toString()); HivePartition partition = new HivePartition( - new SimpleTableInfo("", "tbl"), + NameMapping.createForTest("", "tbl"), false, "", "file://" + tempPath.toAbsolutePath() + "", @@ -469,7 +469,7 @@ public void testMinorCompactedDeltaMakesInBetweenDelteDeltaObsolete() throws Exc Map tableProps = new HashMap<>(); HivePartition partition = new HivePartition( - new SimpleTableInfo("", "tbl"), + NameMapping.createForTest("", "tbl"), false, "", "file://" + tempPath.toAbsolutePath() + "", @@ -515,7 +515,7 @@ public void deleteDeltasWithOpenTxnInRead() throws Exception { Map tableProps = new HashMap<>(); HivePartition partition = new HivePartition( - new SimpleTableInfo("", "tbl"), + NameMapping.createForTest("", "tbl"), false, "", "file://" + tempPath.toAbsolutePath() + "", @@ -578,7 +578,7 @@ public void testBaseDeltas() throws Exception { Map tableProps = new HashMap<>(); HivePartition partition = new HivePartition( - new SimpleTableInfo("", "tbl"), + NameMapping.createForTest("", "tbl"), false, "", "file://" + tempPath.toAbsolutePath() + "", diff --git a/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HiveMetaStoreCacheTest.java b/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HiveMetaStoreCacheTest.java index 83e5a93e914dd5..f10f7868660088 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HiveMetaStoreCacheTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HiveMetaStoreCacheTest.java @@ -18,6 +18,8 @@ package org.apache.doris.datasource.hive; import org.apache.doris.common.ThreadPoolManager; +import org.apache.doris.common.util.Util; +import org.apache.doris.datasource.NameMapping; import com.github.benmanes.caffeine.cache.LoadingCache; import org.junit.jupiter.api.Assertions; @@ -57,12 +59,12 @@ public void testInvalidateTableCache() { Assertions.assertEquals(2, partitionCache.asMap().size()); Assertions.assertEquals(2, partitionValuesCache.asMap().size()); - hiveMetaStoreCache.invalidateTableCache(dbName, tbName2); + hiveMetaStoreCache.invalidateTableCache(NameMapping.createForTest(dbName, tbName2)); Assertions.assertEquals(2, fileCache.asMap().size()); Assertions.assertEquals(1, partitionCache.asMap().size()); Assertions.assertEquals(1, partitionValuesCache.asMap().size()); - hiveMetaStoreCache.invalidateTableCache(dbName, tbName); + hiveMetaStoreCache.invalidateTableCache(NameMapping.createForTest(dbName, tbName)); Assertions.assertEquals(0, fileCache.asMap().size()); Assertions.assertEquals(0, partitionCache.asMap().size()); Assertions.assertEquals(0, partitionValuesCache.asMap().size()); @@ -73,19 +75,22 @@ private void putCache( LoadingCache partitionCache, LoadingCache partitionValuesCache, String dbName, String tbName) { - HiveMetaStoreCache.FileCacheKey fileCacheKey1 = new HiveMetaStoreCache.FileCacheKey(dbName, tbName, tbName, "", new ArrayList<>()); - HiveMetaStoreCache.FileCacheKey fileCacheKey2 = HiveMetaStoreCache.FileCacheKey.createDummyCacheKey(dbName, tbName, tbName, ""); + NameMapping nameMapping = NameMapping.createForTest(dbName, tbName); + long fileId = Util.genIdByName(dbName, tbName); + HiveMetaStoreCache.FileCacheKey fileCacheKey1 = new HiveMetaStoreCache.FileCacheKey(fileId, tbName, "", new ArrayList<>()); + HiveMetaStoreCache.FileCacheKey fileCacheKey2 = HiveMetaStoreCache.FileCacheKey.createDummyCacheKey(fileId, tbName, ""); fileCache.put(fileCacheKey1, new HiveMetaStoreCache.FileCacheValue()); fileCache.put(fileCacheKey2, new HiveMetaStoreCache.FileCacheValue()); HiveMetaStoreCache.PartitionCacheKey partitionCacheKey = new HiveMetaStoreCache.PartitionCacheKey( - dbName, - tbName, + nameMapping, new ArrayList<>() ); - partitionCache.put(partitionCacheKey, new HivePartition(dbName, tbName, false, "", "", new ArrayList<>(), new HashMap<>())); + partitionCache.put(partitionCacheKey, + new HivePartition(nameMapping, false, "", "", new ArrayList<>(), new HashMap<>())); - HiveMetaStoreCache.PartitionValueCacheKey partitionValueCacheKey = new HiveMetaStoreCache.PartitionValueCacheKey(dbName, tbName, new ArrayList<>()); + HiveMetaStoreCache.PartitionValueCacheKey partitionValueCacheKey + = new HiveMetaStoreCache.PartitionValueCacheKey(nameMapping, new ArrayList<>()); partitionValuesCache.put(partitionValueCacheKey, new HiveMetaStoreCache.HivePartitionValues()); } diff --git a/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HiveMetadataOpsTest.java b/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HiveMetadataOpsTest.java index 4ef250de507fa1..ab7b076a866618 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HiveMetadataOpsTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HiveMetadataOpsTest.java @@ -17,12 +17,8 @@ package org.apache.doris.datasource.hive; -import org.apache.doris.analysis.CreateDbStmt; import org.apache.doris.analysis.CreateTableStmt; -import org.apache.doris.analysis.DbName; import org.apache.doris.analysis.DistributionDesc; -import org.apache.doris.analysis.DropDbStmt; -import org.apache.doris.analysis.DropTableStmt; import org.apache.doris.analysis.HashDistributionDesc; import org.apache.doris.analysis.KeysDesc; import org.apache.doris.analysis.PartitionDesc; @@ -100,13 +96,11 @@ public void createTable(TableMetadata catalogTable, boolean ignoreIfExists) { } private void createDb(String dbName, Map props) throws DdlException { - CreateDbStmt createDbStmt = new CreateDbStmt(true, new DbName("hive", dbName), props); - metadataOps.createDb(createDbStmt); + metadataOps.createDb(dbName, true, props); } private void dropDb(String dbName, boolean forceDrop) throws DdlException { - DropDbStmt dropDbStmt = new DropDbStmt(true, new DbName("hive", dbName), forceDrop); - metadataOps.dropDb(dropDbStmt); + metadataOps.dropDb(dbName, true, forceDrop); } private void createTable(TableName tableName, @@ -136,8 +130,11 @@ private void createTable(TableName tableName, } private void dropTable(TableName tableName, boolean forceDrop) throws DdlException { - DropTableStmt dropTblStmt = new DropTableStmt(true, tableName, forceDrop); - metadataOps.dropTable(dropTblStmt); + HMSExternalDatabase externalDatabase = new HMSExternalDatabase( + mockedCatalog, 0, tableName.getDb(), tableName.getDb()); + HMSExternalTable externalTable = new HMSExternalTable( + 1, tableName.getTbl(), tableName.getTbl(), mockedCatalog, externalDatabase); + metadataOps.dropTable(externalTable, true); } @Test diff --git a/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HmsCommitTest.java b/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HmsCommitTest.java index f17a46aab92d9b..72e2639a89ada6 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HmsCommitTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HmsCommitTest.java @@ -21,8 +21,8 @@ import org.apache.doris.backup.Status; import org.apache.doris.catalog.Column; import org.apache.doris.catalog.PrimitiveType; -import org.apache.doris.common.info.SimpleTableInfo; import org.apache.doris.common.util.DebugUtil; +import org.apache.doris.datasource.NameMapping; import org.apache.doris.datasource.TestHMSCachedClient; import org.apache.doris.fs.FileSystem; import org.apache.doris.fs.FileSystemProvider; @@ -412,7 +412,7 @@ public void commit(String dbName, ctx.setQueryId(queryId); ctx.setWritePath(getWritePath()); hmsTransaction.beginInsertTable(ctx); - hmsTransaction.finishInsertTable(new SimpleTableInfo(dbName, tableName)); + hmsTransaction.finishInsertTable(NameMapping.createForTest(dbName, tableName)); hmsTransaction.commit(); } @@ -701,7 +701,7 @@ public void testCommitWithRollback() { ctx.setQueryId(queryId); ctx.setWritePath(getWritePath()); hmsTransaction.beginInsertTable(ctx); - hmsTransaction.finishInsertTable(new SimpleTableInfo(dbName, tbWithoutPartition)); + hmsTransaction.finishInsertTable(NameMapping.createForTest(dbName, tbWithoutPartition)); hmsTransaction.commit(); Assert.fail(); } catch (Throwable t) { diff --git a/fe/fe-core/src/test/java/org/apache/doris/datasource/iceberg/CreateIcebergTableTest.java b/fe/fe-core/src/test/java/org/apache/doris/datasource/iceberg/CreateIcebergTableTest.java index 45fe77387835cf..e1d127d36c7dc1 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/datasource/iceberg/CreateIcebergTableTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/datasource/iceberg/CreateIcebergTableTest.java @@ -18,10 +18,7 @@ package org.apache.doris.datasource.iceberg; import org.apache.doris.analysis.CreateCatalogStmt; -import org.apache.doris.analysis.CreateDbStmt; import org.apache.doris.analysis.CreateTableStmt; -import org.apache.doris.analysis.DbName; -import org.apache.doris.analysis.DropDbStmt; import org.apache.doris.common.DdlException; import org.apache.doris.common.UserException; import org.apache.doris.datasource.CatalogFactory; @@ -31,6 +28,7 @@ import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; import org.apache.doris.qe.ConnectContext; +import com.google.common.collect.Maps; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; import org.apache.iceberg.Table; @@ -77,8 +75,7 @@ public static void beforeClass() throws Throwable { // create db ops = new IcebergMetadataOps(icebergCatalog, icebergCatalog.getCatalog()); - CreateDbStmt createDbStmt = new CreateDbStmt(true, new DbName("iceberg", dbName), null); - ops.createDb(createDbStmt); + ops.createDb(dbName, true, Maps.newHashMap()); if (icebergCatalog.getUseMetaCache().get()) { icebergCatalog.makeSureInitialized(); } else { @@ -206,21 +203,17 @@ public String getTableName() { @Test public void testDropDB() { - String dbName = "db_to_delete"; - CreateDbStmt createDBStmt = new CreateDbStmt(false, new DbName("iceberg", dbName), new HashMap<>()); - DropDbStmt dropDbStmt = new DropDbStmt(false, new DbName("iceberg", dbName), false); - DropDbStmt dropDbStmt2 = new DropDbStmt(false, new DbName("iceberg", "not_exists"), false); try { // create db success - ops.createDb(createDBStmt); + ops.createDb("iceberg", false, Maps.newHashMap()); // drop db success - ops.dropDb(dropDbStmt); + ops.dropDb("iceberg", false, false); } catch (Throwable t) { Assert.fail(); } try { - ops.dropDb(dropDbStmt2); + ops.dropDb("iceberg", false, false); Assert.fail(); } catch (Throwable t) { Assert.assertTrue(t instanceof DdlException); diff --git a/fe/fe-core/src/test/java/org/apache/doris/datasource/iceberg/IcebergExternalTableBranchAndTagTest.java b/fe/fe-core/src/test/java/org/apache/doris/datasource/iceberg/IcebergExternalTableBranchAndTagTest.java index 3a65fe9cf6b596..b61199fa0bae3f 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/datasource/iceberg/IcebergExternalTableBranchAndTagTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/datasource/iceberg/IcebergExternalTableBranchAndTagTest.java @@ -18,6 +18,7 @@ package org.apache.doris.datasource.iceberg; import org.apache.doris.catalog.Env; +import org.apache.doris.catalog.RefreshManager; import org.apache.doris.common.UserException; import org.apache.doris.nereids.trees.plans.commands.info.BranchOptions; import org.apache.doris.nereids.trees.plans.commands.info.CreateOrReplaceBranchInfo; @@ -93,7 +94,7 @@ public void setUp() throws IOException { // mock IcebergUtils.getIcebergTable to return our test icebergTable mockedIcebergUtils = Mockito.mockStatic(IcebergUtils.class); - mockedIcebergUtils.when(() -> IcebergUtils.getIcebergTable(Mockito.any(), Mockito.any(), Mockito.any())) + mockedIcebergUtils.when(() -> IcebergUtils.getIcebergTable(Mockito.any())) .thenReturn(icebergTable); // mock Env.getCurrentEnv().getEditLog().logBranchOrTag(info) to do nothing @@ -103,6 +104,14 @@ public void setUp() throws IOException { mockedEnv.when(Env::getCurrentEnv).thenReturn(mockEnv); Mockito.when(mockEnv.getEditLog()).thenReturn(mockEditLog); Mockito.doNothing().when(mockEditLog).logBranchOrTag(Mockito.any()); + + // mock refresh table after branch/tag operation + // Env.getCurrentEnv().getRefreshManager() + // .refreshTableInternal(dorisCatalog, db, tbl, System.currentTimeMillis()); + RefreshManager refreshManager = Mockito.mock(RefreshManager.class); + Mockito.when(mockEnv.getRefreshManager()).thenReturn(refreshManager); + Mockito.doNothing().when(refreshManager) + .refreshTableInternal(Mockito.any(), Mockito.any(), Mockito.any(), Mockito.anyLong()); } @AfterEach @@ -134,7 +143,7 @@ public void testCreateTagWithTable() throws UserException, IOException { new CreateOrReplaceTagInfo(tag1, true, false, false, TagOptions.EMPTY); Assertions.assertThrows( UserException.class, - () -> catalog.createOrReplaceTag(dbName, tblName, info)); + () -> catalog.createOrReplaceTag(dorisTable, info)); // add some data addSomeDataIntoIcebergTable(); @@ -142,7 +151,7 @@ public void testCreateTagWithTable() throws UserException, IOException { Assertions.assertEquals(1, snapshots.size()); // create a new tag: tag1 - catalog.createOrReplaceTag(dbName, tblName, info); + catalog.createOrReplaceTag(dorisTable, info); assertSnapshotRef( icebergTable.refs().get(tag1), icebergTable.currentSnapshot().snapshotId(), @@ -151,12 +160,12 @@ public void testCreateTagWithTable() throws UserException, IOException { // create an existed tag: tag1 Assertions.assertThrows( IllegalArgumentException.class, - () -> catalog.createOrReplaceTag(dbName, tblName, info)); + () -> catalog.createOrReplaceTag(dorisTable, info)); // create an existed tag with replace CreateOrReplaceTagInfo info2 = new CreateOrReplaceTagInfo(tag1, true, true, false, TagOptions.EMPTY); - catalog.createOrReplaceTag(dbName, tblName, info2); + catalog.createOrReplaceTag(dorisTable, info2); assertSnapshotRef( icebergTable.refs().get(tag1), icebergTable.currentSnapshot().snapshotId(), @@ -165,7 +174,7 @@ public void testCreateTagWithTable() throws UserException, IOException { // create an existed tag with if not exists CreateOrReplaceTagInfo info3 = new CreateOrReplaceTagInfo(tag1, true, false, true, TagOptions.EMPTY); - catalog.createOrReplaceTag(dbName, tblName, info3); + catalog.createOrReplaceTag(dorisTable, info3); assertSnapshotRef( icebergTable.refs().get(tag1), icebergTable.currentSnapshot().snapshotId(), @@ -183,7 +192,7 @@ public void testCreateTagWithTable() throws UserException, IOException { Optional.empty()); CreateOrReplaceTagInfo info4 = new CreateOrReplaceTagInfo(tag2, true, false, false, tagOps); - catalog.createOrReplaceTag(dbName, tblName, info4); + catalog.createOrReplaceTag(dorisTable, info4); assertSnapshotRef( icebergTable.refs().get(tag2), snapshots.get(1).snapshotId(), @@ -195,7 +204,7 @@ public void testCreateTagWithTable() throws UserException, IOException { Optional.of(2L)); CreateOrReplaceTagInfo info5 = new CreateOrReplaceTagInfo(tag2, true, true, false, tagOps2); - catalog.createOrReplaceTag(dbName, tblName, info5); + catalog.createOrReplaceTag(dorisTable, info5); assertSnapshotRef( icebergTable.refs().get(tag2), icebergTable.currentSnapshot().snapshotId(), @@ -204,7 +213,7 @@ public void testCreateTagWithTable() throws UserException, IOException { // create new tag: tag3 CreateOrReplaceTagInfo info6 = new CreateOrReplaceTagInfo(tag3, true, false, false, tagOps2); - catalog.createOrReplaceTag(dbName, tblName, info6); + catalog.createOrReplaceTag(dorisTable, info6); assertSnapshotRef( icebergTable.refs().get(tag3), icebergTable.currentSnapshot().snapshotId(), @@ -223,7 +232,7 @@ public void testCreateBranchWithNotEmptyTable() throws UserException, IOExceptio // create a new branch: branch1 CreateOrReplaceBranchInfo info = new CreateOrReplaceBranchInfo(branch1, true, false, false, BranchOptions.EMPTY); - catalog.createOrReplaceBranch(dbName, tblName, info); + catalog.createOrReplaceBranch(dorisTable, info); List snapshots = Lists.newArrayList(icebergTable.snapshots()); Assertions.assertEquals(1, snapshots.size()); assertSnapshotRef( @@ -234,7 +243,7 @@ public void testCreateBranchWithNotEmptyTable() throws UserException, IOExceptio // create an existed branch, failed Assertions.assertThrows( IllegalArgumentException.class, - () -> catalog.createOrReplaceBranch(dbName, tblName, info)); + () -> catalog.createOrReplaceBranch(dorisTable, info)); // create or replace an empty branch, will fail // because cannot perform a replace operation on an empty branch. @@ -242,12 +251,12 @@ public void testCreateBranchWithNotEmptyTable() throws UserException, IOExceptio new CreateOrReplaceBranchInfo(branch1, true, true, false, BranchOptions.EMPTY); Assertions.assertThrows( UserException.class, - () -> catalog.createOrReplaceBranch(dbName, tblName, info2)); + () -> catalog.createOrReplaceBranch(dorisTable, info2)); // create an existed branch with ifNotExists CreateOrReplaceBranchInfo info4 = new CreateOrReplaceBranchInfo(branch1, true, false, true, BranchOptions.EMPTY); - catalog.createOrReplaceBranch(dbName, tblName, info4); + catalog.createOrReplaceBranch(dorisTable, info4); assertSnapshotRef( icebergTable.refs().get(branch1), snapshots.get(0).snapshotId(), @@ -259,7 +268,7 @@ public void testCreateBranchWithNotEmptyTable() throws UserException, IOExceptio Assertions.assertEquals(2, snapshots.size()); // update branch1 - catalog.createOrReplaceBranch(dbName, tblName, info2); + catalog.createOrReplaceBranch(dorisTable, info2); assertSnapshotRef( icebergTable.refs().get(branch1), icebergTable.currentSnapshot().snapshotId(), @@ -268,7 +277,7 @@ public void testCreateBranchWithNotEmptyTable() throws UserException, IOExceptio // create or replace a new branch: branch2 CreateOrReplaceBranchInfo info3 = new CreateOrReplaceBranchInfo(branch2, true, true, false, BranchOptions.EMPTY); - catalog.createOrReplaceBranch(dbName, tblName, info3); + catalog.createOrReplaceBranch(dorisTable, info3); assertSnapshotRef( icebergTable.refs().get(branch2), icebergTable.currentSnapshot().snapshotId(), @@ -282,7 +291,7 @@ public void testCreateBranchWithNotEmptyTable() throws UserException, IOExceptio Optional.of(3L)); CreateOrReplaceBranchInfo info5 = new CreateOrReplaceBranchInfo(branch2, true, true, false, brOps); - catalog.createOrReplaceBranch(dbName, tblName, info5); + catalog.createOrReplaceBranch(dorisTable, info5); assertSnapshotRef( icebergTable.refs().get(branch2), icebergTable.currentSnapshot().snapshotId(), @@ -308,14 +317,14 @@ public void testCreateBranchWithNotEmptyTable() throws UserException, IOExceptio Optional.of(3L)); CreateOrReplaceBranchInfo info6 = new CreateOrReplaceBranchInfo(branch3, true, true, false, brOps2); - catalog.createOrReplaceBranch(dbName, tblName, info6); + catalog.createOrReplaceBranch(dorisTable, info6); assertSnapshotRef( icebergTable.refs().get(branch3), snapshots.get(4).snapshotId(), true, 1L, 2, 3L); // update branch1 - catalog.createOrReplaceBranch(dbName, tblName, info2); + catalog.createOrReplaceBranch(dorisTable, info2); assertSnapshotRef( icebergTable.refs().get(branch1), icebergTable.currentSnapshot().snapshotId(), @@ -369,23 +378,23 @@ public void testDropBranchAndTag() throws IOException, UserException { addSomeDataIntoIcebergTable(); CreateOrReplaceTagInfo tagInfo = new CreateOrReplaceTagInfo(tag1, true, false, false, TagOptions.EMPTY); - catalog.createOrReplaceTag(dbName, tblName, tagInfo); + catalog.createOrReplaceTag(dorisTable, tagInfo); // create a new branch: branch1 CreateOrReplaceBranchInfo branchInfo = new CreateOrReplaceBranchInfo(branch1, true, false, false, BranchOptions.EMPTY); - catalog.createOrReplaceBranch(dbName, tblName, branchInfo); + catalog.createOrReplaceBranch(dorisTable, branchInfo); // create a new tag: tag2 addSomeDataIntoIcebergTable(); CreateOrReplaceTagInfo tagInfo2 = new CreateOrReplaceTagInfo(tag2, true, false, false, TagOptions.EMPTY); - catalog.createOrReplaceTag(dbName, tblName, tagInfo2); + catalog.createOrReplaceTag(dorisTable, tagInfo2); // create a new branch: branch2 CreateOrReplaceBranchInfo branchInfo2 = new CreateOrReplaceBranchInfo(branch2, true, false, false, BranchOptions.EMPTY); - catalog.createOrReplaceBranch(dbName, tblName, branchInfo2); + catalog.createOrReplaceBranch(dorisTable, branchInfo2); Assertions.assertEquals(5, icebergTable.refs().size()); @@ -405,53 +414,53 @@ public void testDropBranchAndTag() throws IOException, UserException { DropBranchInfo dropBranchInfoWithTag1 = new DropBranchInfo(tag1, false); DropBranchInfo dropBranchInfoIfExistsWithTag1 = new DropBranchInfo(tag1, true); Assertions.assertThrows(RuntimeException.class, - () -> catalog.dropBranch(dbName, tblName, dropBranchInfoWithTag1)); + () -> catalog.dropBranch(dorisTable, dropBranchInfoWithTag1)); Assertions.assertThrows(RuntimeException.class, - () -> catalog.dropBranch(dbName, tblName, dropBranchInfoIfExistsWithTag1)); + () -> catalog.dropBranch(dorisTable, dropBranchInfoIfExistsWithTag1)); // drop branch with tag interface, will fail DropTagInfo dropTagInfoWithBranch1 = new DropTagInfo(branch1, false); DropTagInfo dropTagInfoWithBranchIfExists1 = new DropTagInfo(branch1, true); Assertions.assertThrows(RuntimeException.class, - () -> catalog.dropTag(dbName, tblName, dropTagInfoWithBranch1)); + () -> catalog.dropTag(dorisTable, dropTagInfoWithBranch1)); Assertions.assertThrows(RuntimeException.class, - () -> catalog.dropTag(dbName, tblName, dropTagInfoWithBranchIfExists1)); + () -> catalog.dropTag(dorisTable, dropTagInfoWithBranchIfExists1)); // drop not exists tag DropTagInfo dropTagInfoWithNotExistsTag1 = new DropTagInfo(tagNotExists, true); DropTagInfo dropTagInfoWithNotExistsTag2 = new DropTagInfo(tagNotExists, false); DropTagInfo dropTagInfoWithNotExistsBranch1 = new DropTagInfo(branchNotExists, true); DropTagInfo dropTagInfoWithNotExistsBranch2 = new DropTagInfo(branchNotExists, false); - catalog.dropTag(dbName, tblName, dropTagInfoWithNotExistsTag1); + catalog.dropTag(dorisTable, dropTagInfoWithNotExistsTag1); Assertions.assertThrows(RuntimeException.class, - () -> catalog.dropTag(dbName, tblName, dropTagInfoWithNotExistsTag2)); - catalog.dropTag(dbName, tblName, dropTagInfoWithNotExistsBranch1); + () -> catalog.dropTag(dorisTable, dropTagInfoWithNotExistsTag2)); + catalog.dropTag(dorisTable, dropTagInfoWithNotExistsBranch1); Assertions.assertThrows(RuntimeException.class, - () -> catalog.dropTag(dbName, tblName, dropTagInfoWithNotExistsBranch2)); + () -> catalog.dropTag(dorisTable, dropTagInfoWithNotExistsBranch2)); // drop not exists branch DropBranchInfo dropBranchInfoWithNotExistsTag1 = new DropBranchInfo(tagNotExists, true); DropBranchInfo dropBranchInfoWithNotExistsTag2 = new DropBranchInfo(tagNotExists, false); DropBranchInfo dropBranchInfoIfExistsWithBranch1 = new DropBranchInfo(branchNotExists, true); DropBranchInfo dropBranchInfoIfExistsWithBranch2 = new DropBranchInfo(branchNotExists, false); - catalog.dropBranch(dbName, tblName, dropBranchInfoWithNotExistsTag1); + catalog.dropBranch(dorisTable, dropBranchInfoWithNotExistsTag1); Assertions.assertThrows(RuntimeException.class, - () -> catalog.dropBranch(dbName, tblName, dropBranchInfoWithNotExistsTag2)); - catalog.dropBranch(dbName, tblName, dropBranchInfoIfExistsWithBranch1); + () -> catalog.dropBranch(dorisTable, dropBranchInfoWithNotExistsTag2)); + catalog.dropBranch(dorisTable, dropBranchInfoIfExistsWithBranch1); Assertions.assertThrows(RuntimeException.class, - () -> catalog.dropBranch(dbName, tblName, dropBranchInfoIfExistsWithBranch2)); + () -> catalog.dropBranch(dorisTable, dropBranchInfoIfExistsWithBranch2)); // drop branch1 and branch2 DropBranchInfo dropBranchInfoWithBranch1 = new DropBranchInfo(branch1, false); DropBranchInfo dropBranchInfoWithBranch2 = new DropBranchInfo(branch2, true); - catalog.dropBranch(dbName, tblName, dropBranchInfoWithBranch1); - catalog.dropBranch(dbName, tblName, dropBranchInfoWithBranch2); + catalog.dropBranch(dorisTable, dropBranchInfoWithBranch1); + catalog.dropBranch(dorisTable, dropBranchInfoWithBranch2); // drop tag1 and tag2 DropTagInfo dropTagInfoWithTag1 = new DropTagInfo(tag1, false); DropTagInfo dropTagInfoWithTag2 = new DropTagInfo(tag2, true); - catalog.dropTag(dbName, tblName, dropTagInfoWithTag1); - catalog.dropTag(dbName, tblName, dropTagInfoWithTag2); + catalog.dropTag(dorisTable, dropTagInfoWithTag1); + catalog.dropTag(dorisTable, dropTagInfoWithTag2); Assertions.assertEquals(1, icebergTable.refs().size()); } diff --git a/fe/fe-core/src/test/java/org/apache/doris/datasource/iceberg/IcebergTransactionTest.java b/fe/fe-core/src/test/java/org/apache/doris/datasource/iceberg/IcebergTransactionTest.java index 79f7d5b5ad6555..4125738743b766 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/datasource/iceberg/IcebergTransactionTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/datasource/iceberg/IcebergTransactionTest.java @@ -18,23 +18,20 @@ package org.apache.doris.datasource.iceberg; import org.apache.doris.common.UserException; -import org.apache.doris.common.info.SimpleTableInfo; +import org.apache.doris.common.security.authentication.PreExecutionAuthenticator; import org.apache.doris.common.util.SerializationUtils; -import org.apache.doris.datasource.ExternalCatalog; +import org.apache.doris.datasource.ExternalTable; +import org.apache.doris.datasource.NameMapping; import org.apache.doris.nereids.trees.plans.commands.insert.IcebergInsertCommandContext; import org.apache.doris.thrift.TFileContent; import org.apache.doris.thrift.TIcebergCommitData; -import com.google.common.collect.Maps; -import mockit.Mock; -import mockit.MockUp; import org.apache.hadoop.conf.Configuration; import org.apache.iceberg.CatalogProperties; import org.apache.iceberg.FileScanTask; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; import org.apache.iceberg.Table; -import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.expressions.Expression; @@ -49,6 +46,9 @@ import org.junit.Assert; import org.junit.Before; import org.junit.Test; +import org.mockito.ArgumentMatchers; +import org.mockito.MockedStatic; +import org.mockito.Mockito; import java.io.IOException; import java.io.Serializable; @@ -68,10 +68,9 @@ public class IcebergTransactionTest { private static String tbWithPartition = "tbWithPartition"; private static String tbWithoutPartition = "tbWithoutPartition"; - private IcebergExternalCatalog externalCatalog; + private IcebergExternalCatalog spyExternalCatalog; private IcebergMetadataOps ops; - @Before public void init() throws IOException { createCatalog(); @@ -86,18 +85,13 @@ private void createCatalog() throws IOException { props.put(CatalogProperties.WAREHOUSE_LOCATION, warehouse); hadoopCatalog.setConf(new Configuration()); hadoopCatalog.initialize("df", props); - this.externalCatalog = new IcebergHMSExternalCatalog(1L, "iceberg", "", Maps.newHashMap(), ""); - externalCatalog.initLocalObjectsImpl(); - new MockUp() { - @Mock - public Catalog getCatalog() { - return hadoopCatalog; - } - }; - ops = new IcebergMetadataOps(externalCatalog, hadoopCatalog); + this.spyExternalCatalog = Mockito.mock(IcebergExternalCatalog.class); + Mockito.when(spyExternalCatalog.getCatalog()).thenReturn(hadoopCatalog); + Mockito.when(spyExternalCatalog.getPreExecutionAuthenticator()).thenReturn(new PreExecutionAuthenticator()); + ops = new IcebergMetadataOps(spyExternalCatalog, hadoopCatalog); } - private void createTable() throws IOException { + private void createTable() { HadoopCatalog icebergCatalog = (HadoopCatalog) ops.getCatalog(); icebergCatalog.createNamespace(Namespace.of(dbName)); Schema schema = new Schema( @@ -187,19 +181,20 @@ public void testPartitionedTable() throws UserException { Table table = ops.getCatalog().loadTable(TableIdentifier.of(dbName, tbWithPartition)); - new MockUp() { - @Mock - public Table getRemoteTable(ExternalCatalog catalog, SimpleTableInfo tableInfo) { - return table; - } - }; - - IcebergTransaction txn = getTxn(); - txn.updateIcebergCommitData(ctdList); - SimpleTableInfo tableInfo = new SimpleTableInfo(dbName, tbWithPartition); - txn.beginInsert(tableInfo); - txn.finishInsert(tableInfo, Optional.empty()); - txn.commit(); + IcebergExternalTable icebergExternalTable = Mockito.mock(IcebergExternalTable.class); + Mockito.when(icebergExternalTable.getCatalog()).thenReturn(spyExternalCatalog); + Mockito.when(icebergExternalTable.getDbName()).thenReturn(dbName); + Mockito.when(icebergExternalTable.getName()).thenReturn(tbWithPartition); + + try (MockedStatic mockedStatic = Mockito.mockStatic(IcebergUtils.class)) { + mockedStatic.when(() -> IcebergUtils.getIcebergTable(ArgumentMatchers.any(ExternalTable.class))) + .thenReturn(table); + IcebergTransaction txn = getTxn(); + txn.updateIcebergCommitData(ctdList); + txn.beginInsert(icebergExternalTable); + txn.finishInsert(NameMapping.createForTest(dbName, tbWithPartition), Optional.empty()); + txn.commit(); + } checkSnapshotAddProperties(table.currentSnapshot().summary(), "6", "2", "6"); checkPushDownByPartitionForTs(table, "ts1"); @@ -298,19 +293,21 @@ public void testUnPartitionedTable() throws UserException { ctdList.add(ctd2); Table table = ops.getCatalog().loadTable(TableIdentifier.of(dbName, tbWithoutPartition)); - new MockUp() { - @Mock - public Table getRemoteTable(ExternalCatalog catalog, SimpleTableInfo tableInfo) { - return table; - } - }; - - IcebergTransaction txn = getTxn(); - txn.updateIcebergCommitData(ctdList); - SimpleTableInfo tableInfo = new SimpleTableInfo(dbName, tbWithPartition); - txn.beginInsert(tableInfo); - txn.finishInsert(tableInfo, Optional.empty()); - txn.commit(); + IcebergExternalTable icebergExternalTable = Mockito.mock(IcebergExternalTable.class); + Mockito.when(icebergExternalTable.getCatalog()).thenReturn(spyExternalCatalog); + Mockito.when(icebergExternalTable.getDbName()).thenReturn(dbName); + Mockito.when(icebergExternalTable.getName()).thenReturn(tbWithoutPartition); + + try (MockedStatic mockedStatic = Mockito.mockStatic(IcebergUtils.class)) { + mockedStatic.when(() -> IcebergUtils.getIcebergTable(ArgumentMatchers.any(ExternalTable.class))) + .thenReturn(table); + + IcebergTransaction txn = getTxn(); + txn.updateIcebergCommitData(ctdList); + txn.beginInsert(icebergExternalTable); + txn.finishInsert(NameMapping.createForTest(dbName, tbWithPartition), Optional.empty()); + txn.commit(); + } checkSnapshotAddProperties(table.currentSnapshot().summary(), "6", "2", "6"); } @@ -408,21 +405,22 @@ public void testUnPartitionedTableOverwriteWithData() throws UserException { ctdList.add(ctd3); Table table = ops.getCatalog().loadTable(TableIdentifier.of(dbName, tbWithoutPartition)); - new MockUp() { - @Mock - public Table getRemoteTable(ExternalCatalog catalog, SimpleTableInfo tableInfo) { - return table; - } - }; - - IcebergTransaction txn = getTxn(); - txn.updateIcebergCommitData(ctdList); - SimpleTableInfo tableInfo = new SimpleTableInfo(dbName, tbWithPartition); - txn.beginInsert(tableInfo); - IcebergInsertCommandContext ctx = new IcebergInsertCommandContext(); - ctx.setOverwrite(true); - txn.finishInsert(tableInfo, Optional.of(ctx)); - txn.commit(); + IcebergExternalTable icebergExternalTable = Mockito.mock(IcebergExternalTable.class); + Mockito.when(icebergExternalTable.getCatalog()).thenReturn(spyExternalCatalog); + Mockito.when(icebergExternalTable.getDbName()).thenReturn(dbName); + Mockito.when(icebergExternalTable.getName()).thenReturn(tbWithoutPartition); + try (MockedStatic mockedStatic = Mockito.mockStatic(IcebergUtils.class)) { + mockedStatic.when(() -> IcebergUtils.getIcebergTable(ArgumentMatchers.any(ExternalTable.class))) + .thenReturn(table); + + IcebergTransaction txn = getTxn(); + txn.updateIcebergCommitData(ctdList); + txn.beginInsert(icebergExternalTable); + IcebergInsertCommandContext ctx = new IcebergInsertCommandContext(); + ctx.setOverwrite(true); + txn.finishInsert(NameMapping.createForTest(dbName, tbWithPartition), Optional.of(ctx)); + txn.commit(); + } checkSnapshotTotalProperties(table.currentSnapshot().summary(), "24", "3", "24"); } @@ -433,20 +431,21 @@ public void testUnpartitionedTableOverwriteWithoutData() throws UserException { testUnPartitionedTableOverwriteWithData(); Table table = ops.getCatalog().loadTable(TableIdentifier.of(dbName, tbWithoutPartition)); - new MockUp() { - @Mock - public Table getRemoteTable(ExternalCatalog catalog, SimpleTableInfo tableInfo) { - return table; - } - }; - - IcebergTransaction txn = getTxn(); - SimpleTableInfo tableInfo = new SimpleTableInfo(dbName, tbWithPartition); - txn.beginInsert(tableInfo); - IcebergInsertCommandContext ctx = new IcebergInsertCommandContext(); - ctx.setOverwrite(true); - txn.finishInsert(tableInfo, Optional.of(ctx)); - txn.commit(); + IcebergExternalTable icebergExternalTable = Mockito.mock(IcebergExternalTable.class); + Mockito.when(icebergExternalTable.getCatalog()).thenReturn(spyExternalCatalog); + Mockito.when(icebergExternalTable.getDbName()).thenReturn(dbName); + Mockito.when(icebergExternalTable.getName()).thenReturn(tbWithoutPartition); + try (MockedStatic mockedStatic = Mockito.mockStatic(IcebergUtils.class)) { + mockedStatic.when(() -> IcebergUtils.getIcebergTable(ArgumentMatchers.any(ExternalTable.class))) + .thenReturn(table); + + IcebergTransaction txn = getTxn(); + txn.beginInsert(icebergExternalTable); + IcebergInsertCommandContext ctx = new IcebergInsertCommandContext(); + ctx.setOverwrite(true); + txn.finishInsert(NameMapping.createForTest(dbName, tbWithPartition), Optional.of(ctx)); + txn.commit(); + } checkSnapshotTotalProperties(table.currentSnapshot().summary(), "0", "0", "0"); } diff --git a/fe/fe-core/src/test/java/org/apache/doris/datasource/iceberg/dlf/client/IcebergDLFExternalCatalogTest.java b/fe/fe-core/src/test/java/org/apache/doris/datasource/iceberg/dlf/client/IcebergDLFExternalCatalogTest.java index 7d05fcdef5898f..ae375b6e8a9464 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/datasource/iceberg/dlf/client/IcebergDLFExternalCatalogTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/datasource/iceberg/dlf/client/IcebergDLFExternalCatalogTest.java @@ -17,12 +17,10 @@ package org.apache.doris.datasource.iceberg.dlf.client; -import org.apache.doris.analysis.CreateDbStmt; import org.apache.doris.datasource.iceberg.IcebergDLFExternalCatalog; import org.apache.doris.nereids.exceptions.NotSupportedException; -import org.apache.doris.nereids.trees.plans.commands.CreateDatabaseCommand; -import org.apache.doris.nereids.trees.plans.commands.TruncateTableCommand; +import com.google.common.collect.Maps; import org.apache.hadoop.conf.Configuration; import org.junit.Assert; import org.junit.Test; @@ -42,19 +40,16 @@ public void testDatabaseList() { // This cache should belong to the catalog level, // so the object addresses of clients in different pools must be different Assert.assertNotSame(dlfClientPool1, dlfClientPool2); - } @Test public void testNotSupportOperation() { HashMap props = new HashMap<>(); IcebergDLFExternalCatalog catalog = new IcebergDLFExternalCatalog(1, "test", "test", props, "test"); - Assert.assertThrows(NotSupportedException.class, () -> catalog.createDb((CreateDbStmt) null)); - Assert.assertThrows(NotSupportedException.class, () -> catalog.createDb((CreateDatabaseCommand) null)); + Assert.assertThrows(NotSupportedException.class, () -> catalog.createDb("db1", true, Maps.newHashMap())); Assert.assertThrows(NotSupportedException.class, () -> catalog.dropDb("", true, true)); Assert.assertThrows(NotSupportedException.class, () -> catalog.createTable(null)); - Assert.assertThrows(NotSupportedException.class, () -> catalog.dropTable(null)); Assert.assertThrows(NotSupportedException.class, () -> catalog.dropTable("", "", true, true, true, true)); - Assert.assertThrows(NotSupportedException.class, () -> catalog.truncateTable((TruncateTableCommand) null)); + Assert.assertThrows(NotSupportedException.class, () -> catalog.truncateTable("", "", null, true, "")); } } diff --git a/fe/fe-core/src/test/java/org/apache/doris/datasource/paimon/PaimonExternalCatalogTest.java b/fe/fe-core/src/test/java/org/apache/doris/datasource/paimon/PaimonExternalCatalogTest.java deleted file mode 100644 index da66b3c5ba9a24..00000000000000 --- a/fe/fe-core/src/test/java/org/apache/doris/datasource/paimon/PaimonExternalCatalogTest.java +++ /dev/null @@ -1,43 +0,0 @@ -// 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.junit.Assert; -import org.junit.Test; - -import java.util.HashMap; - - -public class PaimonExternalCatalogTest { - - @Test - public void testGetPaimonTable() { - - HashMap props = new HashMap<>(); - props.put("warehouse", "not_exist"); - PaimonExternalCatalog catalog = new PaimonFileExternalCatalog(1, "name", "resource", props, "comment"); - catalog.setInitializedForTest(true); - - try { - catalog.getPaimonTable("dbName", "tblName"); - Assert.fail(); - } catch (Exception e) { - Assert.assertTrue(e.getMessage().contains("Failed to get Paimon table")); - } - } -} diff --git a/fe/fe-core/src/test/java/org/apache/doris/external/hms/HmsCatalogTest.java b/fe/fe-core/src/test/java/org/apache/doris/external/hms/HmsCatalogTest.java index 7d475c5bfee952..1398d095bb535c 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/external/hms/HmsCatalogTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/external/hms/HmsCatalogTest.java @@ -18,7 +18,6 @@ package org.apache.doris.external.hms; import org.apache.doris.analysis.CreateCatalogStmt; -import org.apache.doris.analysis.CreateDbStmt; import org.apache.doris.analysis.CreateTableStmt; import org.apache.doris.catalog.Column; import org.apache.doris.catalog.Env; @@ -40,6 +39,7 @@ import org.apache.doris.qe.SessionVariable; import com.google.common.collect.Lists; +import com.google.common.collect.Maps; import mockit.Expectations; import mockit.Mocked; import org.junit.Assert; @@ -81,8 +81,7 @@ protected void runBeforeAll() throws Exception { mgr.createCatalog(hmsCatalog); // create inner db and tbl for test - CreateDbStmt createDbStmt = (CreateDbStmt) parseAndAnalyzeStmt("create database test", connectContext); - mgr.getInternalCatalog().createDb(createDbStmt); + mgr.getInternalCatalog().createDb("test", false, Maps.newHashMap()); CreateTableStmt createTableStmt = (CreateTableStmt) parseAndAnalyzeStmt("create table test.tbl1(\n" + "k1 int comment 'test column k1', " @@ -142,7 +141,7 @@ private void createDbAndTableForHmsCatalog(HMSExternalCatalog hmsCatalog) { result = TableIf.TableType.HMS_EXTERNAL_TABLE; // mock initSchemaAndUpdateTime and do nothing - tbl.initSchemaAndUpdateTime(new ExternalSchemaCache.SchemaCacheKey("hms_db", "hms_tbl")); + tbl.initSchemaAndUpdateTime(new ExternalSchemaCache.SchemaCacheKey(tbl.getOrBuildNameMapping())); minTimes = 0; tbl.getDatabase(); diff --git a/fe/fe-core/src/test/java/org/apache/doris/planner/ListPartitionPrunerV2Test.java b/fe/fe-core/src/test/java/org/apache/doris/planner/ListPartitionPrunerV2Test.java index 5af7b831e98c86..c9c3f5361780ed 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/planner/ListPartitionPrunerV2Test.java +++ b/fe/fe-core/src/test/java/org/apache/doris/planner/ListPartitionPrunerV2Test.java @@ -25,9 +25,11 @@ import org.apache.doris.catalog.Type; import org.apache.doris.common.AnalysisException; import org.apache.doris.common.ThreadPoolManager; +import org.apache.doris.datasource.NameMapping; import org.apache.doris.datasource.hive.HMSCachedClient; import org.apache.doris.datasource.hive.HMSExternalCatalog; import org.apache.doris.datasource.hive.HiveMetaStoreCache; +import org.apache.doris.datasource.hive.HiveMetaStoreCache.PartitionValueCacheKey; import org.apache.doris.datasource.hive.ThriftHMSCachedClient; import com.google.common.collect.Lists; @@ -101,7 +103,8 @@ public List listPartitionNames(String dbName, String tblName) { // the original partition of the table (in mock) will be loaded here String dbName = "db"; String tblName = "tb"; - HiveMetaStoreCache.HivePartitionValues partitionValues = cache.getPartitionValues(dbName, tblName, types); + PartitionValueCacheKey key = new PartitionValueCacheKey(NameMapping.createForTest("db", "tb"), types); + HiveMetaStoreCache.HivePartitionValues partitionValues = cache.getPartitionValues(key); Assert.assertEquals(1, partitionValues.getIdToPartitionItem().size()); Assert.assertTrue(partitionValues.getIdToPartitionItem().containsKey(8882801933302843777L)); List items = partitionValues.getIdToPartitionItem().get(8882801933302843777L).getItems(); @@ -113,8 +116,9 @@ public List listPartitionNames(String dbName, String tblName) { // test add cache ArrayList values = new ArrayList<>(); values.add("c1=5.678000"); - cache.addPartitionsCache(dbName, tblName, values, types); - HiveMetaStoreCache.HivePartitionValues partitionValues2 = cache.getPartitionValues(dbName, tblName, types); + cache.addPartitionsCache(NameMapping.createForTest(dbName, tblName), values, types); + HiveMetaStoreCache.HivePartitionValues partitionValues2 = cache.getPartitionValues( + new PartitionValueCacheKey(NameMapping.createForTest(dbName, tblName), types)); Assert.assertEquals(2, partitionValues2.getIdToPartitionItem().size()); Assert.assertTrue(partitionValues2.getIdToPartitionItem().containsKey(7070400225537799947L)); List items2 = partitionValues2.getIdToPartitionItem().get(7070400225537799947L).getItems(); @@ -125,8 +129,9 @@ public List listPartitionNames(String dbName, String tblName) { // test refresh table // simulates the manually added partition table being deleted, leaving only one original partition in mock - cache.invalidateTableCache(dbName, tblName); - HiveMetaStoreCache.HivePartitionValues partitionValues3 = cache.getPartitionValues(dbName, tblName, types); + cache.invalidateTableCache(NameMapping.createForTest(dbName, tblName)); + HiveMetaStoreCache.HivePartitionValues partitionValues3 = cache.getPartitionValues( + new PartitionValueCacheKey(NameMapping.createForTest(dbName, tblName), types)); Assert.assertEquals(1, partitionValues3.getIdToPartitionItem().size()); Assert.assertTrue(partitionValues3.getIdToPartitionItem().containsKey(8882801933302843777L)); List items3 = partitionValues3.getIdToPartitionItem().get(8882801933302843777L).getItems(); diff --git a/fe/fe-core/src/test/java/org/apache/doris/qe/HmsQueryCacheTest.java b/fe/fe-core/src/test/java/org/apache/doris/qe/HmsQueryCacheTest.java index 872bca03f34f2c..6c02e769f5a53d 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/qe/HmsQueryCacheTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/qe/HmsQueryCacheTest.java @@ -18,7 +18,6 @@ package org.apache.doris.qe; import org.apache.doris.analysis.CreateCatalogStmt; -import org.apache.doris.analysis.CreateDbStmt; import org.apache.doris.analysis.CreateTableStmt; import org.apache.doris.analysis.StatementBase; import org.apache.doris.analysis.TupleDescriptor; @@ -47,10 +46,10 @@ import org.apache.doris.qe.cache.SqlCache; import com.google.common.collect.Lists; -import mockit.Expectations; -import mockit.Mocked; +import com.google.common.collect.Maps; import org.junit.Assert; import org.junit.jupiter.api.Test; +import org.mockito.Mockito; import java.util.Arrays; import java.util.List; @@ -63,21 +62,13 @@ public class HmsQueryCacheTest extends AnalyzeCheckTestBase { private CatalogMgr mgr; private OlapScanNode olapScanNode; - @Mocked private HMSExternalTable tbl; - @Mocked private HMSExternalTable tbl2; - @Mocked private HMSExternalTable view1; - @Mocked private HMSExternalTable view2; - @Mocked private HiveScanNode hiveScanNode1; - @Mocked private HiveScanNode hiveScanNode2; - @Mocked private HiveScanNode hiveScanNode3; - @Mocked private HiveScanNode hiveScanNode4; @Override @@ -98,8 +89,7 @@ protected void runBeforeAll() throws Exception { mgr.createCatalog(hmsCatalogStmt); // create inner db and tbl for test - CreateDbStmt createDbStmt = (CreateDbStmt) parseAndAnalyzeStmt("create database test", connectContext); - mgr.getInternalCatalog().createDb(createDbStmt); + mgr.getInternalCatalog().createDb("test", false, Maps.newHashMap()); CreateTableStmt createTableStmt = (CreateTableStmt) parseAndAnalyzeStmt("create table test.tbl1(\n" + "k1 int comment 'test column k1', " @@ -109,239 +99,119 @@ protected void runBeforeAll() throws Exception { mgr.getInternalCatalog().createTable(createTableStmt); } + private void setField(Object target, String fieldName, Object value) { + // try { + // Field field = target.getClass().getDeclaredField(fieldName); + // field.setAccessible(true); + // field.set(target, value); + // } catch (Exception e) { + // throw new RuntimeException(e); + // } + + Deencapsulation.setField(target, fieldName, value); + } + private void init(HMSExternalCatalog hmsCatalog) { - Deencapsulation.setField(hmsCatalog, "initialized", true); - Deencapsulation.setField(hmsCatalog, "objectCreated", true); - Deencapsulation.setField(hmsCatalog, "useMetaCache", Optional.of(false)); + // Create mock objects + tbl = Mockito.mock(HMSExternalTable.class); + tbl2 = Mockito.mock(HMSExternalTable.class); + view1 = Mockito.mock(HMSExternalTable.class); + view2 = Mockito.mock(HMSExternalTable.class); + hiveScanNode1 = Mockito.mock(HiveScanNode.class); + hiveScanNode2 = Mockito.mock(HiveScanNode.class); + hiveScanNode3 = Mockito.mock(HiveScanNode.class); + hiveScanNode4 = Mockito.mock(HiveScanNode.class); + + setField(hmsCatalog, "initialized", true); + setField(hmsCatalog, "objectCreated", true); + setField(hmsCatalog, "useMetaCache", Optional.of(false)); List schema = Lists.newArrayList(); schema.add(new Column("k1", PrimitiveType.INT)); HMSExternalDatabase db = new HMSExternalDatabase(hmsCatalog, 10000, "hms_db", "hms_db"); - Deencapsulation.setField(db, "initialized", true); - - Deencapsulation.setField(tbl, "objectCreated", true); - Deencapsulation.setField(tbl, "schemaUpdateTime", NOW); - Deencapsulation.setField(tbl, "eventUpdateTime", 0); - Deencapsulation.setField(tbl, "catalog", hmsCatalog); - Deencapsulation.setField(tbl, "dbName", "hms_db"); - Deencapsulation.setField(tbl, "name", "hms_tbl"); - Deencapsulation.setField(tbl, "dlaTable", new HiveDlaTable(tbl)); - Deencapsulation.setField(tbl, "dlaType", DLAType.HIVE); - new Expectations(tbl) { - { - tbl.getId(); - minTimes = 0; - result = 10001; - - tbl.getName(); - minTimes = 0; - result = "hms_tbl"; - - tbl.getDbName(); - minTimes = 0; - result = "hms_db"; - - tbl.getFullSchema(); - minTimes = 0; - result = schema; - - tbl.isSupportedHmsTable(); - minTimes = 0; - result = true; - - tbl.isView(); - minTimes = 0; - result = false; - - tbl.getType(); - minTimes = 0; - result = TableIf.TableType.HMS_EXTERNAL_TABLE; - - tbl.getDlaType(); - minTimes = 0; - result = DLAType.HIVE; - - // mock initSchemaAndUpdateTime and do nothing - tbl.initSchemaAndUpdateTime(new ExternalSchemaCache.SchemaCacheKey("hms_db", "hms_tbl")); - minTimes = 0; - - tbl.getDatabase(); - minTimes = 0; - result = db; - } - }; - - Deencapsulation.setField(tbl2, "objectCreated", true); - Deencapsulation.setField(tbl2, "schemaUpdateTime", NOW); - Deencapsulation.setField(tbl2, "eventUpdateTime", 0); - Deencapsulation.setField(tbl2, "catalog", hmsCatalog); - Deencapsulation.setField(tbl2, "dbName", "hms_db"); - Deencapsulation.setField(tbl2, "name", "hms_tbl2"); - Deencapsulation.setField(tbl2, "dlaTable", new HiveDlaTable(tbl2)); - Deencapsulation.setField(tbl, "dlaType", DLAType.HIVE); - new Expectations(tbl2) { - { - tbl2.getId(); - minTimes = 0; - result = 10004; - - tbl2.getName(); - minTimes = 0; - result = "hms_tbl2"; - - tbl2.getDbName(); - minTimes = 0; - result = "hms_db"; - - tbl2.getFullSchema(); - minTimes = 0; - result = schema; - - tbl2.isSupportedHmsTable(); - minTimes = 0; - result = true; - - tbl2.isView(); - minTimes = 0; - result = false; - - tbl2.getType(); - minTimes = 0; - result = TableIf.TableType.HMS_EXTERNAL_TABLE; - - tbl2.getDlaType(); - minTimes = 0; - result = DLAType.HIVE; - - // mock initSchemaAndUpdateTime and do nothing - tbl2.initSchemaAndUpdateTime(new ExternalSchemaCache.SchemaCacheKey("hms_db", "hms_tbl2")); - minTimes = 0; - - tbl2.getDatabase(); - minTimes = 0; - result = db; - - tbl2.getSupportedSysTables(); - minTimes = 0; - result = SupportedSysTables.HIVE_SUPPORTED_SYS_TABLES; - } - }; - - Deencapsulation.setField(view1, "objectCreated", true); - - new Expectations(view1) { - { - view1.getId(); - minTimes = 0; - result = 10002; - - view1.getName(); - minTimes = 0; - result = "hms_view1"; - - view1.getDbName(); - minTimes = 0; - result = "hms_db"; - - view1.isView(); - minTimes = 0; - result = true; - - view1.getCatalog(); - minTimes = 0; - result = hmsCatalog; - - view1.getType(); - minTimes = 0; - result = TableIf.TableType.HMS_EXTERNAL_TABLE; - - view1.getFullSchema(); - minTimes = 0; - result = schema; - - view1.getViewText(); - minTimes = 0; - result = "SELECT * FROM hms_db.hms_tbl"; - - view1.isSupportedHmsTable(); - minTimes = 0; - result = true; - - view1.getDlaType(); - minTimes = 0; - result = DLAType.HIVE; - - view1.getUpdateTime(); - minTimes = 0; - result = NOW; - - view1.getDatabase(); - minTimes = 0; - result = db; - - view1.getSupportedSysTables(); - minTimes = 0; - result = SupportedSysTables.HIVE_SUPPORTED_SYS_TABLES; - } - }; - - Deencapsulation.setField(view2, "objectCreated", true); - new Expectations(view2) { - { - view2.getId(); - minTimes = 0; - result = 10003; - - view2.getName(); - minTimes = 0; - result = "hms_view2"; - - view2.getDbName(); - minTimes = 0; - result = "hms_db"; - - view2.isView(); - minTimes = 0; - result = true; - - view2.getCatalog(); - minTimes = 0; - result = hmsCatalog; - - view2.getType(); - minTimes = 0; - result = TableIf.TableType.HMS_EXTERNAL_TABLE; - - view2.getFullSchema(); - minTimes = 0; - result = schema; - - view2.getViewText(); - minTimes = 0; - result = "SELECT * FROM hms_db.hms_view1"; - - view2.isSupportedHmsTable(); - minTimes = 0; - result = true; - - view2.getDlaType(); - minTimes = 0; - result = DLAType.HIVE; - - view2.getUpdateTime(); - minTimes = 0; - result = NOW; - - view2.getDatabase(); - minTimes = 0; - result = db; - - view2.getSupportedSysTables(); - minTimes = 0; - result = SupportedSysTables.HIVE_SUPPORTED_SYS_TABLES; - } - }; + setField(db, "initialized", true); + + setField(tbl, "objectCreated", true); + setField(tbl, "schemaUpdateTime", NOW); + setField(tbl, "eventUpdateTime", 0); + setField(tbl, "catalog", hmsCatalog); + setField(tbl, "dbName", "hms_db"); + setField(tbl, "name", "hms_tbl"); + setField(tbl, "dlaTable", new HiveDlaTable(tbl)); + setField(tbl, "dlaType", DLAType.HIVE); + + Mockito.when(tbl.getId()).thenReturn(10001L); + Mockito.when(tbl.getName()).thenReturn("hms_tbl"); + Mockito.when(tbl.getDbName()).thenReturn("hms_db"); + Mockito.when(tbl.getFullSchema()).thenReturn(schema); + Mockito.when(tbl.isSupportedHmsTable()).thenReturn(true); + Mockito.when(tbl.isView()).thenReturn(false); + Mockito.when(tbl.getType()).thenReturn(TableIf.TableType.HMS_EXTERNAL_TABLE); + Mockito.when(tbl.getDlaType()).thenReturn(DLAType.HIVE); + Mockito.when(tbl.getDatabase()).thenReturn(db); + Mockito.when(tbl.getUpdateTime()).thenReturn(NOW); + // mock initSchemaAndUpdateTime and do nothing + Mockito.when(tbl.initSchema(Mockito.any(ExternalSchemaCache.SchemaCacheKey.class))) + .thenReturn(Optional.empty()); + + setField(tbl2, "objectCreated", true); + setField(tbl2, "schemaUpdateTime", NOW); + setField(tbl2, "eventUpdateTime", 0); + setField(tbl2, "catalog", hmsCatalog); + setField(tbl2, "dbName", "hms_db"); + setField(tbl2, "name", "hms_tbl2"); + setField(tbl2, "dlaTable", new HiveDlaTable(tbl2)); + setField(tbl2, "dlaType", DLAType.HIVE); + + Mockito.when(tbl2.getId()).thenReturn(10004L); + Mockito.when(tbl2.getName()).thenReturn("hms_tbl2"); + Mockito.when(tbl2.getDbName()).thenReturn("hms_db"); + Mockito.when(tbl2.getFullSchema()).thenReturn(schema); + Mockito.when(tbl2.isSupportedHmsTable()).thenReturn(true); + Mockito.when(tbl2.isView()).thenReturn(false); + Mockito.when(tbl2.getType()).thenReturn(TableIf.TableType.HMS_EXTERNAL_TABLE); + Mockito.when(tbl2.getDlaType()).thenReturn(DLAType.HIVE); + Mockito.when(tbl2.getDatabase()).thenReturn(db); + Mockito.when(tbl2.getSupportedSysTables()).thenReturn(SupportedSysTables.HIVE_SUPPORTED_SYS_TABLES); + Mockito.when(tbl2.getUpdateTime()).thenReturn(NOW); + Mockito.when(tbl2.getSchemaUpdateTime()).thenReturn(NOW); + // mock initSchemaAndUpdateTime and do nothing + Mockito.when(tbl2.initSchemaAndUpdateTime(Mockito.any(ExternalSchemaCache.SchemaCacheKey.class))) + .thenReturn(Optional.empty()); + Mockito.doNothing().when(tbl2).setEventUpdateTime(Mockito.anyLong()); + + setField(view1, "objectCreated", true); + + Mockito.when(view1.getId()).thenReturn(10002L); + Mockito.when(view1.getName()).thenReturn("hms_view1"); + Mockito.when(view1.getDbName()).thenReturn("hms_db"); + Mockito.when(view1.isView()).thenReturn(true); + Mockito.when(view1.getCatalog()).thenReturn(hmsCatalog); + Mockito.when(view1.getType()).thenReturn(TableIf.TableType.HMS_EXTERNAL_TABLE); + Mockito.when(view1.getFullSchema()).thenReturn(schema); + Mockito.when(view1.getViewText()).thenReturn("SELECT * FROM hms_db.hms_tbl"); + Mockito.when(view1.isSupportedHmsTable()).thenReturn(true); + Mockito.when(view1.getDlaType()).thenReturn(DLAType.HIVE); + Mockito.when(view1.getUpdateTime()).thenReturn(NOW); + Mockito.when(view1.getDatabase()).thenReturn(db); + Mockito.when(view1.getSupportedSysTables()).thenReturn(SupportedSysTables.HIVE_SUPPORTED_SYS_TABLES); + + setField(view2, "objectCreated", true); + + Mockito.when(view2.getId()).thenReturn(10003L); + Mockito.when(view2.getName()).thenReturn("hms_view2"); + Mockito.when(view2.getDbName()).thenReturn("hms_db"); + Mockito.when(view2.isView()).thenReturn(true); + Mockito.when(view2.getCatalog()).thenReturn(hmsCatalog); + Mockito.when(view2.getType()).thenReturn(TableIf.TableType.HMS_EXTERNAL_TABLE); + Mockito.when(view2.getFullSchema()).thenReturn(schema); + Mockito.when(view2.getViewText()).thenReturn("SELECT * FROM hms_db.hms_view1"); + Mockito.when(view2.isSupportedHmsTable()).thenReturn(true); + Mockito.when(view2.getDlaType()).thenReturn(DLAType.HIVE); + Mockito.when(view2.getUpdateTime()).thenReturn(NOW); + Mockito.when(view2.getDatabase()).thenReturn(db); + Mockito.when(view2.getSupportedSysTables()).thenReturn(SupportedSysTables.HIVE_SUPPORTED_SYS_TABLES); db.addTableForTest(tbl); db.addTableForTest(tbl2); @@ -349,37 +219,10 @@ private void init(HMSExternalCatalog hmsCatalog) { db.addTableForTest(view2); hmsCatalog.addDatabaseForTest(db); - new Expectations(hiveScanNode1) { - { - hiveScanNode1.getTargetTable(); - minTimes = 0; - result = tbl; - } - }; - - new Expectations(hiveScanNode2) { - { - hiveScanNode2.getTargetTable(); - minTimes = 0; - result = view1; - } - }; - - new Expectations(hiveScanNode3) { - { - hiveScanNode3.getTargetTable(); - minTimes = 0; - result = view2; - } - }; - - new Expectations(hiveScanNode4) { - { - hiveScanNode4.getTargetTable(); - minTimes = 0; - result = tbl2; - } - }; + Mockito.when(hiveScanNode1.getTargetTable()).thenReturn(tbl); + Mockito.when(hiveScanNode2.getTargetTable()).thenReturn(view1); + Mockito.when(hiveScanNode3.getTargetTable()).thenReturn(view2); + Mockito.when(hiveScanNode4.getTargetTable()).thenReturn(tbl2); TupleDescriptor desc = new TupleDescriptor(new TupleId(1)); desc.setTable(mgr.getInternalCatalog().getDbNullable("test").getTableNullable("tbl1")); @@ -405,7 +248,7 @@ public void testHitSqlCacheAfterPartitionUpdateTimeChanged() throws Exception { List scanNodes = Arrays.asList(hiveScanNode4); // invoke initSchemaAndUpdateTime first and init schemaUpdateTime - tbl2.initSchemaAndUpdateTime(new ExternalSchemaCache.SchemaCacheKey(tbl2.getDbName(), tbl2.getName())); + tbl2.initSchemaAndUpdateTime(new ExternalSchemaCache.SchemaCacheKey(tbl2.getOrBuildNameMapping())); CacheAnalyzer ca = new CacheAnalyzer(connectContext, parseStmt, scanNodes); ca.checkCacheMode(System.currentTimeMillis() + Config.cache_last_version_interval_second * 1000L * 2); @@ -422,7 +265,7 @@ public void testHitSqlCacheAfterPartitionUpdateTimeChanged() throws Exception { // do nothing } long later = System.currentTimeMillis(); - tbl2.setEventUpdateTime(later); + Mockito.when(tbl2.getUpdateTime()).thenReturn(later); // check cache mode again ca.checkCacheMode(System.currentTimeMillis() + Config.cache_last_version_interval_second * 1000L * 2); @@ -453,7 +296,7 @@ public void testHitSqlCacheByNereidsAfterPartitionUpdateTimeChanged() { List scanNodes = Arrays.asList(hiveScanNode4); // invoke initSchemaAndUpdateTime first and init schemaUpdateTime - tbl2.initSchemaAndUpdateTime(new ExternalSchemaCache.SchemaCacheKey(tbl2.getDbName(), tbl2.getName())); + tbl2.initSchemaAndUpdateTime(new ExternalSchemaCache.SchemaCacheKey(tbl2.getOrBuildNameMapping())); CacheAnalyzer ca = new CacheAnalyzer(connectContext, parseStmt, scanNodes); ca.checkCacheModeForNereids(System.currentTimeMillis() + Config.cache_last_version_interval_second * 1000L * 2); @@ -470,7 +313,7 @@ public void testHitSqlCacheByNereidsAfterPartitionUpdateTimeChanged() { // do nothing } long later = System.currentTimeMillis(); - tbl2.setEventUpdateTime(later); + Mockito.when(tbl2.getUpdateTime()).thenReturn(later); // check cache mode again ca.checkCacheModeForNereids(System.currentTimeMillis() + Config.cache_last_version_interval_second * 1000L * 2); diff --git a/fe/fe-core/src/test/java/org/apache/doris/statistics/StatisticsAutoCollectorTest.java b/fe/fe-core/src/test/java/org/apache/doris/statistics/StatisticsAutoCollectorTest.java index 305eb201dc0882..095a0df5138eac 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/statistics/StatisticsAutoCollectorTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/statistics/StatisticsAutoCollectorTest.java @@ -22,15 +22,19 @@ import org.apache.doris.catalog.Env; import org.apache.doris.catalog.OlapTable; import org.apache.doris.catalog.PrimitiveType; +import org.apache.doris.common.DdlException; import org.apache.doris.common.Pair; import org.apache.doris.datasource.ExternalTable; +import org.apache.doris.datasource.hive.HMSExternalCatalog; import org.apache.doris.datasource.hive.HMSExternalDatabase; import org.apache.doris.datasource.hive.HMSExternalTable; import org.apache.doris.datasource.hive.HMSExternalTable.DLAType; +import org.apache.doris.datasource.jdbc.JdbcExternalCatalog; import org.apache.doris.datasource.jdbc.JdbcExternalDatabase; import org.apache.doris.datasource.jdbc.JdbcExternalTable; import org.apache.doris.statistics.AnalysisInfo.AnalysisMethod; +import com.google.common.collect.Maps; import mockit.Mock; import mockit.MockUp; import org.junit.jupiter.api.Assertions; @@ -111,7 +115,7 @@ public AnalysisManager getAnalysisManager() { } @Test - public void testSupportAutoAnalyze() { + public void testSupportAutoAnalyze() throws DdlException { StatisticsAutoCollector collector = new StatisticsAutoCollector(); Assertions.assertFalse(collector.supportAutoAnalyze(null)); Column column1 = new Column("placeholder", PrimitiveType.INT); @@ -121,7 +125,9 @@ public void testSupportAutoAnalyze() { Assertions.assertTrue(collector.supportAutoAnalyze(table1)); JdbcExternalDatabase jdbcExternalDatabase = new JdbcExternalDatabase(null, 1L, "jdbcdb", "jdbcdb"); - ExternalTable externalTable = new JdbcExternalTable(1, "jdbctable", "jdbctable", null, jdbcExternalDatabase); + JdbcExternalCatalog jdbcCatalog = new JdbcExternalCatalog(0, "jdbc_ctl", null, Maps.newHashMap(), ""); + ExternalTable externalTable = new JdbcExternalTable(1, "jdbctable", "jdbctable", jdbcCatalog, + jdbcExternalDatabase); Assertions.assertFalse(collector.supportAutoAnalyze(externalTable)); new MockUp() { @@ -131,7 +137,9 @@ public DLAType getDlaType() { } }; HMSExternalDatabase hmsExternalDatabase = new HMSExternalDatabase(null, 1L, "hmsDb", "hmsDb"); - ExternalTable icebergExternalTable = new HMSExternalTable(1, "hmsTable", "hmsDb", null, hmsExternalDatabase); + HMSExternalCatalog hmsCatalog = new HMSExternalCatalog(0, "jdbc_ctl", null, Maps.newHashMap(), ""); + ExternalTable icebergExternalTable = new HMSExternalTable(1, "hmsTable", "hmsDb", hmsCatalog, + hmsExternalDatabase); Assertions.assertFalse(collector.supportAutoAnalyze(icebergExternalTable)); new MockUp() { @@ -140,7 +148,7 @@ public DLAType getDlaType() { return DLAType.HIVE; } }; - ExternalTable hiveExternalTable = new HMSExternalTable(1, "hmsTable", "hmsDb", null, hmsExternalDatabase); + ExternalTable hiveExternalTable = new HMSExternalTable(1, "hmsTable", "hmsDb", hmsCatalog, hmsExternalDatabase); Assertions.assertTrue(collector.supportAutoAnalyze(hiveExternalTable)); } diff --git a/fe/fe-core/src/test/java/org/apache/doris/statistics/util/StatisticsUtilTest.java b/fe/fe-core/src/test/java/org/apache/doris/statistics/util/StatisticsUtilTest.java index 15a0c65e49c63f..e6a9a27d7b7856 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/statistics/util/StatisticsUtilTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/statistics/util/StatisticsUtilTest.java @@ -36,8 +36,10 @@ import org.apache.doris.datasource.hive.HMSExternalDatabase; import org.apache.doris.datasource.hive.HMSExternalTable; import org.apache.doris.datasource.hive.HMSExternalTable.DLAType; +import org.apache.doris.datasource.iceberg.IcebergExternalCatalog; import org.apache.doris.datasource.iceberg.IcebergExternalDatabase; import org.apache.doris.datasource.iceberg.IcebergExternalTable; +import org.apache.doris.datasource.iceberg.IcebergHadoopExternalCatalog; import org.apache.doris.datasource.jdbc.JdbcExternalCatalog; import org.apache.doris.datasource.jdbc.JdbcExternalDatabase; import org.apache.doris.datasource.jdbc.JdbcExternalTable; @@ -49,8 +51,10 @@ import org.apache.doris.thrift.TStorageType; import com.google.common.collect.Lists; +import com.google.common.collect.Maps; import mockit.Mock; import mockit.MockUp; +import org.apache.iceberg.CatalogProperties; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; @@ -403,7 +407,10 @@ public boolean autoAnalyzeEnabled() { } }; IcebergExternalDatabase icebergDatabase = new IcebergExternalDatabase(null, 1L, "", ""); - IcebergExternalTable icebergTable = new IcebergExternalTable(0, "", "", null, icebergDatabase); + Map props = Maps.newHashMap(); + props.put(CatalogProperties.WAREHOUSE_LOCATION, "s3://tmp"); + IcebergExternalCatalog catalog = new IcebergHadoopExternalCatalog(0, "iceberg_ctl", "", props, ""); + IcebergExternalTable icebergTable = new IcebergExternalTable(0, "", "", catalog, icebergDatabase); Assertions.assertFalse(StatisticsUtil.isLongTimeColumn(icebergTable, Pair.of("index", column.getName()))); // Test table stats meta is null. diff --git a/regression-test/data/external_table_p0/hive/test_hive_case_sensibility.out b/regression-test/data/external_table_p0/hive/test_hive_case_sensibility.out new file mode 100644 index 00000000000000..8270e6e2bdee82 --- /dev/null +++ b/regression-test/data/external_table_p0/hive/test_hive_case_sensibility.out @@ -0,0 +1,255 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !sql1 -- +case_db1 + +-- !sql2 -- + +-- !sql3 -- +case_db2 + +-- !sql4 -- +case_db1 + +-- !sql5 -- + +-- !sql6 -- + +-- !sql7 -- + +-- !sql8 -- + +-- !sql9 -- +case_tbl14 + +-- !sql10 -- +case_tbl13 + +-- !sql11 -- + +-- !sql12 -- +case_tbl21 + +-- !sql13 -- +hive3_test_case_sensibility case_db1 case_tbl11 BASE TABLE hms \N \N -1 0 0 \N 0 \N \N \N \N \N utf-8 \N \N +hive3_test_case_sensibility case_db1 case_tbl12 BASE TABLE hms \N \N -1 0 0 \N 0 \N \N \N \N \N utf-8 \N \N +hive3_test_case_sensibility case_db1 case_tbl13 BASE TABLE hms \N \N -1 0 0 \N 0 \N \N \N \N \N utf-8 \N \N +hive3_test_case_sensibility case_db1 case_tbl14 BASE TABLE hms \N \N -1 0 0 \N 0 \N \N \N \N \N utf-8 \N \N + +-- !sql14 -- +hive3_test_case_sensibility case_db1 case_tbl11 k1 1 \N YES int \N \N 10 0 \N \N \N int(11) 10 0 \N \N +hive3_test_case_sensibility case_db1 case_tbl12 k1 1 \N YES int \N \N 10 0 \N \N \N int(11) 10 0 \N \N +hive3_test_case_sensibility case_db1 case_tbl13 k1 1 \N YES int \N \N 10 0 \N \N \N int(11) 10 0 \N \N +hive3_test_case_sensibility case_db1 case_tbl14 k1 1 \N YES int \N \N 10 0 \N \N \N int(11) 10 0 \N \N + +-- !sql15 -- +1 + +-- !sql16 -- +2 + +-- !sql17 -- +3 + +-- !sql18 -- +4 + +-- !sql19 -- + +-- !sql20 -- + +-- !sql21 -- +5 + +-- !sql22 -- +6 + +-- !sql222 -- +7 + +-- !sql23 -- +8 + +-- !sql24 -- +9 + +-- !sql25 -- +9 + +-- !sql26 -- + +-- !sql1 -- +case_db1 + +-- !sql2 -- + +-- !sql3 -- +case_db2 + +-- !sql4 -- +case_db1 + +-- !sql5 -- + +-- !sql6 -- + +-- !sql7 -- + +-- !sql8 -- + +-- !sql9 -- +case_tbl14 + +-- !sql10 -- +case_tbl13 + +-- !sql11 -- + +-- !sql12 -- +case_tbl21 + +-- !sql13 -- +hive3_test_case_sensibility case_db1 case_tbl11 BASE TABLE hms \N \N -1 0 0 \N 0 \N \N \N \N \N utf-8 \N \N +hive3_test_case_sensibility case_db1 case_tbl12 BASE TABLE hms \N \N -1 0 0 \N 0 \N \N \N \N \N utf-8 \N \N +hive3_test_case_sensibility case_db1 case_tbl13 BASE TABLE hms \N \N -1 0 0 \N 0 \N \N \N \N \N utf-8 \N \N +hive3_test_case_sensibility case_db1 case_tbl14 BASE TABLE hms \N \N -1 0 0 \N 0 \N \N \N \N \N utf-8 \N \N + +-- !sql14 -- +hive3_test_case_sensibility case_db1 case_tbl11 k1 1 \N YES int \N \N 10 0 \N \N \N int(11) 10 0 \N \N +hive3_test_case_sensibility case_db1 case_tbl12 k1 1 \N YES int \N \N 10 0 \N \N \N int(11) 10 0 \N \N +hive3_test_case_sensibility case_db1 case_tbl13 k1 1 \N YES int \N \N 10 0 \N \N \N int(11) 10 0 \N \N +hive3_test_case_sensibility case_db1 case_tbl14 k1 1 \N YES int \N \N 10 0 \N \N \N int(11) 10 0 \N \N + +-- !sql141 -- +1 +11 + +-- !sql15 -- +1 +11 + +-- !sql16 -- +2 + +-- !sql151 -- +3 +3 + +-- !sql17 -- +3 +3 + +-- !sql18 -- +4 + +-- !sql19 -- + +-- !sql20 -- + +-- !sql21 -- +5 + +-- !sql22 -- +6 + +-- !sql222 -- +7 + +-- !sql23 -- +8 + +-- !sql24 -- +9 + +-- !sql25 -- +9 + +-- !sql26 -- + +-- !sql1 -- +case_db1 + +-- !sql2 -- + +-- !sql3 -- +case_db2 + +-- !sql4 -- +case_db1 + +-- !sql5 -- + +-- !sql6 -- + +-- !sql7 -- + +-- !sql8 -- + +-- !sql9 -- +case_tbl14 + +-- !sql10 -- +case_tbl13 + +-- !sql11 -- + +-- !sql12 -- +case_tbl21 + +-- !sql13 -- +hive3_test_case_sensibility case_db1 case_tbl11 BASE TABLE hms \N \N -1 0 0 \N 0 \N \N \N \N \N utf-8 \N \N +hive3_test_case_sensibility case_db1 case_tbl12 BASE TABLE hms \N \N -1 0 0 \N 0 \N \N \N \N \N utf-8 \N \N +hive3_test_case_sensibility case_db1 case_tbl13 BASE TABLE hms \N \N -1 0 0 \N 0 \N \N \N \N \N utf-8 \N \N +hive3_test_case_sensibility case_db1 case_tbl14 BASE TABLE hms \N \N -1 0 0 \N 0 \N \N \N \N \N utf-8 \N \N + +-- !sql14 -- +hive3_test_case_sensibility case_db1 case_tbl11 k1 1 \N YES int \N \N 10 0 \N \N \N int(11) 10 0 \N \N +hive3_test_case_sensibility case_db1 case_tbl12 k1 1 \N YES int \N \N 10 0 \N \N \N int(11) 10 0 \N \N +hive3_test_case_sensibility case_db1 case_tbl13 k1 1 \N YES int \N \N 10 0 \N \N \N int(11) 10 0 \N \N +hive3_test_case_sensibility case_db1 case_tbl14 k1 1 \N YES int \N \N 10 0 \N \N \N int(11) 10 0 \N \N + +-- !sql141 -- +1 +11 + +-- !sql15 -- +1 +11 + +-- !sql16 -- +2 + +-- !sql151 -- +3 +3 + +-- !sql17 -- +3 +3 + +-- !sql18 -- +4 + +-- !sql19 -- + +-- !sql20 -- + +-- !sql21 -- +5 + +-- !sql22 -- +6 + +-- !sql222 -- +7 + +-- !sql23 -- +8 + +-- !sql24 -- +9 + +-- !sql25 -- +9 + +-- !sql26 -- + diff --git a/regression-test/data/external_table_p0/iceberg/test_iceberg_case_sensibility.out b/regression-test/data/external_table_p0/iceberg/test_iceberg_case_sensibility.out deleted file mode 100644 index 5475521a32122c..00000000000000 --- a/regression-test/data/external_table_p0/iceberg/test_iceberg_case_sensibility.out +++ /dev/null @@ -1,172 +0,0 @@ --- This file is automatically generated. You should know what you did if you want to edit this --- !sql -- -iceberg_case_sensibility_TBL1 - --- !sql -- -iceberg_case_sensibility_tbl2 - --- !sql11 -- -1 -2 - --- !sql12 -- -1 -2 - --- !sql -- -iceberg_case_sensibility_TBL1 - --- !sql -- -iceberg_case_sensibility_tbl2 - --- !sql11 -- -1 -2 - --- !sql12 -- -1 -2 - --- !sql -- -iceberg_case_sensibility_tbl1 - --- !sql -- -iceberg_case_sensibility_tbl2 - --- !sql11 -- -1 - --- !sql12 -- -1 - --- !sql -- -iceberg_case_sensibility_tbl1 - --- !sql -- -iceberg_case_sensibility_tbl2 - --- !sql11 -- -1 -2 - --- !sql12 -- -1 -2 - --- !sql11 -- -1 -2 - --- !sql12 -- -1 -2 - --- !sql -- -iceberg_case_sensibility_tbl1 - --- !sql -- -iceberg_case_sensibility_tbl2 - --- !sql11 -- -1 -2 - --- !sql12 -- -1 -2 - --- !sql11 -- -1 -2 - --- !sql12 -- -1 -2 - --- !sql -- -iceberg_case_sensibility_tbl1 - --- !sql -- -iceberg_case_sensibility_tbl2 - --- !sql11 -- -1 -2 - --- !sql12 -- -1 -2 - --- !sql11 -- -1 -2 - --- !sql12 -- -1 -2 - --- !sql -- -iceberg_case_sensibility_TBL1 - --- !sql -- -iceberg_case_sensibility_tbl2 - --- !sql11 -- -1 -2 - --- !sql12 -- -1 -2 - --- !sql11 -- -1 -2 - --- !sql12 -- -1 -2 - --- !sql -- -iceberg_case_sensibility_TBL1 - --- !sql -- -iceberg_case_sensibility_tbl2 - --- !sql11 -- -1 -2 - --- !sql12 -- -1 -2 - --- !sql11 -- -1 -2 - --- !sql12 -- -1 -2 - --- !sql -- - --- !sql -- -iceberg_case_sensibility_tbl2 - --- !sql11 -- -1 -2 - --- !sql12 -- -1 -2 - --- !sql11 -- -1 -2 - --- !sql12 -- -1 -2 - diff --git a/regression-test/data/external_table_p0/iceberg/test_iceberg_hadoop_case_sensibility.out b/regression-test/data/external_table_p0/iceberg/test_iceberg_hadoop_case_sensibility.out new file mode 100644 index 00000000000000..2b0147259bbc54 --- /dev/null +++ b/regression-test/data/external_table_p0/iceberg/test_iceberg_hadoop_case_sensibility.out @@ -0,0 +1,277 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !sql1 -- +iceberg_hadoop_case_db1 + +-- !sql3 -- +iceberg_hadoop_case_db2 + +-- !sql2 -- +ICEBERG_HADOOP_CASE_DB2 + +-- !sql4 -- +iceberg_hadoop_case_db1 + +-- !sql5 -- + +-- !sql6 -- + +-- !sql7 -- + +-- !sqlx -- +CASE_TBL11 + +-- !sql8 -- +CASE_TBL14 + +-- !sql9 -- + +-- !sql10 -- +case_tbl13 + +-- !sql11 -- + +-- !sql12 -- + +-- !sql12 -- +case_tbl22 + +-- !sql13 -- +test_iceberg_case_sensibility_hadoop iceberg_hadoop_case_db2 CASE_TBL22 BASE TABLE iceberg \N \N -1 0 0 \N 0 \N \N \N \N \N utf-8 \N \N +test_iceberg_case_sensibility_hadoop iceberg_hadoop_case_db2 case_tbl22 BASE TABLE iceberg \N \N -1 0 0 \N 0 \N \N \N \N \N utf-8 \N \N + +-- !sql14 -- +test_iceberg_case_sensibility_hadoop iceberg_hadoop_case_db2 CASE_TBL22 k1 1 \N YES int \N \N 10 0 \N \N \N int(11) 10 0 \N \N +test_iceberg_case_sensibility_hadoop iceberg_hadoop_case_db2 case_tbl22 k1 1 \N YES int \N \N 10 0 \N \N \N int(11) 10 0 \N \N + +-- !sql131 -- +test_iceberg_case_sensibility_hadoop ICEBERG_HADOOP_CASE_DB2 case_tbl21 BASE TABLE iceberg \N \N -1 0 0 \N 0 \N \N \N \N \N utf-8 \N \N + +-- !sql141 -- +test_iceberg_case_sensibility_hadoop ICEBERG_HADOOP_CASE_DB2 case_tbl21 k1 1 \N YES int \N \N 10 0 \N \N \N int(11) 10 0 \N \N + +-- !sqlz1 -- +1 + +-- !sqlz2 -- +1 + +-- !sql16 -- +20 + +-- !sql17 -- +3 + +-- !sql18 -- +4 + +-- !sql21 -- +5 + +-- !sql22 -- +6 + +-- !sql222 -- +7 + +-- !sql23 -- +8 + +-- !sql24 -- +9 + +-- !sql25 -- +9 + +-- !sql26 -- + +-- !sql1 -- +iceberg_hadoop_case_db1 + +-- !sql3 -- +iceberg_hadoop_case_db2 + +-- !sql2 -- +ICEBERG_HADOOP_CASE_DB2 + +-- !sql4 -- +iceberg_hadoop_case_db1 + +-- !sql5 -- + +-- !sql6 -- + +-- !sql7 -- + +-- !sqlx -- + +-- !sql8 -- + +-- !sql9 -- +case_tbl14 + +-- !sql10 -- +case_tbl13 + +-- !sql11 -- + +-- !sql12 -- + +-- !sql12 -- +case_tbl22 + +-- !sql13 -- +test_iceberg_case_sensibility_hadoop iceberg_hadoop_case_db2 case_tbl22 BASE TABLE iceberg \N \N -1 0 0 \N 0 \N \N \N \N \N utf-8 \N \N + +-- !sql14 -- +test_iceberg_case_sensibility_hadoop iceberg_hadoop_case_db2 case_tbl22 k1 1 \N YES int \N \N 10 0 \N \N \N int(11) 10 0 \N \N + +-- !sql131 -- +test_iceberg_case_sensibility_hadoop ICEBERG_HADOOP_CASE_DB2 case_tbl21 BASE TABLE iceberg \N \N -1 0 0 \N 0 \N \N \N \N \N utf-8 \N \N + +-- !sql141 -- +test_iceberg_case_sensibility_hadoop ICEBERG_HADOOP_CASE_DB2 case_tbl21 k1 1 \N YES int \N \N 10 0 \N \N \N int(11) 10 0 \N \N + +-- !sqly -- +11 + +-- !sqlz1 -- +1 +1 + +-- !sqlz2 -- +1 +1 + +-- !sql16 -- +21 + +-- !sql1511 -- +3 +3 + +-- !sql1512 -- +3 +3 + +-- !sql18 -- +5 + +-- !sql18 -- +5 + +-- !sql21 -- +5 + +-- !sql22 -- +6 + +-- !sql222 -- +7 + +-- !sql23 -- +11 +8 + +-- !sql24 -- +9 + +-- !sql25 -- +9 + +-- !sql26 -- + +-- !sql1 -- +iceberg_hadoop_case_db1 + +-- !sql3 -- +iceberg_hadoop_case_db2 + +-- !sql2 -- +ICEBERG_HADOOP_CASE_DB2 + +-- !sql4 -- +iceberg_hadoop_case_db1 + +-- !sql5 -- + +-- !sql6 -- + +-- !sql7 -- + +-- !sqlx -- + +-- !sql8 -- +CASE_TBL14 + +-- !sql9 -- + +-- !sql10 -- +case_tbl13 + +-- !sql11 -- + +-- !sql12 -- + +-- !sql12 -- + +-- !sql13 -- +test_iceberg_case_sensibility_hadoop iceberg_hadoop_case_db2 CASE_TBL22 BASE TABLE iceberg \N \N -1 0 0 \N 0 \N \N \N \N \N utf-8 \N \N + +-- !sql14 -- +test_iceberg_case_sensibility_hadoop iceberg_hadoop_case_db2 CASE_TBL22 k1 1 \N YES int \N \N 10 0 \N \N \N int(11) 10 0 \N \N + +-- !sql131 -- +test_iceberg_case_sensibility_hadoop ICEBERG_HADOOP_CASE_DB2 case_tbl21 BASE TABLE iceberg \N \N -1 0 0 \N 0 \N \N \N \N \N utf-8 \N \N + +-- !sql141 -- +test_iceberg_case_sensibility_hadoop ICEBERG_HADOOP_CASE_DB2 case_tbl21 k1 1 \N YES int \N \N 10 0 \N \N \N int(11) 10 0 \N \N + +-- !sqly -- +11 + +-- !sqlz1 -- +1 +1 + +-- !sqlz2 -- +1 +1 + +-- !sql16 -- +21 + +-- !sql1511 -- +3 +3 + +-- !sql1512 -- +3 +3 + +-- !sql18 -- +5 + +-- !sql18 -- +5 + +-- !sql21 -- +5 + +-- !sql22 -- +6 + +-- !sql222 -- +7 + +-- !sql23 -- +11 +8 + +-- !sql24 -- +9 + +-- !sql25 -- +9 + +-- !sql26 -- + diff --git a/regression-test/data/external_table_p0/iceberg/test_iceberg_hms_case_sensibility.out b/regression-test/data/external_table_p0/iceberg/test_iceberg_hms_case_sensibility.out new file mode 100644 index 00000000000000..a3a1d68e7e2aa2 --- /dev/null +++ b/regression-test/data/external_table_p0/iceberg/test_iceberg_hms_case_sensibility.out @@ -0,0 +1,240 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !sql1 -- +iceberg_hms_case_db1 + +-- !sql2 -- + +-- !sql3 -- +iceberg_hms_case_db2 + +-- !sql4 -- +iceberg_hms_case_db1 + +-- !sql5 -- + +-- !sql6 -- + +-- !sql7 -- + +-- !sql8 -- + +-- !sql9 -- +case_tbl14 + +-- !sql10 -- +case_tbl13 + +-- !sql11 -- + +-- !sql12 -- +case_tbl21 + +-- !sql13 -- +test_iceberg_case_sensibility_hms iceberg_hms_case_db1 case_tbl11 BASE TABLE iceberg \N \N -1 0 0 \N 0 \N \N \N \N \N utf-8 \N \N +test_iceberg_case_sensibility_hms iceberg_hms_case_db1 case_tbl12 BASE TABLE iceberg \N \N -1 0 0 \N 0 \N \N \N \N \N utf-8 \N \N +test_iceberg_case_sensibility_hms iceberg_hms_case_db1 case_tbl13 BASE TABLE iceberg \N \N -1 0 0 \N 0 \N \N \N \N \N utf-8 \N \N +test_iceberg_case_sensibility_hms iceberg_hms_case_db1 case_tbl14 BASE TABLE iceberg \N \N -1 0 0 \N 0 \N \N \N \N \N utf-8 \N \N + +-- !sql14 -- +test_iceberg_case_sensibility_hms iceberg_hms_case_db1 case_tbl11 k1 1 \N YES int \N \N 10 0 \N \N \N int(11) 10 0 \N \N +test_iceberg_case_sensibility_hms iceberg_hms_case_db1 case_tbl12 k1 1 \N YES int \N \N 10 0 \N \N \N int(11) 10 0 \N \N +test_iceberg_case_sensibility_hms iceberg_hms_case_db1 case_tbl13 k1 1 \N YES int \N \N 10 0 \N \N \N int(11) 10 0 \N \N +test_iceberg_case_sensibility_hms iceberg_hms_case_db1 case_tbl14 k1 1 \N YES int \N \N 10 0 \N \N \N int(11) 10 0 \N \N + +-- !sql15 -- +1 + +-- !sql16 -- +2 + +-- !sql17 -- +3 + +-- !sql18 -- +4 + +-- !sql21 -- +5 + +-- !sql22 -- +6 + +-- !sql222 -- +7 + +-- !sql23 -- +8 + +-- !sql24 -- +9 + +-- !sql25 -- + +-- !sql26 -- + +-- !sql1 -- +iceberg_hms_case_db1 + +-- !sql2 -- + +-- !sql3 -- +iceberg_hms_case_db2 + +-- !sql4 -- +iceberg_hms_case_db1 + +-- !sql5 -- + +-- !sql6 -- + +-- !sql7 -- + +-- !sql8 -- + +-- !sql9 -- +case_tbl14 + +-- !sql10 -- +case_tbl13 + +-- !sql11 -- + +-- !sql12 -- +case_tbl21 + +-- !sql13 -- +test_iceberg_case_sensibility_hms iceberg_hms_case_db1 case_tbl11 BASE TABLE iceberg \N \N -1 0 0 \N 0 \N \N \N \N \N utf-8 \N \N +test_iceberg_case_sensibility_hms iceberg_hms_case_db1 case_tbl12 BASE TABLE iceberg \N \N -1 0 0 \N 0 \N \N \N \N \N utf-8 \N \N +test_iceberg_case_sensibility_hms iceberg_hms_case_db1 case_tbl13 BASE TABLE iceberg \N \N -1 0 0 \N 0 \N \N \N \N \N utf-8 \N \N +test_iceberg_case_sensibility_hms iceberg_hms_case_db1 case_tbl14 BASE TABLE iceberg \N \N -1 0 0 \N 0 \N \N \N \N \N utf-8 \N \N + +-- !sql14 -- +test_iceberg_case_sensibility_hms iceberg_hms_case_db1 case_tbl11 k1 1 \N YES int \N \N 10 0 \N \N \N int(11) 10 0 \N \N +test_iceberg_case_sensibility_hms iceberg_hms_case_db1 case_tbl12 k1 1 \N YES int \N \N 10 0 \N \N \N int(11) 10 0 \N \N +test_iceberg_case_sensibility_hms iceberg_hms_case_db1 case_tbl13 k1 1 \N YES int \N \N 10 0 \N \N \N int(11) 10 0 \N \N +test_iceberg_case_sensibility_hms iceberg_hms_case_db1 case_tbl14 k1 1 \N YES int \N \N 10 0 \N \N \N int(11) 10 0 \N \N + +-- !sql141 -- +1 +11 + +-- !sql15 -- +1 +11 + +-- !sql16 -- +2 + +-- !sql151 -- +3 +3 + +-- !sql17 -- +3 +3 + +-- !sql18 -- +4 + +-- !sql21 -- +5 + +-- !sql22 -- +6 + +-- !sql222 -- +7 + +-- !sql23 -- +8 + +-- !sql24 -- +9 + +-- !sql25 -- + +-- !sql26 -- + +-- !sql1 -- +iceberg_hms_case_db1 + +-- !sql2 -- + +-- !sql3 -- +iceberg_hms_case_db2 + +-- !sql4 -- +iceberg_hms_case_db1 + +-- !sql5 -- + +-- !sql6 -- + +-- !sql7 -- + +-- !sql8 -- + +-- !sql9 -- +case_tbl14 + +-- !sql10 -- +case_tbl13 + +-- !sql11 -- + +-- !sql12 -- +case_tbl21 + +-- !sql13 -- +test_iceberg_case_sensibility_hms iceberg_hms_case_db1 case_tbl11 BASE TABLE iceberg \N \N -1 0 0 \N 0 \N \N \N \N \N utf-8 \N \N +test_iceberg_case_sensibility_hms iceberg_hms_case_db1 case_tbl12 BASE TABLE iceberg \N \N -1 0 0 \N 0 \N \N \N \N \N utf-8 \N \N +test_iceberg_case_sensibility_hms iceberg_hms_case_db1 case_tbl13 BASE TABLE iceberg \N \N -1 0 0 \N 0 \N \N \N \N \N utf-8 \N \N +test_iceberg_case_sensibility_hms iceberg_hms_case_db1 case_tbl14 BASE TABLE iceberg \N \N -1 0 0 \N 0 \N \N \N \N \N utf-8 \N \N + +-- !sql14 -- +test_iceberg_case_sensibility_hms iceberg_hms_case_db1 case_tbl11 k1 1 \N YES int \N \N 10 0 \N \N \N int(11) 10 0 \N \N +test_iceberg_case_sensibility_hms iceberg_hms_case_db1 case_tbl12 k1 1 \N YES int \N \N 10 0 \N \N \N int(11) 10 0 \N \N +test_iceberg_case_sensibility_hms iceberg_hms_case_db1 case_tbl13 k1 1 \N YES int \N \N 10 0 \N \N \N int(11) 10 0 \N \N +test_iceberg_case_sensibility_hms iceberg_hms_case_db1 case_tbl14 k1 1 \N YES int \N \N 10 0 \N \N \N int(11) 10 0 \N \N + +-- !sql141 -- +1 +11 + +-- !sql15 -- +1 +11 + +-- !sql16 -- +2 + +-- !sql151 -- +3 +3 + +-- !sql17 -- +3 +3 + +-- !sql18 -- +4 + +-- !sql21 -- +5 + +-- !sql22 -- +6 + +-- !sql222 -- +7 + +-- !sql23 -- +8 + +-- !sql24 -- +9 + +-- !sql25 -- + +-- !sql26 -- + diff --git a/regression-test/data/external_table_p0/iceberg/test_iceberg_rest_case_sensibility.out b/regression-test/data/external_table_p0/iceberg/test_iceberg_rest_case_sensibility.out new file mode 100644 index 00000000000000..d6d88db3dbed05 --- /dev/null +++ b/regression-test/data/external_table_p0/iceberg/test_iceberg_rest_case_sensibility.out @@ -0,0 +1,277 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !sql1 -- +iceberg_rest_case_db1 + +-- !sql3 -- +iceberg_rest_case_db2 + +-- !sql2 -- +ICEBERG_REST_CASE_DB2 + +-- !sql4 -- +iceberg_rest_case_db1 + +-- !sql5 -- + +-- !sql6 -- + +-- !sql7 -- + +-- !sqlx -- +CASE_TBL11 + +-- !sql8 -- +CASE_TBL14 + +-- !sql9 -- + +-- !sql10 -- +case_tbl13 + +-- !sql11 -- + +-- !sql12 -- + +-- !sql12 -- +case_tbl22 + +-- !sql13 -- +test_iceberg_case_sensibility_rest iceberg_rest_case_db2 CASE_TBL22 BASE TABLE iceberg \N \N -1 0 0 \N 0 \N \N \N \N \N utf-8 \N \N +test_iceberg_case_sensibility_rest iceberg_rest_case_db2 case_tbl22 BASE TABLE iceberg \N \N -1 0 0 \N 0 \N \N \N \N \N utf-8 \N \N + +-- !sql14 -- +test_iceberg_case_sensibility_rest iceberg_rest_case_db2 CASE_TBL22 k1 1 \N YES int \N \N 10 0 \N \N \N int(11) 10 0 \N \N +test_iceberg_case_sensibility_rest iceberg_rest_case_db2 case_tbl22 k1 1 \N YES int \N \N 10 0 \N \N \N int(11) 10 0 \N \N + +-- !sql131 -- +test_iceberg_case_sensibility_rest ICEBERG_REST_CASE_DB2 case_tbl21 BASE TABLE iceberg \N \N -1 0 0 \N 0 \N \N \N \N \N utf-8 \N \N + +-- !sql141 -- +test_iceberg_case_sensibility_rest ICEBERG_REST_CASE_DB2 case_tbl21 k1 1 \N YES int \N \N 10 0 \N \N \N int(11) 10 0 \N \N + +-- !sqlz1 -- +1 + +-- !sqlz2 -- +1 + +-- !sql16 -- +20 + +-- !sql17 -- +3 + +-- !sql18 -- +4 + +-- !sql21 -- +5 + +-- !sql22 -- +6 + +-- !sql222 -- +7 + +-- !sql23 -- +8 + +-- !sql24 -- +9 + +-- !sql25 -- +9 + +-- !sql26 -- + +-- !sql1 -- +iceberg_rest_case_db1 + +-- !sql3 -- +iceberg_rest_case_db2 + +-- !sql2 -- +ICEBERG_REST_CASE_DB2 + +-- !sql4 -- +iceberg_rest_case_db1 + +-- !sql5 -- + +-- !sql6 -- + +-- !sql7 -- + +-- !sqlx -- + +-- !sql8 -- + +-- !sql9 -- +case_tbl14 + +-- !sql10 -- +case_tbl13 + +-- !sql11 -- + +-- !sql12 -- + +-- !sql12 -- +case_tbl22 + +-- !sql13 -- +test_iceberg_case_sensibility_rest iceberg_rest_case_db2 case_tbl22 BASE TABLE iceberg \N \N -1 0 0 \N 0 \N \N \N \N \N utf-8 \N \N + +-- !sql14 -- +test_iceberg_case_sensibility_rest iceberg_rest_case_db2 case_tbl22 k1 1 \N YES int \N \N 10 0 \N \N \N int(11) 10 0 \N \N + +-- !sql131 -- +test_iceberg_case_sensibility_rest ICEBERG_REST_CASE_DB2 case_tbl21 BASE TABLE iceberg \N \N -1 0 0 \N 0 \N \N \N \N \N utf-8 \N \N + +-- !sql141 -- +test_iceberg_case_sensibility_rest ICEBERG_REST_CASE_DB2 case_tbl21 k1 1 \N YES int \N \N 10 0 \N \N \N int(11) 10 0 \N \N + +-- !sqly -- +11 + +-- !sqlz1 -- +1 +1 + +-- !sqlz2 -- +1 +1 + +-- !sql16 -- +21 + +-- !sql1511 -- +3 +3 + +-- !sql1512 -- +3 +3 + +-- !sql18 -- +5 + +-- !sql18 -- +5 + +-- !sql21 -- +5 + +-- !sql22 -- +6 + +-- !sql222 -- +7 + +-- !sql23 -- +11 +8 + +-- !sql24 -- +9 + +-- !sql25 -- +9 + +-- !sql26 -- + +-- !sql1 -- +iceberg_rest_case_db1 + +-- !sql3 -- +iceberg_rest_case_db2 + +-- !sql2 -- +ICEBERG_REST_CASE_DB2 + +-- !sql4 -- +iceberg_rest_case_db1 + +-- !sql5 -- + +-- !sql6 -- + +-- !sql7 -- + +-- !sqlx -- + +-- !sql8 -- +CASE_TBL14 + +-- !sql9 -- + +-- !sql10 -- +case_tbl13 + +-- !sql11 -- + +-- !sql12 -- + +-- !sql12 -- + +-- !sql13 -- +test_iceberg_case_sensibility_rest iceberg_rest_case_db2 CASE_TBL22 BASE TABLE iceberg \N \N -1 0 0 \N 0 \N \N \N \N \N utf-8 \N \N + +-- !sql14 -- +test_iceberg_case_sensibility_rest iceberg_rest_case_db2 CASE_TBL22 k1 1 \N YES int \N \N 10 0 \N \N \N int(11) 10 0 \N \N + +-- !sql131 -- +test_iceberg_case_sensibility_rest ICEBERG_REST_CASE_DB2 case_tbl21 BASE TABLE iceberg \N \N -1 0 0 \N 0 \N \N \N \N \N utf-8 \N \N + +-- !sql141 -- +test_iceberg_case_sensibility_rest ICEBERG_REST_CASE_DB2 case_tbl21 k1 1 \N YES int \N \N 10 0 \N \N \N int(11) 10 0 \N \N + +-- !sqly -- +11 + +-- !sqlz1 -- +1 +1 + +-- !sqlz2 -- +1 +1 + +-- !sql16 -- +21 + +-- !sql1511 -- +3 +3 + +-- !sql1512 -- +3 +3 + +-- !sql18 -- +5 + +-- !sql18 -- +5 + +-- !sql21 -- +5 + +-- !sql22 -- +6 + +-- !sql222 -- +7 + +-- !sql23 -- +11 +8 + +-- !sql24 -- +9 + +-- !sql25 -- +9 + +-- !sql26 -- + diff --git a/regression-test/suites/external_table_p0/hive/ddl/test_hive_ctas.groovy b/regression-test/suites/external_table_p0/hive/ddl/test_hive_ctas.groovy index 265d200984ed58..5cdf749414a7e0 100644 --- a/regression-test/suites/external_table_p0/hive/ddl/test_hive_ctas.groovy +++ b/regression-test/suites/external_table_p0/hive/ddl/test_hive_ctas.groovy @@ -358,7 +358,7 @@ suite("test_hive_ctas", "p0,external,hive,external_docker,external_docker_hive") // 1. external to external un-partitioned table test { sql """ DROP TABLE ${catalog_name}.test_no_err.hive_ctas1 """ - exception "errCode = 2, detailMessage = Unknown table 'hive_ctas1' in test_no_err" + exception "Failed to get table: 'hive_ctas1'" } test { sql """ CREATE TABLE ${catalog_name}.test_no_err.hive_ctas1 (col1) ENGINE=hive @@ -568,7 +568,7 @@ suite("test_hive_ctas", "p0,external,hive,external_docker,external_docker_hive") test_ctas_exception(file_format, catalog_name) test_ctas_all_types(file_format, catalog_name) } - sql """drop catalog if exists ${catalog_name}""" + // sql """drop catalog if exists ${catalog_name}""" } finally { } } diff --git a/regression-test/suites/external_table_p0/hive/ddl/test_hive_database.groovy b/regression-test/suites/external_table_p0/hive/ddl/test_hive_database.groovy deleted file mode 100644 index efa92dac8da229..00000000000000 --- a/regression-test/suites/external_table_p0/hive/ddl/test_hive_database.groovy +++ /dev/null @@ -1,54 +0,0 @@ -// 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_hive_database", "p0,external,hive,external_docker,external_docker_hive") { - String enabled = context.config.otherConfigs.get("enableHiveTest") - if (enabled != null && enabled.equalsIgnoreCase("true")) { - String externalEnvIp = context.config.otherConfigs.get("externalEnvIp") - String hms_port = context.config.otherConfigs.get("hive2HmsPort") - String hdfs_port = context.config.otherConfigs.get("hive2HdfsPort") - String catalog_name = "test_hive_database" - String prefix = catalog_name - - sql """drop catalog if exists ${catalog_name};""" - - sql """ - create catalog if not exists ${catalog_name} properties ( - 'type'='hms', - 'hive.metastore.uris' = 'thrift://${externalEnvIp}:${hms_port}', - 'fs.defaultFS' = 'hdfs://${externalEnvIp}:${hdfs_port}', - 'use_meta_cache' = 'true' - ); - """ - - // Database names are case insensitive - logger.info("catalog " + catalog_name + " created") - sql """switch ${catalog_name};""" - logger.info("switched to catalog " + catalog_name) - - sql """ drop database if exists ${prefix}_db1 """ - sql """ create database ${prefix}_Db1 """ - def ret1 = sql """ show databases """ - assertTrue(ret1.toString().contains("${prefix}_db1")) - - test { - sql """ create database ${prefix}_dB1 """ - exception "database exists" - } - sql """ drop database if exists ${prefix}_db1 """ - } -} diff --git a/regression-test/suites/external_table_p0/hive/test_hive_case_sensibility.groovy b/regression-test/suites/external_table_p0/hive/test_hive_case_sensibility.groovy new file mode 100644 index 00000000000000..caea986adecd15 --- /dev/null +++ b/regression-test/suites/external_table_p0/hive/test_hive_case_sensibility.groovy @@ -0,0 +1,358 @@ +// 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_hive_case_sensibility", "p0,external,doris,external_docker,external_docker_doris") { + String enabled = context.config.otherConfigs.get("enableHiveTest") + if (enabled == null || !enabled.equalsIgnoreCase("true")) { + return; + } + + try { + for (String hivePrefix : ["hive3"]) { + String hms_port = context.config.otherConfigs.get(hivePrefix + "HmsPort") + String catalog_name = "${hivePrefix}_test_case_sensibility" + String externalEnvIp = context.config.otherConfigs.get("externalEnvIp") + String hdfs_port = context.config.otherConfigs.get(hivePrefix + "HdfsPort") + + for (String case_type : ["0", "1", "2"]) { + sql """drop catalog if exists ${catalog_name};""" + sql """ + create catalog if not exists ${catalog_name} properties ( + 'type'='hms', + 'hive.metastore.uris' = 'thrift://${externalEnvIp}:${hms_port}', + 'only_test_lower_case_table_names' = '${case_type}' + ); + """ + sql """switch ${catalog_name};""" + // 0. clear env + sql "drop database if exists case_db1 force"; + sql "drop database if exists case_db2 force"; + + // 1. create db + sql """create database case_db1;""" + test { + sql """create database CASE_DB1;""" // conflict + exception "Can't create database 'CASE_DB1'; database exists" + } + sql """create database CASE_DB2;""" + sql """create database if not exists CASE_DB1;""" + sql """create database if not exists case_db1;""" + sql """create database if not exists case_db2;""" + + qt_sql1 """show databases like "%case_db1%";""" + qt_sql2 """show databases like "%CASE_DB1%";""" // empty + qt_sql3 """show databases like "%case_db2%";""" + test { + sql """create database CASE_DB2;""" // conflict + exception "database exists" + exception "CASE_DB2" + } + test { + sql """create database case_db2;""" // conflict + exception "database exists" + exception "case_db2" + } + // 2. drop database + test { + sql """drop database CASE_DB1""" + exception "database doesn't exist" + exception "CASE_DB1" + } + sql """drop database if exists CASE_DB1;""" + qt_sql4 """show databases like "%case_db1%";""" // still exists + sql """drop database case_db1;""" + qt_sql5 """show databases like "%case_db1%";""" // empty + + test { + sql """drop database CASE_DB2;""" + exception "database doesn't exist" + exception "CASE_DB2" + } + sql """drop database case_db2;""" + test { + sql """drop database case_db1""" + exception "database doesn't exist" + exception "case_db1" + } + test { + sql """drop database case_db2""" + exception "database doesn't exist" + exception "case_db2" + } + sql """drop database if exists case_db2;""" + qt_sql6 """show databases like "%case_db1%";""" // empty + qt_sql7 """show databases like "%case_db2%";""" // empty + + // 3. recreate db to test create table + sql """create database case_db1;""" + sql """create database CASE_DB2;""" + + test { + sql """use CASE_DB2""" + exception "Unknown database 'CASE_DB2'" + } + + test { + sql """create table CASE_DB2.case_tbl21 (k1 int);""" + exception "Failed to get database: 'CASE_DB2'" + } + test { + sql """create table if not exists CASE_DB2.case_tbl21 (k1 int);""" + exception "Failed to get database: 'CASE_DB2'" + } + sql """create table case_db2.case_tbl21 (k1 int);""" + sql """create table case_db2.CASE_TBL22 (k1 int);""" + sql """create table case_db1.case_tbl11 (k1 int);""" + + test { + sql """create table case_db1.case_tbl11 (k1 int);""" + exception "Table 'case_tbl11' already exists" + } + sql """create table if not exists case_db1.case_tbl11 (k1 int);""" + sql """create table if not exists case_db1.CASE_TBL11 (k1 int);""" + + sql """create table case_db1.CASE_TBL12 (k1 int);""" + sql """use case_db1;""" + sql """create table case_tbl13 (k1 int);""" + sql """create table CASE_TBL14 (k1 int);""" + + qt_sql8 """show tables like "%CASE_TBL14%"""" // empty + qt_sql9 """show tables like "%case_tbl14%"""" + qt_sql10 """show tables like "%case_tbl13%"""" + + test { + sql """show tables from CASE_DB2 like "%case_tbl14%"""" + exception "Unknown database 'CASE_DB2'" + } + qt_sql11 """show tables from case_db2 like "%case_tbl14%"""" // empty + qt_sql12 """show tables from case_db2 like "%case_tbl21%"""" + + order_qt_sql13 """select * from information_schema.tables where TABLE_SCHEMA="case_db1";""" + order_qt_sql14 """select * from information_schema.columns where TABLE_SCHEMA="case_db1";""" + + // 4. insert + /// full qualified name + test { + sql """insert into CASE_DB2.CASE_TBL22 values(1);""" + exception "Database [CASE_DB2] does not exist" + } + test { + sql """insert into CASE_DB2.case_tbl22 values(1);""" + exception "Database [CASE_DB2] does not exist." + } + + if (case_type.equals("0")) { + test { + sql """insert into case_db2.CASE_TBL22 values(1);""" + exception "Table [CASE_TBL22] does not exist in database [case_db2]" + } + } else if (case_type.equals("1") || case_type.equals("2")) { + sql """insert into case_db2.CASE_TBL22 values(11);""" + } + + sql """insert into case_db2.case_tbl22 values(1);""" + test { + sql """select * from CASE_DB2.CASE_TBL22""" + exception "Database [CASE_DB2] does not exist" + } + test { + sql """select * from CASE_DB2.case_tbl22""" + exception "Database [CASE_DB2] does not exist" + } + + if (case_type.equals("0")) { + test { + sql """select * from case_db2.CASE_TBL22""" + exception "Table [CASE_TBL22] does not exist in database [case_db2]" + } + } else if (case_type.equals("1") || case_type.equals("2")) { + order_qt_sql141 """select * from case_db2.CASE_TBL22""" + } + order_qt_sql15 """select * from case_db2.case_tbl22""" + + test { + sql """insert overwrite table CASE_DB2.CASE_TBL22 values(2);""" + exception "Database [CASE_DB2] does not exist" + } + test { + sql """insert overwrite table CASE_DB2.case_tbl22 values(2); """ + exception "Database [CASE_DB2] does not exist" + } + if (case_type.equals("0")) { + test { + sql """insert overwrite table case_db2.CASE_TBL22 values(2);""" + exception "Table [CASE_TBL22] does not exist in database [case_db2]" + } + } else if (case_type.equals("1") || case_type.equals("2")) { + sql """insert overwrite table case_db2.CASE_TBL22 values(2);""" + } + + sql """insert overwrite table case_db2.case_tbl22 values(2);""" + order_qt_sql16 """select * from case_db2.case_tbl22;""" + + /// not full qualified + sql """use case_db1;""" + if (case_type.equals("0")) { + test { + sql """insert into CASE_TBL12 values(3);""" + exception "Table [CASE_TBL12] does not exist in database [case_db1]" + } + } else if (case_type.equals("1") || case_type.equals("2")) { + sql """insert into CASE_TBL12 values(3);""" + } + + sql """insert into case_tbl12 values(3);""" + if (case_type.equals("0")) { + test { + sql """select * from CASE_TBL12""" + exception "Table [CASE_TBL12] does not exist in database [case_db1]" + } + } else if (case_type.equals("1") || case_type.equals("2")) { + order_qt_sql151 """select * from CASE_TBL12""" + } + order_qt_sql17 """select * from case_tbl12""" + + if (case_type.equals("0")) { + test { + sql """insert overwrite table CASE_TBL12 values(4);""" + exception "Table [CASE_TBL12] does not exist in database [case_db1]" + } + } else if (case_type.equals("1") || case_type.equals("2")) { + sql """insert overwrite table CASE_TBL12 values(4);""" + } + sql """insert overwrite table case_tbl12 values(4);""" + order_qt_sql18 """select * from case_tbl12;""" + + // 5. truncate + /// full qualified + test { + sql """truncate table CASE_DB2.CASE_TBL22""" + exception "Unknown database 'CASE_DB2'" + } + test { + sql """truncate table CASE_DB2.case_tbl22""" + exception "Unknown database 'CASE_DB2'" + } + if (case_type.equals("0")) { + test { + sql """truncate table case_db2.CASE_TBL22""" + exception "Unknown table 'CASE_TBL22'" + } + } else { + sql """truncate table case_db2.CASE_TBL22""" + } + sql """truncate table case_db2.case_tbl22 ;""" + qt_sql19 """select * from case_db2.case_tbl22;""" // empty + /// not full qualified + sql """use case_db1;""" + if (case_type.equals("0")) { + test { + sql """truncate table CASE_TBL12;""" + exception "Unknown table 'CASE_TBL12'" + } + } else { + sql """truncate table CASE_TBL12;""" + } + sql """truncate table case_tbl12;""" + qt_sql20 """select * from case_tbl12;""" // empty + + // 6. drop table + /// full qualified + test { + sql """drop table CASE_DB2.CASE_TBL22""" + exception "Failed to get database: 'CASE_DB2' in catalog" + } + test { + sql """drop table CASE_DB2.case_tbl22""" + exception "Failed to get database: 'CASE_DB2'" + } + test { + sql """drop table if exists CASE_DB2.case_tbl22;""" + exception "Failed to get database: 'CASE_DB2'" + } + if (case_type.equals("0")) { + test { + sql """drop table case_db2.CASE_TBL22""" + exception "Failed to get table: 'CASE_TBL22'" + } + sql """drop table case_db2.case_tbl22""" + } else { + sql """drop table case_db2.CASE_TBL22""" + } + + test { + sql """drop table case_db2.case_tbl22""" + exception "Failed to get table: 'case_tbl22'" + } + sql """drop table if exists case_db2.case_tbl22""" + + test { + sql """select * from case_db2.case_tbl22;""" + exception "Table [case_tbl22] does not exist in database [case_db2]" + } + sql """create table case_db2.case_tbl22 (k1 int);""" // recreate + sql """insert into case_db2.case_tbl22 values(5);""" + order_qt_sql21 """select * from case_db2.case_tbl22;""" + + /// not full qualified + sql """use case_db1;""" + if (case_type.equals("0")) { + test { + sql """drop table CASE_TBL12;""" + exception "Failed to get table: 'CASE_TBL12' in database: case_db1" + } + sql """drop table case_tbl12;""" + } else { + sql """drop table CASE_TBL12;""" + } + + test { + sql """select * from case_db2.case_tbl12;""" + exception "Table [case_tbl12] does not exist in database [case_db2]" + } + + // 7. re create and insert + sql """create table case_db2.case_tbl12 (k1 int);""" + sql """insert into case_db2.case_tbl12 values(6);""" + order_qt_sql22 """select * from case_db2.case_tbl12;""" + sql """insert overwrite table case_db2.case_tbl12 values(7);""" + order_qt_sql222 """select * from case_db2.case_tbl12;""" + + // 8. drop db force + sql """insert into case_db1.case_tbl13 values(8)""" + sql """insert into case_db1.case_tbl14 values(9)""" + order_qt_sql23 """select * from case_db1.case_tbl13;""" + order_qt_sql24 """select * from case_db1.case_tbl14;""" + + // use tvf to check data under dir + order_qt_sql25 """select * from hdfs( + "uri" = "hdfs://${externalEnvIp}:${hdfs_port}/user/hive/warehouse/case_db1.db/case_tbl14/*", + "format" = "orc" + ); + """ + sql """drop database case_db1 force;""" + order_qt_sql26 """select * from hdfs( + "uri" = "hdfs://${externalEnvIp}:${hdfs_port}/user/hive/warehouse/case_db1.db/case_tbl14/*", + "format" = "orc" + ); + """ // empty + } + } + } finally { + // sql """set enable_external_table_batch_mode=true""" + } +} diff --git a/regression-test/suites/external_table_p0/iceberg/test_iceberg_case_sensibility.groovy b/regression-test/suites/external_table_p0/iceberg/test_iceberg_case_sensibility.groovy deleted file mode 100644 index b258a17a98bf43..00000000000000 --- a/regression-test/suites/external_table_p0/iceberg/test_iceberg_case_sensibility.groovy +++ /dev/null @@ -1,218 +0,0 @@ -// 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_iceberg_case_sensibility", "p0,external,doris,external_docker,external_docker_doris") { - String enabled = context.config.otherConfigs.get("enableIcebergTest") - if (enabled != null && enabled.equalsIgnoreCase("true")) { - try { - String rest_port = context.config.otherConfigs.get("iceberg_rest_uri_port") - String minio_port = context.config.otherConfigs.get("iceberg_minio_port") - String externalEnvIp = context.config.otherConfigs.get("externalEnvIp") - String hms_port = context.config.otherConfigs.get("hive2HmsPort") - String hdfs_port = context.config.otherConfigs.get("hive2HdfsPort") - String default_fs = "hdfs://${externalEnvIp}:${hdfs_port}" - String warehouse = "${default_fs}/warehouse" - - // rest - for (String case_type : ["0", "1", "2"]) { - sql """drop catalog if exists test_iceberg_case_sensibility_rest""" - sql """CREATE CATALOG test_iceberg_case_sensibility_rest PROPERTIES ( - 'type'='iceberg', - 'iceberg.catalog.type'='rest', - 'uri' = 'http://${externalEnvIp}:${rest_port}', - "s3.access_key" = "admin", - "s3.secret_key" = "password", - "s3.endpoint" = "http://${externalEnvIp}:${minio_port}", - "s3.region" = "us-east-1", - "only_test_lower_case_table_names" = "${case_type}" - );""" - - sql """ switch test_iceberg_case_sensibility_rest """ - sql """ use test_db""" - sql """ drop table if exists iceberg_case_sensibility_TBL1""" - sql """ drop table if exists iceberg_case_sensibility_tbl2""" - sql """ create table iceberg_case_sensibility_TBL1(k1 int);""" - sql """ create table iceberg_case_sensibility_tbl2(k1 int);""" - sql """insert into iceberg_case_sensibility_TBL1 values(1)""" - sql """insert into iceberg_case_sensibility_tbl2 values(1)""" - - if (case_type.equals("0")) { - // store in origin, query case sensitive - qt_sql """show tables like "iceberg_case_sensibility_TBL1"""" - qt_sql """show tables like "iceberg_case_sensibility_tbl2"""" - sql """insert into iceberg_case_sensibility_TBL1 values(2)""" - sql """insert into iceberg_case_sensibility_tbl2 values(2)""" - order_qt_sql11 """select * from iceberg_case_sensibility_TBL1"""; - order_qt_sql12 """select * from iceberg_case_sensibility_tbl2"""; - test { - sql "select * from iceberg_case_sensibility_tbl1" - exception "does not exist in database" - } - test { - sql "insert into iceberg_case_sensibility_tbl1 values(1)" - exception "does not exist in database" - } - } else if (case_type.equals("1")) { - // store in lower case, query case insensitive - qt_sql """show tables like "iceberg_case_sensibility_tbl1"""" - qt_sql """show tables like "iceberg_case_sensibility_tbl2"""" - sql """insert into iceberg_case_sensibility_tbl1 values(2)""" - sql """insert into iceberg_case_sensibility_tbl2 values(2)""" - order_qt_sql11 """select * from iceberg_case_sensibility_TBL1"""; - order_qt_sql12 """select * from iceberg_case_sensibility_tbl2"""; - order_qt_sql11 """select * from iceberg_case_sensibility_tbl1"""; - order_qt_sql12 """select * from iceberg_case_sensibility_TBL2"""; - } else if (case_type.equals("2")) { - // store in origin, query case insensitive - qt_sql """show tables like "iceberg_case_sensibility_TBL1"""" - qt_sql """show tables like "iceberg_case_sensibility_tbl2"""" - sql """insert into iceberg_case_sensibility_tbl1 values(2)""" - sql """insert into iceberg_case_sensibility_tbl2 values(2)""" - order_qt_sql11 """select * from iceberg_case_sensibility_TBL1"""; - order_qt_sql12 """select * from iceberg_case_sensibility_tbl2"""; - order_qt_sql11 """select * from iceberg_case_sensibility_tbl1"""; - order_qt_sql12 """select * from iceberg_case_sensibility_TBL2"""; - } - - // hadoop - sql """drop catalog if exists test_iceberg_case_sensibility_hadoop""" - sql """CREATE CATALOG test_iceberg_case_sensibility_hadoop PROPERTIES ( - 'type'='iceberg', - 'iceberg.catalog.type'='hadoop', - 'warehouse' = 's3a://warehouse/wh', - "s3.access_key" = "admin", - "s3.secret_key" = "password", - "s3.endpoint" = "http://${externalEnvIp}:${minio_port}", - "s3.region" = "us-east-1", - "only_test_lower_case_table_names" = "${case_type}" - );""" - - sql """ switch test_iceberg_case_sensibility_hadoop """ - sql """ drop database if exists iceberg_case_sensibility_hadoop_db force""" - sql """ create database iceberg_case_sensibility_hadoop_db""" - sql """ use iceberg_case_sensibility_hadoop_db""" - sql """ drop table if exists iceberg_case_sensibility_TBL1""" - sql """ drop table if exists iceberg_case_sensibility_tbl2""" - sql """ create table iceberg_case_sensibility_TBL1(k1 int);""" - sql """ create table iceberg_case_sensibility_tbl2(k1 int);""" - sql """insert into iceberg_case_sensibility_TBL1 values(1)""" - sql """insert into iceberg_case_sensibility_tbl2 values(1)""" - - if (case_type.equals("0")) { - // store in origin, query case sensitive - qt_sql """show tables like "iceberg_case_sensibility_TBL1"""" - qt_sql """show tables like "iceberg_case_sensibility_tbl2"""" - sql """insert into iceberg_case_sensibility_TBL1 values(2)""" - sql """insert into iceberg_case_sensibility_tbl2 values(2)""" - order_qt_sql11 """select * from iceberg_case_sensibility_TBL1"""; - order_qt_sql12 """select * from iceberg_case_sensibility_tbl2"""; - test { - sql "select * from iceberg_case_sensibility_tbl1" - exception "does not exist in database" - } - test { - sql "insert into iceberg_case_sensibility_tbl1 values(1)" - exception "does not exist in database" - } - } else if (case_type.equals("1")) { - // store in lower case, query case insensitive - qt_sql """show tables like "iceberg_case_sensibility_tbl1"""" - qt_sql """show tables like "iceberg_case_sensibility_tbl2"""" - sql """insert into iceberg_case_sensibility_tbl1 values(2)""" - sql """insert into iceberg_case_sensibility_tbl2 values(2)""" - order_qt_sql11 """select * from iceberg_case_sensibility_TBL1"""; - order_qt_sql12 """select * from iceberg_case_sensibility_tbl2"""; - order_qt_sql11 """select * from iceberg_case_sensibility_tbl1"""; - order_qt_sql12 """select * from iceberg_case_sensibility_TBL2"""; - } else if (case_type.equals("2")) { - // store in origin, query case insensitive - qt_sql """show tables like "iceberg_case_sensibility_TBL1"""" - qt_sql """show tables like "iceberg_case_sensibility_tbl2"""" - sql """insert into iceberg_case_sensibility_tbl1 values(2)""" - sql """insert into iceberg_case_sensibility_tbl2 values(2)""" - order_qt_sql11 """select * from iceberg_case_sensibility_TBL1"""; - order_qt_sql12 """select * from iceberg_case_sensibility_tbl2"""; - order_qt_sql11 """select * from iceberg_case_sensibility_tbl1"""; - order_qt_sql12 """select * from iceberg_case_sensibility_TBL2"""; - } - - // hms - sql """drop catalog if exists test_iceberg_case_sensibility_hms""" - sql """create catalog test_iceberg_case_sensibility_hms properties ( - 'type'='iceberg', - 'iceberg.catalog.type'='hms', - 'hive.metastore.uris' = 'thrift://${externalEnvIp}:${hms_port}', - 'fs.defaultFS' = '${default_fs}', - 'warehouse' = '${warehouse}', - "only_test_lower_case_table_names" = "${case_type}" - );""" - - sql """ switch test_iceberg_case_sensibility_hms """ - sql """ drop database if exists iceberg_case_sensibility_hms_db force""" - sql """ create database iceberg_case_sensibility_hms_db""" - sql """ use iceberg_case_sensibility_hms_db""" - sql """ drop table if exists iceberg_case_sensibility_tbl1""" - sql """ drop table if exists iceberg_case_sensibility_tbl2""" - sql """ create table iceberg_case_sensibility_TBL1(k1 int);""" - sql """ create table iceberg_case_sensibility_tbl2(k1 int);""" - sql """insert into iceberg_case_sensibility_tbl1 values(1)""" - sql """insert into iceberg_case_sensibility_tbl2 values(1)""" - - if (case_type.equals("0")) { - // hms is case insensitive, so table name will be stored in lower, query case sensitive - qt_sql """show tables like "iceberg_case_sensibility_tbl1"""" - qt_sql """show tables like "iceberg_case_sensibility_tbl2"""" - order_qt_sql11 """select * from iceberg_case_sensibility_tbl1"""; - order_qt_sql12 """select * from iceberg_case_sensibility_tbl2"""; - test { - sql "select * from iceberg_case_sensibility_TBL1" - exception "does not exist in database" - } - test { - sql "insert into iceberg_case_sensibility_TBL1 values(1)" - exception "does not exist in database" - } - } else if (case_type.equals("1")) { - // store in lower case, query case insensitive - qt_sql """show tables like "iceberg_case_sensibility_tbl1"""" - qt_sql """show tables like "iceberg_case_sensibility_tbl2"""" - sql """insert into iceberg_case_sensibility_TBL1 values(2)""" - sql """insert into iceberg_case_sensibility_tbl2 values(2)""" - order_qt_sql11 """select * from iceberg_case_sensibility_TBL1"""; - order_qt_sql12 """select * from iceberg_case_sensibility_tbl2"""; - order_qt_sql11 """select * from iceberg_case_sensibility_tbl1"""; - order_qt_sql12 """select * from iceberg_case_sensibility_TBL2"""; - } else if (case_type.equals("2")) { - // store in origin, query case insensitive - qt_sql """show tables like "iceberg_case_sensibility_TBL1"""" - qt_sql """show tables like "iceberg_case_sensibility_tbl2"""" - sql """insert into iceberg_case_sensibility_TBL1 values(2)""" - sql """insert into iceberg_case_sensibility_tbl2 values(2)""" - order_qt_sql11 """select * from iceberg_case_sensibility_TBL1"""; - order_qt_sql12 """select * from iceberg_case_sensibility_tbl2"""; - order_qt_sql11 """select * from iceberg_case_sensibility_tbl1"""; - order_qt_sql12 """select * from iceberg_case_sensibility_TBL2"""; - } - } - - } finally { - // sql """set enable_external_table_batch_mode=true""" - } - } -} - - diff --git a/regression-test/suites/external_table_p0/iceberg/test_iceberg_hadoop_case_sensibility.groovy b/regression-test/suites/external_table_p0/iceberg/test_iceberg_hadoop_case_sensibility.groovy new file mode 100644 index 00000000000000..ee8ee903e1f9bd --- /dev/null +++ b/regression-test/suites/external_table_p0/iceberg/test_iceberg_hadoop_case_sensibility.groovy @@ -0,0 +1,343 @@ +// 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_iceberg_hadoop_case_sensibility", "p0,external,doris,external_docker,external_docker_doris") { + String enabled = context.config.otherConfigs.get("enableIcebergTest") + if (enabled == null || !enabled.equalsIgnoreCase("true")) { + return; + } + + String minio_port = context.config.otherConfigs.get("iceberg_minio_port") + String externalEnvIp = context.config.otherConfigs.get("externalEnvIp") + + try { + for (String case_type : ["0", "1", "2"]) { + sql """drop catalog if exists test_iceberg_case_sensibility_hadoop""" + sql """CREATE CATALOG test_iceberg_case_sensibility_hadoop PROPERTIES ( + 'type'='iceberg', + 'iceberg.catalog.type'='hadoop', + 'warehouse' = 's3a://warehouse/wh', + "s3.access_key" = "admin", + "s3.secret_key" = "password", + "s3.endpoint" = "http://${externalEnvIp}:${minio_port}", + "s3.region" = "us-east-1", + "only_test_lower_case_table_names" = "${case_type}" + );""" + sql """switch test_iceberg_case_sensibility_hadoop;""" + // 0. clear env + sql "drop database if exists iceberg_hadoop_case_db1 force"; + sql "drop database if exists iceberg_hadoop_case_db2 force"; + sql "drop database if exists ICEBERG_HADOOP_CASE_DB2 force"; + + // 1. create db + sql """create database iceberg_hadoop_case_db1;""" + sql """create database if not exists iceberg_hadoop_case_db1;""" + sql """create database if not exists iceberg_hadoop_case_db2;""" + sql """create database ICEBERG_HADOOP_CASE_DB2;""" + + qt_sql1 """show databases like "%iceberg_hadoop_case_db1%";""" + qt_sql3 """show databases like "%iceberg_hadoop_case_db2%";""" + qt_sql2 """show databases like "%ICEBERG_HADOOP_CASE_DB2%";""" + test { + sql """create database ICEBERG_HADOOP_CASE_DB2;""" // conflict + exception "database exists" + exception "ICEBERG_HADOOP_CASE_DB2" + } + test { + sql """create database iceberg_hadoop_case_db2;""" // conflict + exception "database exists" + exception "iceberg_hadoop_case_db2" + } + // 2. drop database + qt_sql4 """show databases like "%iceberg_hadoop_case_db1%";""" + sql """drop database iceberg_hadoop_case_db1;""" + qt_sql5 """show databases like "%iceberg_hadoop_case_db1%";""" // empty + + sql """drop database ICEBERG_HADOOP_CASE_DB2;""" + sql """drop database iceberg_hadoop_case_db2;""" + test { + sql """drop database iceberg_hadoop_case_db1""" + exception "database doesn't exist" + exception "iceberg_hadoop_case_db1" + } + test { + sql """drop database iceberg_hadoop_case_db2""" + exception "database doesn't exist" + exception "iceberg_hadoop_case_db2" + } + sql """drop database if exists iceberg_hadoop_case_db2;""" + qt_sql6 """show databases like "%ICEBERG_HADOOP_CASE_DB2%";""" // empty + qt_sql7 """show databases like "%iceberg_hadoop_case_db2%";""" // empty + + // 3. recreate db to test create table + sql """create database iceberg_hadoop_case_db1;""" + sql """create database iceberg_hadoop_case_db2;""" + sql """create database ICEBERG_HADOOP_CASE_DB2;""" + + sql """use iceberg_hadoop_case_db1""" + sql """use iceberg_hadoop_case_db2""" + sql """use ICEBERG_HADOOP_CASE_DB2""" + + test { + sql """create table ICEBERG_HADOOP_CASE_DB1.case_tbl21 (k1 int);""" + exception "Failed to get database: 'ICEBERG_HADOOP_CASE_DB1'" + } + test { + sql """create table if not exists ICEBERG_HADOOP_CASE_DB1.case_tbl21 (k1 int);""" + exception "Failed to get database: 'ICEBERG_HADOOP_CASE_DB1'" + } + sql """create table iceberg_hadoop_case_db1.case_tbl11 (k1 int);""" + sql """create table iceberg_hadoop_case_db2.CASE_TBL22 (k1 int);""" + if (case_type.equals("0")) { + sql """create table iceberg_hadoop_case_db2.case_tbl22 (k1 int);""" + } else { + test { + sql """create table iceberg_hadoop_case_db2.case_tbl22 (k1 int);""" + exception "Table 'case_tbl22' already exists" + } + } + sql """create table ICEBERG_HADOOP_CASE_DB2.case_tbl21 (k1 int);""" + + test { + sql """create table iceberg_hadoop_case_db1.case_tbl11 (k1 int);""" + exception "Table 'case_tbl11' already exists" + } + sql """create table if not exists iceberg_hadoop_case_db1.case_tbl11 (k1 int);""" + if (case_type.equals("0")) { + sql """create table if not exists iceberg_hadoop_case_db1.CASE_TBL11 (k1 int);""" + } else { + test { + sql """create table iceberg_hadoop_case_db1.CASE_TBL11 (k1 int);""" + exception "Table 'CASE_TBL11' already exists" + } + } + qt_sqlx """show tables from iceberg_hadoop_case_db1 like "%CASE_TBL11%"""" + + sql """create table iceberg_hadoop_case_db1.CASE_TBL12 (k1 int);""" + sql """use iceberg_hadoop_case_db1;""" + sql """create table case_tbl13 (k1 int);""" + sql """create table CASE_TBL14 (k1 int);""" + + qt_sql8 """show tables like "%CASE_TBL14%"""" + qt_sql9 """show tables like "%case_tbl14%"""" // empty + qt_sql10 """show tables like "%case_tbl13%"""" + + test { + sql """show tables from ICEBERG_HADOOP_CASE_DB1 like "%case_tbl14%"""" + exception "Unknown database 'ICEBERG_HADOOP_CASE_DB1'" + } + qt_sql11 """show tables from iceberg_hadoop_case_db2 like "%case_tbl14%"""" // empty + qt_sql12 """show tables from iceberg_hadoop_case_db2 like "%case_tbl21%"""" // empty + qt_sql12 """show tables from iceberg_hadoop_case_db2 like "%case_tbl22%"""" + + order_qt_sql13 """select * from information_schema.tables where TABLE_SCHEMA="iceberg_hadoop_case_db2";""" + order_qt_sql14 """select * from information_schema.columns where TABLE_SCHEMA="iceberg_hadoop_case_db2";""" + order_qt_sql131 """select * from information_schema.tables where TABLE_SCHEMA="ICEBERG_HADOOP_CASE_DB2";""" + order_qt_sql141 """select * from information_schema.columns where TABLE_SCHEMA="ICEBERG_HADOOP_CASE_DB2";""" + + // 4. insert + /// full qualified name + test { + sql """insert into ICEBERG_HADOOP_CASE_DB1.CASE_TBL11 values(1);""" + exception "Database [ICEBERG_HADOOP_CASE_DB1] does not exist" + } + test { + sql """insert into ICEBERG_HADOOP_CASE_DB1.case_tbl11 values(1);""" + exception "Database [ICEBERG_HADOOP_CASE_DB1] does not exist." + } + + if (case_type.equals("0")) { + test { + sql """insert into iceberg_hadoop_case_db1.CASE_TBL13 values(1);""" + exception "Table [CASE_TBL13] does not exist in database [iceberg_hadoop_case_db1]" + } + } else if (case_type.equals("1") || case_type.equals("2")) { + sql """insert into iceberg_hadoop_case_db1.CASE_TBL13 values(11);""" + order_qt_sqly """select * from iceberg_hadoop_case_db1.CASE_TBL13""" + } + + + sql """insert into iceberg_hadoop_case_db2.case_tbl22 values(1);""" + sql """insert into iceberg_hadoop_case_db2.CASE_TBL22 values(1);""" + order_qt_sqlz1 """select * from iceberg_hadoop_case_db2.CASE_TBL22""" + order_qt_sqlz2 """select * from iceberg_hadoop_case_db2.case_tbl22""" + test { + sql """select * from ICEBERG_HADOOP_CASE_DB1.CASE_TBL11""" + exception "Database [ICEBERG_HADOOP_CASE_DB1] does not exist" + } + test { + sql """select * from ICEBERG_HADOOP_CASE_DB1.case_tbl13""" + exception "Database [ICEBERG_HADOOP_CASE_DB1] does not exist" + } + + test { + sql """insert overwrite table ICEBERG_HADOOP_CASE_DB1.CASE_TBL11 values(2);""" + exception "Database [ICEBERG_HADOOP_CASE_DB1] does not exist" + } + test { + sql """insert overwrite table ICEBERG_HADOOP_CASE_DB1.case_tbl11 values(2); """ + exception "Database [ICEBERG_HADOOP_CASE_DB1] does not exist" + } + + sql """insert overwrite table iceberg_hadoop_case_db2.case_tbl22 values(20);""" + sql """insert overwrite table iceberg_hadoop_case_db2.CASE_TBL22 values(21);""" + // 0: 20, 1,2: 21 + order_qt_sql16 """select * from iceberg_hadoop_case_db2.case_tbl22;""" + + /// not full qualified + sql """use iceberg_hadoop_case_db1;""" + if (case_type.equals("0")) { + test { + sql """insert into case_tbl12 values(3);""" + exception "Table [case_tbl12] does not exist in database [iceberg_hadoop_case_db1]" + } + sql """insert into CASE_TBL12 values(3);""" + } else if (case_type.equals("1") || case_type.equals("2")) { + sql """insert into CASE_TBL12 values(3);""" + sql """insert into case_tbl12 values(3);""" + } + + if (case_type.equals("0")) { + test { + sql """select * from case_tbl12""" + exception "Table [case_tbl12] does not exist in database [iceberg_hadoop_case_db1]" + } + order_qt_sql17 """select * from CASE_TBL12""" + } else if (case_type.equals("1") || case_type.equals("2")) { + order_qt_sql1511 """select * from CASE_TBL12""" + order_qt_sql1512 """select * from case_tbl12""" + } + + if (case_type.equals("0")) { + test { + sql """insert overwrite table case_tbl12 values(4);""" + exception "Table [case_tbl12] does not exist in database [iceberg_hadoop_case_db1]" + } + sql """insert overwrite table CASE_TBL12 values(4);""" + order_qt_sql18 """select * from CASE_TBL12;""" + } else if (case_type.equals("1") || case_type.equals("2")) { + sql """insert overwrite table case_tbl12 values(4);""" + sql """insert overwrite table CASE_TBL12 values(5);""" + order_qt_sql18 """select * from case_tbl12;""" + order_qt_sql18 """select * from CASE_TBL12;""" + } + + // // 5. truncate(todo not support for iceberg now) + + // 6. drop table + /// full qualified + test { + sql """drop table ICEBERG_HADOOP_CASE_DB1.CASE_TBL11""" + exception "Failed to get database: 'ICEBERG_HADOOP_CASE_DB1' in catalog" + } + test { + sql """drop table ICEBERG_HADOOP_CASE_DB1.case_tbl13""" + exception "Failed to get database: 'ICEBERG_HADOOP_CASE_DB1'" + } + test { + sql """drop table if exists ICEBERG_HADOOP_CASE_DB1.case_tbl22;""" + exception "Failed to get database: 'ICEBERG_HADOOP_CASE_DB1'" + } + if (case_type.equals("0")) { + sql """drop table iceberg_hadoop_case_db2.CASE_TBL22""" + sql """drop table iceberg_hadoop_case_db2.case_tbl22""" + test { + sql """drop table iceberg_hadoop_case_db2.CASE_TBL22""" + exception "Failed to get table: 'CASE_TBL22'" + } + test { + sql """drop table iceberg_hadoop_case_db2.case_tbl22""" + exception "Failed to get table: 'case_tbl22'" + } + } else { + sql """drop table iceberg_hadoop_case_db2.CASE_TBL22""" + test { + sql """drop table iceberg_hadoop_case_db2.case_tbl22""" + exception "Failed to get table: 'case_tbl22'" + } + } + + sql """drop table if exists iceberg_hadoop_case_db2.case_tbl22""" + + test { + sql """select * from iceberg_hadoop_case_db2.case_tbl22;""" + exception "Table [case_tbl22] does not exist in database [iceberg_hadoop_case_db2]" + } + sql """create table iceberg_hadoop_case_db2.case_tbl22 (k1 int);""" // recreate + sql """show tables from iceberg_hadoop_case_db2 like "%case_tbl22%"""" + + sql """insert into iceberg_hadoop_case_db2.case_tbl22 values(5);""" + order_qt_sql21 """select * from iceberg_hadoop_case_db2.case_tbl22;""" + + /// not full qualified + sql """use iceberg_hadoop_case_db1;""" + if (case_type.equals("0")) { + test { + sql """drop table case_tbl12;""" + exception "Failed to get table: 'case_tbl12' in database: iceberg_hadoop_case_db1" + } + sql """drop table CASE_TBL12;""" + } else { + sql """drop table CASE_TBL11;""" + test { + sql """drop table case_tbl11;""" + exception "Failed to get table: 'case_tbl11' in database: iceberg_hadoop_case_db1" + } + } + + test { + sql """select * from iceberg_hadoop_case_db2.case_tbl12;""" + exception "Table [case_tbl12] does not exist in database [iceberg_hadoop_case_db2]" + } + + // 7. re create and insert + sql """create table iceberg_hadoop_case_db2.case_tbl12 (k1 int);""" + sql """insert into iceberg_hadoop_case_db2.case_tbl12 values(6);""" + order_qt_sql22 """select * from iceberg_hadoop_case_db2.case_tbl12;""" + sql """insert overwrite table iceberg_hadoop_case_db2.case_tbl12 values(7);""" + order_qt_sql222 """select * from iceberg_hadoop_case_db2.case_tbl12;""" + + // 8. drop db force + sql """insert into iceberg_hadoop_case_db1.case_tbl13 values(8)""" + sql """insert into iceberg_hadoop_case_db1.CASE_TBL14 values(9)""" + order_qt_sql23 """select * from iceberg_hadoop_case_db1.case_tbl13;""" + order_qt_sql24 """select * from iceberg_hadoop_case_db1.CASE_TBL14;""" + + // use tvf to check data under dir + order_qt_sql25 """select * from s3( + "uri" = "s3a://warehouse/wh/iceberg_hadoop_case_db1/CASE_TBL14/data/*", + "format" = "parquet", + "s3.access_key" = "admin", + "s3.secret_key" = "password", + "s3.region" = "us-east-1", + "s3.endpoint" = "http://${externalEnvIp}:${minio_port}" + ); + """ + sql """drop database iceberg_hadoop_case_db1 force;""" + order_qt_sql26 """select * from s3( + "uri" = "s3a://warehouse/wh/iceberg_hadoop_case_db1/CASE_TBL14/data/*", + "format" = "parquet", + "s3.access_key" = "admin", + "s3.secret_key" = "password", + "s3.region" = "us-east-1", + "s3.endpoint" = "http://${externalEnvIp}:${minio_port}" + ); + """ // empty + } + } finally { + } +} diff --git a/regression-test/suites/external_table_p0/iceberg/test_iceberg_hms_case_sensibility.groovy b/regression-test/suites/external_table_p0/iceberg/test_iceberg_hms_case_sensibility.groovy new file mode 100644 index 00000000000000..01a10c22ebfad1 --- /dev/null +++ b/regression-test/suites/external_table_p0/iceberg/test_iceberg_hms_case_sensibility.groovy @@ -0,0 +1,357 @@ +// 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_iceberg_hms_case_sensibility", "p0,external,doris,external_docker,external_docker_doris") { + String enabled = context.config.otherConfigs.get("enableIcebergTest") + if (enabled == null || !enabled.equalsIgnoreCase("true")) { + return; + } + + String externalEnvIp = context.config.otherConfigs.get("externalEnvIp") + String hms_port = context.config.otherConfigs.get("hive2HmsPort") + String hdfs_port = context.config.otherConfigs.get("hive2HdfsPort") + String default_fs = "hdfs://${externalEnvIp}:${hdfs_port}" + String warehouse = "${default_fs}/warehouse" + + try { + for (String case_type : ["0", "1", "2"]) { + sql """drop catalog if exists test_iceberg_case_sensibility_hms;""" + sql """create catalog test_iceberg_case_sensibility_hms properties ( + 'type'='iceberg', + 'iceberg.catalog.type'='hms', + 'hive.metastore.uris' = 'thrift://${externalEnvIp}:${hms_port}', + 'fs.defaultFS' = '${default_fs}', + 'warehouse' = '${warehouse}', + "only_test_lower_case_table_names" = "${case_type}" + );""" + sql """switch test_iceberg_case_sensibility_hms;""" + // 0. clear env + sql "drop database if exists iceberg_hms_case_db1 force"; + sql "drop database if exists iceberg_hms_case_db2 force"; + + // 1. create db + sql """create database iceberg_hms_case_db1;""" + test { + sql """create database ICEBERG_HMS_CASE_DB1;""" // conflict + exception "Can't create database 'ICEBERG_HMS_CASE_DB1'; database exists" + } + sql """create database ICEBERG_HMS_CASE_DB2;""" + sql """create database if not exists ICEBERG_HMS_CASE_DB1;""" + sql """create database if not exists iceberg_hms_case_db1;""" + sql """create database if not exists iceberg_hms_case_db2;""" + + qt_sql1 """show databases like "%iceberg_hms_case_db1%";""" + qt_sql2 """show databases like "%ICEBERG_HMS_CASE_DB1%";""" // empty + qt_sql3 """show databases like "%iceberg_hms_case_db2%";""" + test { + sql """create database ICEBERG_HMS_CASE_DB2;""" // conflict + exception "database exists" + exception "ICEBERG_HMS_CASE_DB2" + } + test { + sql """create database iceberg_hms_case_db2;""" // conflict + exception "database exists" + exception "iceberg_hms_case_db2" + } + // 2. drop database + test { + sql """drop database ICEBERG_HMS_CASE_DB1""" + exception "database doesn't exist" + exception "ICEBERG_HMS_CASE_DB1" + } + sql """drop database if exists ICEBERG_HMS_CASE_DB1;""" + qt_sql4 """show databases like "%iceberg_hms_case_db1%";""" // still exists + sql """drop database iceberg_hms_case_db1;""" + qt_sql5 """show databases like "%iceberg_hms_case_db1%";""" // empty + + test { + sql """drop database ICEBERG_HMS_CASE_DB2;""" + exception "database doesn't exist" + exception "ICEBERG_HMS_CASE_DB2" + } + sql """drop database iceberg_hms_case_db2;""" + test { + sql """drop database iceberg_hms_case_db1""" + exception "database doesn't exist" + exception "iceberg_hms_case_db1" + } + test { + sql """drop database iceberg_hms_case_db2""" + exception "database doesn't exist" + exception "iceberg_hms_case_db2" + } + sql """drop database if exists iceberg_hms_case_db2;""" + qt_sql6 """show databases like "%iceberg_hms_case_db1%";""" // empty + qt_sql7 """show databases like "%iceberg_hms_case_db2%";""" // empty + + // 3. recreate db to test create table + sql """create database iceberg_hms_case_db1;""" + sql """create database ICEBERG_HMS_CASE_DB2;""" + + test { + sql """use ICEBERG_HMS_CASE_DB2""" + exception "Unknown database 'ICEBERG_HMS_CASE_DB2'" + } + + test { + sql """create table ICEBERG_HMS_CASE_DB2.case_tbl21 (k1 int);""" + exception "Failed to get database: 'ICEBERG_HMS_CASE_DB2'" + } + test { + sql """create table if not exists ICEBERG_HMS_CASE_DB2.case_tbl21 (k1 int);""" + exception "Failed to get database: 'ICEBERG_HMS_CASE_DB2'" + } + sql """create table iceberg_hms_case_db2.case_tbl21 (k1 int);""" + sql """create table iceberg_hms_case_db2.CASE_TBL22 (k1 int);""" + sql """create table iceberg_hms_case_db1.case_tbl11 (k1 int);""" + + test { + sql """create table iceberg_hms_case_db1.case_tbl11 (k1 int);""" + exception "Table 'case_tbl11' already exists" + } + sql """create table if not exists iceberg_hms_case_db1.case_tbl11 (k1 int);""" + sql """create table if not exists iceberg_hms_case_db1.CASE_TBL11 (k1 int);""" + + sql """create table iceberg_hms_case_db1.CASE_TBL12 (k1 int);""" + sql """use iceberg_hms_case_db1;""" + sql """create table case_tbl13 (k1 int);""" + sql """create table CASE_TBL14 (k1 int);""" + + qt_sql8 """show tables like "%CASE_TBL14%"""" // empty + qt_sql9 """show tables like "%case_tbl14%"""" + qt_sql10 """show tables like "%case_tbl13%"""" + + test { + sql """show tables from ICEBERG_HMS_CASE_DB2 like "%case_tbl14%"""" + exception "Unknown database 'ICEBERG_HMS_CASE_DB2'" + } + qt_sql11 """show tables from iceberg_hms_case_db2 like "%case_tbl14%"""" // empty + qt_sql12 """show tables from iceberg_hms_case_db2 like "%case_tbl21%"""" + + order_qt_sql13 """select * from information_schema.tables where TABLE_SCHEMA="iceberg_hms_case_db1";""" + order_qt_sql14 """select * from information_schema.columns where TABLE_SCHEMA="iceberg_hms_case_db1";""" + + // 4. insert + /// full qualified name + test { + sql """insert into ICEBERG_HMS_CASE_DB2.CASE_TBL22 values(1);""" + exception "Database [ICEBERG_HMS_CASE_DB2] does not exist" + } + test { + sql """insert into ICEBERG_HMS_CASE_DB2.case_tbl22 values(1);""" + exception "Database [ICEBERG_HMS_CASE_DB2] does not exist." + } + + if (case_type.equals("0")) { + test { + sql """insert into iceberg_hms_case_db2.CASE_TBL22 values(1);""" + exception "Table [CASE_TBL22] does not exist in database [iceberg_hms_case_db2]" + } + } else if (case_type.equals("1") || case_type.equals("2")) { + sql """insert into iceberg_hms_case_db2.CASE_TBL22 values(11);""" + } + + sql """insert into iceberg_hms_case_db2.case_tbl22 values(1);""" + test { + sql """select * from ICEBERG_HMS_CASE_DB2.CASE_TBL22""" + exception "Database [ICEBERG_HMS_CASE_DB2] does not exist" + } + test { + sql """select * from ICEBERG_HMS_CASE_DB2.case_tbl22""" + exception "Database [ICEBERG_HMS_CASE_DB2] does not exist" + } + + if (case_type.equals("0")) { + test { + sql """select * from iceberg_hms_case_db2.CASE_TBL22""" + exception "Table [CASE_TBL22] does not exist in database [iceberg_hms_case_db2]" + } + } else if (case_type.equals("1") || case_type.equals("2")) { + order_qt_sql141 """select * from iceberg_hms_case_db2.CASE_TBL22""" + } + order_qt_sql15 """select * from iceberg_hms_case_db2.case_tbl22""" + + test { + sql """insert overwrite table ICEBERG_HMS_CASE_DB2.CASE_TBL22 values(2);""" + exception "Database [ICEBERG_HMS_CASE_DB2] does not exist" + } + test { + sql """insert overwrite table ICEBERG_HMS_CASE_DB2.case_tbl22 values(2); """ + exception "Database [ICEBERG_HMS_CASE_DB2] does not exist" + } + if (case_type.equals("0")) { + test { + sql """insert overwrite table iceberg_hms_case_db2.CASE_TBL22 values(2);""" + exception "Table [CASE_TBL22] does not exist in database [iceberg_hms_case_db2]" + } + } else if (case_type.equals("1") || case_type.equals("2")) { + sql """insert overwrite table iceberg_hms_case_db2.CASE_TBL22 values(2);""" + } + + sql """insert overwrite table iceberg_hms_case_db2.case_tbl22 values(2);""" + order_qt_sql16 """select * from iceberg_hms_case_db2.case_tbl22;""" + + /// not full qualified + sql """use iceberg_hms_case_db1;""" + if (case_type.equals("0")) { + test { + sql """insert into CASE_TBL12 values(3);""" + exception "Table [CASE_TBL12] does not exist in database [iceberg_hms_case_db1]" + } + } else if (case_type.equals("1") || case_type.equals("2")) { + sql """insert into CASE_TBL12 values(3);""" + } + + sql """insert into case_tbl12 values(3);""" + if (case_type.equals("0")) { + test { + sql """select * from CASE_TBL12""" + exception "Table [CASE_TBL12] does not exist in database [iceberg_hms_case_db1]" + } + } else if (case_type.equals("1") || case_type.equals("2")) { + order_qt_sql151 """select * from CASE_TBL12""" + } + order_qt_sql17 """select * from case_tbl12""" + + if (case_type.equals("0")) { + test { + sql """insert overwrite table CASE_TBL12 values(4);""" + exception "Table [CASE_TBL12] does not exist in database [iceberg_hms_case_db1]" + } + } else if (case_type.equals("1") || case_type.equals("2")) { + sql """insert overwrite table CASE_TBL12 values(4);""" + } + sql """insert overwrite table case_tbl12 values(4);""" + order_qt_sql18 """select * from case_tbl12;""" + + // // 5. truncate(todo not support for iceberg now) + // /// full qualified + // test { + // sql """truncate table ICEBERG_HMS_CASE_DB2.CASE_TBL22""" + // exception "Unknown database 'ICEBERG_HMS_CASE_DB2'" + // } + // test { + // sql """truncate table ICEBERG_HMS_CASE_DB2.case_tbl22""" + // exception "Unknown database 'ICEBERG_HMS_CASE_DB2'" + // } + // if (case_type.equals("0")) { + // test { + // sql """truncate table iceberg_hms_case_db2.CASE_TBL22""" + // exception "Unknown table 'CASE_TBL22'" + // } + // } else { + // sql """truncate table iceberg_hms_case_db2.CASE_TBL22""" + // } + // sql """truncate table iceberg_hms_case_db2.case_tbl22 ;""" + // qt_sql19 """select * from iceberg_hms_case_db2.case_tbl22;""" // empty + // /// not full qualified + // sql """use iceberg_hms_case_db1;""" + // if (case_type.equals("0")) { + // test { + // sql """truncate table CASE_TBL12;""" + // exception "Unknown table 'CASE_TBL12'" + // } + // } else { + // sql """truncate table CASE_TBL12;""" + // } + // sql """truncate table case_tbl12;""" + // qt_sql20 """select * from case_tbl12;""" // empty + + // 6. drop table + /// full qualified + test { + sql """drop table ICEBERG_HMS_CASE_DB2.CASE_TBL22""" + exception "Failed to get database: 'ICEBERG_HMS_CASE_DB2' in catalog" + } + test { + sql """drop table ICEBERG_HMS_CASE_DB2.case_tbl22""" + exception "Failed to get database: 'ICEBERG_HMS_CASE_DB2'" + } + test { + sql """drop table if exists ICEBERG_HMS_CASE_DB2.case_tbl22;""" + exception "Failed to get database: 'ICEBERG_HMS_CASE_DB2'" + } + if (case_type.equals("0")) { + test { + sql """drop table iceberg_hms_case_db2.CASE_TBL22""" + exception "Failed to get table: 'CASE_TBL22'" + } + sql """drop table iceberg_hms_case_db2.case_tbl22""" + } else { + sql """drop table iceberg_hms_case_db2.CASE_TBL22""" + } + + test { + sql """drop table iceberg_hms_case_db2.case_tbl22""" + exception "Failed to get table: 'case_tbl22'" + } + sql """drop table if exists iceberg_hms_case_db2.case_tbl22""" + + test { + sql """select * from iceberg_hms_case_db2.case_tbl22;""" + exception "Table [case_tbl22] does not exist in database [iceberg_hms_case_db2]" + } + sql """create table iceberg_hms_case_db2.case_tbl22 (k1 int);""" // recreate + sql """insert into iceberg_hms_case_db2.case_tbl22 values(5);""" + order_qt_sql21 """select * from iceberg_hms_case_db2.case_tbl22;""" + + /// not full qualified + sql """use iceberg_hms_case_db1;""" + if (case_type.equals("0")) { + test { + sql """drop table CASE_TBL12;""" + exception "Failed to get table: 'CASE_TBL12' in database: iceberg_hms_case_db1" + } + sql """drop table case_tbl12;""" + } else { + sql """drop table CASE_TBL12;""" + } + + test { + sql """select * from iceberg_hms_case_db2.case_tbl12;""" + exception "Table [case_tbl12] does not exist in database [iceberg_hms_case_db2]" + } + + // 7. re create and insert + sql """create table iceberg_hms_case_db2.case_tbl12 (k1 int);""" + sql """insert into iceberg_hms_case_db2.case_tbl12 values(6);""" + order_qt_sql22 """select * from iceberg_hms_case_db2.case_tbl12;""" + sql """insert overwrite table iceberg_hms_case_db2.case_tbl12 values(7);""" + order_qt_sql222 """select * from iceberg_hms_case_db2.case_tbl12;""" + + // 8. drop db force + sql """insert into iceberg_hms_case_db1.case_tbl13 values(8)""" + sql """insert into iceberg_hms_case_db1.case_tbl14 values(9)""" + order_qt_sql23 """select * from iceberg_hms_case_db1.case_tbl13;""" + order_qt_sql24 """select * from iceberg_hms_case_db1.case_tbl14;""" + + // use tvf to check data under dir + order_qt_sql25 """select * from hdfs( + "uri" = "hdfs://${externalEnvIp}:${hdfs_port}/user/hive/warehouse/iceberg_hms_case_db1.db/case_tbl14/*", + "format" = "orc" + ); + """ + sql """drop database iceberg_hms_case_db1 force;""" + order_qt_sql26 """select * from hdfs( + "uri" = "hdfs://${externalEnvIp}:${hdfs_port}/user/hive/warehouse/iceberg_hms_case_db1.db/case_tbl14/*", + "format" = "orc" + ); + """ // empty + } + } finally { + } +} diff --git a/regression-test/suites/external_table_p0/iceberg/test_iceberg_rest_case_sensibility.groovy b/regression-test/suites/external_table_p0/iceberg/test_iceberg_rest_case_sensibility.groovy new file mode 100644 index 00000000000000..22161de615d4a4 --- /dev/null +++ b/regression-test/suites/external_table_p0/iceberg/test_iceberg_rest_case_sensibility.groovy @@ -0,0 +1,344 @@ +// 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_iceberg_rest_case_sensibility", "p0,external,doris,external_docker,external_docker_doris") { + String enabled = context.config.otherConfigs.get("enableIcebergTest") + if (enabled == null || !enabled.equalsIgnoreCase("true")) { + return; + } + + String rest_port = context.config.otherConfigs.get("iceberg_rest_uri_port") + String minio_port = context.config.otherConfigs.get("iceberg_minio_port") + String externalEnvIp = context.config.otherConfigs.get("externalEnvIp") + + try { + for (String case_type : ["0", "1", "2"]) { + sql """drop catalog if exists test_iceberg_case_sensibility_rest;""" + sql """CREATE CATALOG test_iceberg_case_sensibility_rest PROPERTIES ( + 'type'='iceberg', + 'iceberg.catalog.type'='rest', + 'uri' = 'http://${externalEnvIp}:${rest_port}', + "s3.access_key" = "admin", + "s3.secret_key" = "password", + "s3.endpoint" = "http://${externalEnvIp}:${minio_port}", + "s3.region" = "us-east-1", + "only_test_lower_case_table_names" = "${case_type}" + );""" + sql """switch test_iceberg_case_sensibility_rest;""" + // 0. clear env + sql "drop database if exists iceberg_rest_case_db1 force"; + sql "drop database if exists iceberg_rest_case_db2 force"; + sql "drop database if exists ICEBERG_REST_CASE_DB2 force"; + + // 1. create db + sql """create database iceberg_rest_case_db1;""" + sql """create database if not exists iceberg_rest_case_db1;""" + sql """create database if not exists iceberg_rest_case_db2;""" + sql """create database ICEBERG_REST_CASE_DB2;""" + + qt_sql1 """show databases like "%iceberg_rest_case_db1%";""" + qt_sql3 """show databases like "%iceberg_rest_case_db2%";""" + qt_sql2 """show databases like "%ICEBERG_REST_CASE_DB2%";""" + test { + sql """create database ICEBERG_REST_CASE_DB2;""" // conflict + exception "database exists" + exception "ICEBERG_REST_CASE_DB2" + } + test { + sql """create database iceberg_rest_case_db2;""" // conflict + exception "database exists" + exception "iceberg_rest_case_db2" + } + // 2. drop database + qt_sql4 """show databases like "%iceberg_rest_case_db1%";""" + sql """drop database iceberg_rest_case_db1;""" + qt_sql5 """show databases like "%iceberg_rest_case_db1%";""" // empty + + sql """drop database ICEBERG_REST_CASE_DB2;""" + sql """drop database iceberg_rest_case_db2;""" + test { + sql """drop database iceberg_rest_case_db1""" + exception "database doesn't exist" + exception "iceberg_rest_case_db1" + } + test { + sql """drop database iceberg_rest_case_db2""" + exception "database doesn't exist" + exception "iceberg_rest_case_db2" + } + sql """drop database if exists iceberg_rest_case_db2;""" + qt_sql6 """show databases like "%ICEBERG_REST_CASE_DB2%";""" // empty + qt_sql7 """show databases like "%iceberg_rest_case_db2%";""" // empty + + // 3. recreate db to test create table + sql """create database iceberg_rest_case_db1;""" + sql """create database iceberg_rest_case_db2;""" + sql """create database ICEBERG_REST_CASE_DB2;""" + + sql """use iceberg_rest_case_db1""" + sql """use iceberg_rest_case_db2""" + sql """use ICEBERG_REST_CASE_DB2""" + + test { + sql """create table ICEBERG_REST_CASE_DB1.case_tbl21 (k1 int);""" + exception "Failed to get database: 'ICEBERG_REST_CASE_DB1'" + } + test { + sql """create table if not exists ICEBERG_REST_CASE_DB1.case_tbl21 (k1 int);""" + exception "Failed to get database: 'ICEBERG_REST_CASE_DB1'" + } + sql """create table iceberg_rest_case_db1.case_tbl11 (k1 int);""" + sql """create table iceberg_rest_case_db2.CASE_TBL22 (k1 int);""" + if (case_type.equals("0")) { + sql """create table iceberg_rest_case_db2.case_tbl22 (k1 int);""" + } else { + test { + sql """create table iceberg_rest_case_db2.case_tbl22 (k1 int);""" + exception "Table 'case_tbl22' already exists" + } + } + sql """create table ICEBERG_REST_CASE_DB2.case_tbl21 (k1 int);""" + + test { + sql """create table iceberg_rest_case_db1.case_tbl11 (k1 int);""" + exception "Table 'case_tbl11' already exists" + } + sql """create table if not exists iceberg_rest_case_db1.case_tbl11 (k1 int);""" + if (case_type.equals("0")) { + sql """create table if not exists iceberg_rest_case_db1.CASE_TBL11 (k1 int);""" + } else { + test { + sql """create table iceberg_rest_case_db1.CASE_TBL11 (k1 int);""" + exception "Table 'CASE_TBL11' already exists" + } + } + qt_sqlx """show tables from iceberg_rest_case_db1 like "%CASE_TBL11%"""" + + sql """create table iceberg_rest_case_db1.CASE_TBL12 (k1 int);""" + sql """use iceberg_rest_case_db1;""" + sql """create table case_tbl13 (k1 int);""" + sql """create table CASE_TBL14 (k1 int);""" + + qt_sql8 """show tables like "%CASE_TBL14%"""" + qt_sql9 """show tables like "%case_tbl14%"""" // empty + qt_sql10 """show tables like "%case_tbl13%"""" + + test { + sql """show tables from ICEBERG_REST_CASE_DB1 like "%case_tbl14%"""" + exception "Unknown database 'ICEBERG_REST_CASE_DB1'" + } + qt_sql11 """show tables from iceberg_rest_case_db2 like "%case_tbl14%"""" // empty + qt_sql12 """show tables from iceberg_rest_case_db2 like "%case_tbl21%"""" // empty + qt_sql12 """show tables from iceberg_rest_case_db2 like "%case_tbl22%"""" + + order_qt_sql13 """select * from information_schema.tables where TABLE_SCHEMA="iceberg_rest_case_db2";""" + order_qt_sql14 """select * from information_schema.columns where TABLE_SCHEMA="iceberg_rest_case_db2";""" + order_qt_sql131 """select * from information_schema.tables where TABLE_SCHEMA="ICEBERG_REST_CASE_DB2";""" + order_qt_sql141 """select * from information_schema.columns where TABLE_SCHEMA="ICEBERG_REST_CASE_DB2";""" + + // 4. insert + /// full qualified name + test { + sql """insert into ICEBERG_REST_CASE_DB1.CASE_TBL11 values(1);""" + exception "Database [ICEBERG_REST_CASE_DB1] does not exist" + } + test { + sql """insert into ICEBERG_REST_CASE_DB1.case_tbl11 values(1);""" + exception "Database [ICEBERG_REST_CASE_DB1] does not exist." + } + + if (case_type.equals("0")) { + test { + sql """insert into iceberg_rest_case_db1.CASE_TBL13 values(1);""" + exception "Table [CASE_TBL13] does not exist in database [iceberg_rest_case_db1]" + } + } else if (case_type.equals("1") || case_type.equals("2")) { + sql """insert into iceberg_rest_case_db1.CASE_TBL13 values(11);""" + order_qt_sqly """select * from iceberg_rest_case_db1.CASE_TBL13""" + } + + + sql """insert into iceberg_rest_case_db2.case_tbl22 values(1);""" + sql """insert into iceberg_rest_case_db2.CASE_TBL22 values(1);""" + order_qt_sqlz1 """select * from iceberg_rest_case_db2.CASE_TBL22""" + order_qt_sqlz2 """select * from iceberg_rest_case_db2.case_tbl22""" + test { + sql """select * from ICEBERG_REST_CASE_DB1.CASE_TBL11""" + exception "Database [ICEBERG_REST_CASE_DB1] does not exist" + } + test { + sql """select * from ICEBERG_REST_CASE_DB1.case_tbl13""" + exception "Database [ICEBERG_REST_CASE_DB1] does not exist" + } + + test { + sql """insert overwrite table ICEBERG_REST_CASE_DB1.CASE_TBL11 values(2);""" + exception "Database [ICEBERG_REST_CASE_DB1] does not exist" + } + test { + sql """insert overwrite table ICEBERG_REST_CASE_DB1.case_tbl11 values(2); """ + exception "Database [ICEBERG_REST_CASE_DB1] does not exist" + } + + sql """insert overwrite table iceberg_rest_case_db2.case_tbl22 values(20);""" + sql """insert overwrite table iceberg_rest_case_db2.CASE_TBL22 values(21);""" + // 0: 20, 1,2: 21 + order_qt_sql16 """select * from iceberg_rest_case_db2.case_tbl22;""" + + /// not full qualified + sql """use iceberg_rest_case_db1;""" + if (case_type.equals("0")) { + test { + sql """insert into case_tbl12 values(3);""" + exception "Table [case_tbl12] does not exist in database [iceberg_rest_case_db1]" + } + sql """insert into CASE_TBL12 values(3);""" + } else if (case_type.equals("1") || case_type.equals("2")) { + sql """insert into CASE_TBL12 values(3);""" + sql """insert into case_tbl12 values(3);""" + } + + if (case_type.equals("0")) { + test { + sql """select * from case_tbl12""" + exception "Table [case_tbl12] does not exist in database [iceberg_rest_case_db1]" + } + order_qt_sql17 """select * from CASE_TBL12""" + } else if (case_type.equals("1") || case_type.equals("2")) { + order_qt_sql1511 """select * from CASE_TBL12""" + order_qt_sql1512 """select * from case_tbl12""" + } + + if (case_type.equals("0")) { + test { + sql """insert overwrite table case_tbl12 values(4);""" + exception "Table [case_tbl12] does not exist in database [iceberg_rest_case_db1]" + } + sql """insert overwrite table CASE_TBL12 values(4);""" + order_qt_sql18 """select * from CASE_TBL12;""" + } else if (case_type.equals("1") || case_type.equals("2")) { + sql """insert overwrite table case_tbl12 values(4);""" + sql """insert overwrite table CASE_TBL12 values(5);""" + order_qt_sql18 """select * from case_tbl12;""" + order_qt_sql18 """select * from CASE_TBL12;""" + } + + // // 5. truncate(todo not support for iceberg now) + + // 6. drop table + /// full qualified + test { + sql """drop table ICEBERG_REST_CASE_DB1.CASE_TBL11""" + exception "Failed to get database: 'ICEBERG_REST_CASE_DB1' in catalog" + } + test { + sql """drop table ICEBERG_REST_CASE_DB1.case_tbl13""" + exception "Failed to get database: 'ICEBERG_REST_CASE_DB1'" + } + test { + sql """drop table if exists ICEBERG_REST_CASE_DB1.case_tbl22;""" + exception "Failed to get database: 'ICEBERG_REST_CASE_DB1'" + } + if (case_type.equals("0")) { + sql """drop table iceberg_rest_case_db2.CASE_TBL22""" + sql """drop table iceberg_rest_case_db2.case_tbl22""" + test { + sql """drop table iceberg_rest_case_db2.CASE_TBL22""" + exception "Failed to get table: 'CASE_TBL22'" + } + test { + sql """drop table iceberg_rest_case_db2.case_tbl22""" + exception "Failed to get table: 'case_tbl22'" + } + } else { + sql """drop table iceberg_rest_case_db2.CASE_TBL22""" + test { + sql """drop table iceberg_rest_case_db2.case_tbl22""" + exception "Failed to get table: 'case_tbl22'" + } + } + + sql """drop table if exists iceberg_rest_case_db2.case_tbl22""" + + test { + sql """select * from iceberg_rest_case_db2.case_tbl22;""" + exception "Table [case_tbl22] does not exist in database [iceberg_rest_case_db2]" + } + sql """create table iceberg_rest_case_db2.case_tbl22 (k1 int);""" // recreate + sql """show tables from iceberg_rest_case_db2 like "%case_tbl22%"""" + + sql """insert into iceberg_rest_case_db2.case_tbl22 values(5);""" + order_qt_sql21 """select * from iceberg_rest_case_db2.case_tbl22;""" + + /// not full qualified + sql """use iceberg_rest_case_db1;""" + if (case_type.equals("0")) { + test { + sql """drop table case_tbl12;""" + exception "Failed to get table: 'case_tbl12' in database: iceberg_rest_case_db1" + } + sql """drop table CASE_TBL12;""" + } else { + sql """drop table CASE_TBL11;""" + test { + sql """drop table case_tbl11;""" + exception "Failed to get table: 'case_tbl11' in database: iceberg_rest_case_db1" + } + } + + test { + sql """select * from iceberg_rest_case_db2.case_tbl12;""" + exception "Table [case_tbl12] does not exist in database [iceberg_rest_case_db2]" + } + + // 7. re create and insert + sql """create table iceberg_rest_case_db2.case_tbl12 (k1 int);""" + sql """insert into iceberg_rest_case_db2.case_tbl12 values(6);""" + order_qt_sql22 """select * from iceberg_rest_case_db2.case_tbl12;""" + sql """insert overwrite table iceberg_rest_case_db2.case_tbl12 values(7);""" + order_qt_sql222 """select * from iceberg_rest_case_db2.case_tbl12;""" + + // 8. drop db force + sql """insert into iceberg_rest_case_db1.case_tbl13 values(8)""" + sql """insert into iceberg_rest_case_db1.CASE_TBL14 values(9)""" + order_qt_sql23 """select * from iceberg_rest_case_db1.case_tbl13;""" + order_qt_sql24 """select * from iceberg_rest_case_db1.CASE_TBL14;""" + + // use tvf to check data under dir + order_qt_sql25 """select * from s3( + "uri" = "s3a://warehouse/wh/iceberg_rest_case_db1/CASE_TBL14/data/*", + "format" = "parquet", + "s3.access_key" = "admin", + "s3.secret_key" = "password", + "s3.region" = "us-east-1", + "s3.endpoint" = "http://${externalEnvIp}:${minio_port}" + ); + """ + sql """drop database iceberg_rest_case_db1 force;""" + order_qt_sql26 """select * from s3( + "uri" = "s3a://warehouse/wh/iceberg_rest_case_db1/CASE_TBL14/data/*", + "format" = "parquet", + "s3.access_key" = "admin", + "s3.secret_key" = "password", + "s3.region" = "us-east-1", + "s3.endpoint" = "http://${externalEnvIp}:${minio_port}" + ); + """ // empty + } + } finally { + } +} diff --git a/regression-test/suites/external_table_p0/iceberg/test_iceberg_show_create.groovy b/regression-test/suites/external_table_p0/iceberg/test_iceberg_show_create.groovy index 23705c5494ddf0..368e7fa2fa445e 100644 --- a/regression-test/suites/external_table_p0/iceberg/test_iceberg_show_create.groovy +++ b/regression-test/suites/external_table_p0/iceberg/test_iceberg_show_create.groovy @@ -47,9 +47,8 @@ suite("test_iceberg_show_create", "p0,external,doris,external_docker,external_do String db2 = "test_db2" String tb1 = "test_tb1" - sql """ drop table if exists ${db1}.${tb1} """ - sql """ drop database if exists ${db1} """ - sql """ drop database if exists ${db2} """ + sql """ drop database if exists ${db1} force""" + sql """ drop database if exists ${db2} force""" sql """ create database ${db1} properties ('location'='${warehouse}/other_location') """ sql """ create database ${db2} """ diff --git a/regression-test/suites/external_table_p0/iceberg/write/test_iceberg_insert_overwrite_with_empty_table.groovy b/regression-test/suites/external_table_p0/iceberg/write/test_iceberg_insert_overwrite_with_empty_table.groovy index b1ead66b70f000..625a836ff30872 100644 --- a/regression-test/suites/external_table_p0/iceberg/write/test_iceberg_insert_overwrite_with_empty_table.groovy +++ b/regression-test/suites/external_table_p0/iceberg/write/test_iceberg_insert_overwrite_with_empty_table.groovy @@ -46,11 +46,8 @@ suite("test_iceberg_insert_overwrite_with_empty_table", "p0,external,doris,exter String tb2 = db1 + "_tb2" String tb3 = db1 + "_tb3" - sql """ drop table if exists ${db1}.${tb1} """ - sql """ drop table if exists ${db1}.${tb2} """ - sql """ drop table if exists ${db1}.${tb3} """ - - sql """ create database if not exists ${db1} """ + sql """ drop database if exists ${db1} force""" + sql """ create database ${db1} """ sql """ create table ${db1}.${tb1} (id int, val int) partition by list (val)() """ sql """ create table ${db1}.${tb2} (id int, val int) """ sql """ create table ${db1}.${tb3} (id int, val int) """ diff --git a/regression-test/suites/mtmv_p0/test_iceberg_mtmv.groovy b/regression-test/suites/mtmv_p0/test_iceberg_mtmv.groovy index 30b9e8e9f7329c..f0019f2cb8b506 100644 --- a/regression-test/suites/mtmv_p0/test_iceberg_mtmv.groovy +++ b/regression-test/suites/mtmv_p0/test_iceberg_mtmv.groovy @@ -94,9 +94,8 @@ suite("test_iceberg_mtmv", "p0,external,iceberg,external_docker,external_docker_ sql """create database if not exists ${dbName}""" sql """use ${dbName}""" - sql """drop table if exists ${catalog_name}.${icebergDb}.${icebergTable1}""" - sql """drop table if exists ${catalog_name}.${icebergDb}.${icebergTable2}""" - sql """create database if not exists ${catalog_name}.${icebergDb}""" + sql """drop database if exists ${catalog_name}.${icebergDb} force""" + sql """create database ${catalog_name}.${icebergDb}""" sql """ CREATE TABLE ${catalog_name}.${icebergDb}.${icebergTable1} ( ts DATETIME,